mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-05-21 07:20:41 +00:00
feat: CREATE OR REPLACE FLOW (#5001)
* feat: Replace flow * refactor: better show create flow&tests: better check * tests: sqlness result update * tests: unit test for update * refactor: cmp with raw bytes * refactor: rename * refactor: per review
This commit is contained in:
@@ -28,6 +28,7 @@ use common_procedure::{
|
||||
use common_telemetry::info;
|
||||
use common_telemetry::tracing_context::TracingContext;
|
||||
use futures::future::join_all;
|
||||
use futures::TryStreamExt;
|
||||
use itertools::Itertools;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{ensure, ResultExt};
|
||||
@@ -43,7 +44,7 @@ use crate::instruction::{CacheIdent, CreateFlow};
|
||||
use crate::key::flow::flow_info::FlowInfoValue;
|
||||
use crate::key::flow::flow_route::FlowRouteValue;
|
||||
use crate::key::table_name::TableNameKey;
|
||||
use crate::key::{FlowId, FlowPartitionId};
|
||||
use crate::key::{DeserializedValueWithBytes, FlowId, FlowPartitionId};
|
||||
use crate::lock_key::{CatalogLock, FlowNameLock, TableNameLock};
|
||||
use crate::peer::Peer;
|
||||
use crate::rpc::ddl::{CreateFlowTask, QueryContext};
|
||||
@@ -75,6 +76,7 @@ impl CreateFlowProcedure {
|
||||
source_table_ids: vec![],
|
||||
query_context,
|
||||
state: CreateFlowState::Prepare,
|
||||
prev_flow_info_value: None,
|
||||
},
|
||||
}
|
||||
}
|
||||
@@ -90,6 +92,7 @@ impl CreateFlowProcedure {
|
||||
let flow_name = &self.data.task.flow_name;
|
||||
let sink_table_name = &self.data.task.sink_table_name;
|
||||
let create_if_not_exists = self.data.task.create_if_not_exists;
|
||||
let or_replace = self.data.task.or_replace;
|
||||
|
||||
let flow_name_value = self
|
||||
.context
|
||||
@@ -98,16 +101,56 @@ impl CreateFlowProcedure {
|
||||
.get(catalog_name, flow_name)
|
||||
.await?;
|
||||
|
||||
if create_if_not_exists && or_replace {
|
||||
// this is forbidden because not clear what does that mean exactly
|
||||
return error::UnsupportedSnafu {
|
||||
operation: "Create flow with both `IF NOT EXISTS` and `OR REPLACE`".to_string(),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
if let Some(value) = flow_name_value {
|
||||
ensure!(
|
||||
create_if_not_exists,
|
||||
create_if_not_exists || or_replace,
|
||||
error::FlowAlreadyExistsSnafu {
|
||||
flow_name: format_full_flow_name(catalog_name, flow_name),
|
||||
}
|
||||
);
|
||||
|
||||
let flow_id = value.flow_id();
|
||||
return Ok(Status::done_with_output(flow_id));
|
||||
if create_if_not_exists {
|
||||
info!("Flow already exists, flow_id: {}", flow_id);
|
||||
return Ok(Status::done_with_output(flow_id));
|
||||
}
|
||||
|
||||
let flow_id = value.flow_id();
|
||||
let peers = self
|
||||
.context
|
||||
.flow_metadata_manager
|
||||
.flow_route_manager()
|
||||
.routes(flow_id)
|
||||
.map_ok(|(_, value)| value.peer)
|
||||
.try_collect::<Vec<_>>()
|
||||
.await?;
|
||||
self.data.flow_id = Some(flow_id);
|
||||
self.data.peers = peers;
|
||||
info!("Replacing flow, flow_id: {}", flow_id);
|
||||
|
||||
let flow_info_value = self
|
||||
.context
|
||||
.flow_metadata_manager
|
||||
.flow_info_manager()
|
||||
.get_raw(flow_id)
|
||||
.await?;
|
||||
|
||||
ensure!(
|
||||
flow_info_value.is_some(),
|
||||
error::FlowNotFoundSnafu {
|
||||
flow_name: format_full_flow_name(catalog_name, flow_name),
|
||||
}
|
||||
);
|
||||
|
||||
self.data.prev_flow_info_value = flow_info_value;
|
||||
}
|
||||
|
||||
// Ensures sink table doesn't exist.
|
||||
@@ -128,7 +171,9 @@ impl CreateFlowProcedure {
|
||||
}
|
||||
|
||||
self.collect_source_tables().await?;
|
||||
self.allocate_flow_id().await?;
|
||||
if self.data.flow_id.is_none() {
|
||||
self.allocate_flow_id().await?;
|
||||
}
|
||||
self.data.state = CreateFlowState::CreateFlows;
|
||||
|
||||
Ok(Status::executing(true))
|
||||
@@ -153,7 +198,10 @@ impl CreateFlowProcedure {
|
||||
.map_err(add_peer_context_if_needed(peer.clone()))
|
||||
});
|
||||
}
|
||||
|
||||
info!(
|
||||
"Creating flow({:?}) on flownodes with peers={:?}",
|
||||
self.data.flow_id, self.data.peers
|
||||
);
|
||||
join_all(create_flow)
|
||||
.await
|
||||
.into_iter()
|
||||
@@ -170,18 +218,29 @@ impl CreateFlowProcedure {
|
||||
async fn on_create_metadata(&mut self) -> Result<Status> {
|
||||
// Safety: The flow id must be allocated.
|
||||
let flow_id = self.data.flow_id.unwrap();
|
||||
// TODO(weny): Support `or_replace`.
|
||||
let (flow_info, flow_routes) = (&self.data).into();
|
||||
self.context
|
||||
.flow_metadata_manager
|
||||
.create_flow_metadata(flow_id, flow_info, flow_routes)
|
||||
.await?;
|
||||
info!("Created flow metadata for flow {flow_id}");
|
||||
if let Some(prev_flow_value) = self.data.prev_flow_info_value.as_ref()
|
||||
&& self.data.task.or_replace
|
||||
{
|
||||
self.context
|
||||
.flow_metadata_manager
|
||||
.update_flow_metadata(flow_id, prev_flow_value, &flow_info, flow_routes)
|
||||
.await?;
|
||||
info!("Replaced flow metadata for flow {flow_id}");
|
||||
} else {
|
||||
self.context
|
||||
.flow_metadata_manager
|
||||
.create_flow_metadata(flow_id, flow_info, flow_routes)
|
||||
.await?;
|
||||
info!("Created flow metadata for flow {flow_id}");
|
||||
}
|
||||
|
||||
self.data.state = CreateFlowState::InvalidateFlowCache;
|
||||
Ok(Status::executing(true))
|
||||
}
|
||||
|
||||
async fn on_broadcast(&mut self) -> Result<Status> {
|
||||
debug_assert!(self.data.state == CreateFlowState::InvalidateFlowCache);
|
||||
// Safety: The flow id must be allocated.
|
||||
let flow_id = self.data.flow_id.unwrap();
|
||||
let ctx = Context {
|
||||
@@ -192,10 +251,13 @@ impl CreateFlowProcedure {
|
||||
.cache_invalidator
|
||||
.invalidate(
|
||||
&ctx,
|
||||
&[CacheIdent::CreateFlow(CreateFlow {
|
||||
source_table_ids: self.data.source_table_ids.clone(),
|
||||
flownodes: self.data.peers.clone(),
|
||||
})],
|
||||
&[
|
||||
CacheIdent::CreateFlow(CreateFlow {
|
||||
source_table_ids: self.data.source_table_ids.clone(),
|
||||
flownodes: self.data.peers.clone(),
|
||||
}),
|
||||
CacheIdent::FlowId(flow_id),
|
||||
],
|
||||
)
|
||||
.await?;
|
||||
|
||||
@@ -270,6 +332,9 @@ pub struct CreateFlowData {
|
||||
pub(crate) peers: Vec<Peer>,
|
||||
pub(crate) source_table_ids: Vec<TableId>,
|
||||
pub(crate) query_context: QueryContext,
|
||||
/// For verify if prev value is consistent when need to update flow metadata.
|
||||
/// only set when `or_replace` is true.
|
||||
pub(crate) prev_flow_info_value: Option<DeserializedValueWithBytes<FlowInfoValue>>,
|
||||
}
|
||||
|
||||
impl From<&CreateFlowData> for CreateRequest {
|
||||
@@ -284,9 +349,9 @@ impl From<&CreateFlowData> for CreateRequest {
|
||||
.map(|table_id| api::v1::TableId { id: *table_id })
|
||||
.collect_vec(),
|
||||
sink_table_name: Some(value.task.sink_table_name.clone().into()),
|
||||
// Always be true
|
||||
// Always be true to ensure idempotent in case of retry
|
||||
create_if_not_exists: true,
|
||||
or_replace: true,
|
||||
or_replace: value.task.or_replace,
|
||||
expire_after: value.task.expire_after.map(|value| ExpireAfter { value }),
|
||||
comment: value.task.comment.clone(),
|
||||
sql: value.task.sql.clone(),
|
||||
|
||||
@@ -655,10 +655,17 @@ async fn handle_create_flow_task(
|
||||
procedure_id: &procedure_id,
|
||||
err_msg: "downcast to `u32`",
|
||||
})?);
|
||||
info!(
|
||||
"Flow {}.{}({flow_id}) is created via procedure_id {id:?}",
|
||||
create_flow_task.catalog_name, create_flow_task.flow_name,
|
||||
);
|
||||
if !create_flow_task.or_replace {
|
||||
info!(
|
||||
"Flow {}.{}({flow_id}) is created via procedure_id {id:?}",
|
||||
create_flow_task.catalog_name, create_flow_task.flow_name,
|
||||
);
|
||||
} else {
|
||||
info!(
|
||||
"Flow {}.{}({flow_id}) is replaced via procedure_id {id:?}",
|
||||
create_flow_task.catalog_name, create_flow_task.flow_name,
|
||||
);
|
||||
}
|
||||
|
||||
Ok(SubmitDdlTaskResponse {
|
||||
key: procedure_id.into(),
|
||||
|
||||
@@ -38,7 +38,7 @@ use crate::key::flow::flow_name::FlowNameManager;
|
||||
use crate::key::flow::flownode_flow::FlownodeFlowManager;
|
||||
pub use crate::key::flow::table_flow::{TableFlowManager, TableFlowManagerRef};
|
||||
use crate::key::txn_helper::TxnOpGetResponseSet;
|
||||
use crate::key::{FlowId, MetadataKey};
|
||||
use crate::key::{DeserializedValueWithBytes, FlowId, MetadataKey};
|
||||
use crate::kv_backend::txn::Txn;
|
||||
use crate::kv_backend::KvBackendRef;
|
||||
use crate::rpc::store::BatchDeleteRequest;
|
||||
@@ -230,6 +230,102 @@ impl FlowMetadataManager {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Update metadata for flow and returns an error if old metadata IS NOT exists.
|
||||
pub async fn update_flow_metadata(
|
||||
&self,
|
||||
flow_id: FlowId,
|
||||
current_flow_info: &DeserializedValueWithBytes<FlowInfoValue>,
|
||||
new_flow_info: &FlowInfoValue,
|
||||
flow_routes: Vec<(FlowPartitionId, FlowRouteValue)>,
|
||||
) -> Result<()> {
|
||||
let (create_flow_flow_name_txn, on_create_flow_flow_name_failure) =
|
||||
self.flow_name_manager.build_update_txn(
|
||||
&new_flow_info.catalog_name,
|
||||
&new_flow_info.flow_name,
|
||||
flow_id,
|
||||
)?;
|
||||
|
||||
let (create_flow_txn, on_create_flow_failure) =
|
||||
self.flow_info_manager
|
||||
.build_update_txn(flow_id, current_flow_info, new_flow_info)?;
|
||||
|
||||
let create_flow_routes_txn = self
|
||||
.flow_route_manager
|
||||
.build_create_txn(flow_id, flow_routes.clone())?;
|
||||
|
||||
let create_flownode_flow_txn = self
|
||||
.flownode_flow_manager
|
||||
.build_create_txn(flow_id, new_flow_info.flownode_ids().clone());
|
||||
|
||||
let create_table_flow_txn = self.table_flow_manager.build_create_txn(
|
||||
flow_id,
|
||||
flow_routes
|
||||
.into_iter()
|
||||
.map(|(partition_id, route)| (partition_id, TableFlowValue { peer: route.peer }))
|
||||
.collect(),
|
||||
new_flow_info.source_table_ids(),
|
||||
)?;
|
||||
|
||||
let txn = Txn::merge_all(vec![
|
||||
create_flow_flow_name_txn,
|
||||
create_flow_txn,
|
||||
create_flow_routes_txn,
|
||||
create_flownode_flow_txn,
|
||||
create_table_flow_txn,
|
||||
]);
|
||||
info!(
|
||||
"Creating flow {}.{}({}), with {} txn operations",
|
||||
new_flow_info.catalog_name,
|
||||
new_flow_info.flow_name,
|
||||
flow_id,
|
||||
txn.max_operations()
|
||||
);
|
||||
|
||||
let mut resp = self.kv_backend.txn(txn).await?;
|
||||
if !resp.succeeded {
|
||||
let mut set = TxnOpGetResponseSet::from(&mut resp.responses);
|
||||
let remote_flow_flow_name =
|
||||
on_create_flow_flow_name_failure(&mut set)?.with_context(|| {
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: format!(
|
||||
"Reads the empty flow name during the updating flow, flow_id: {flow_id}"
|
||||
),
|
||||
}
|
||||
})?;
|
||||
|
||||
if remote_flow_flow_name.flow_id() != flow_id {
|
||||
info!(
|
||||
"Trying to updating flow {}.{}({}), but flow({}) already exists with a different flow id",
|
||||
new_flow_info.catalog_name,
|
||||
new_flow_info.flow_name,
|
||||
flow_id,
|
||||
remote_flow_flow_name.flow_id()
|
||||
);
|
||||
|
||||
return error::UnexpectedSnafu {
|
||||
err_msg: format!(
|
||||
"Reads different flow id when updating flow({2}.{3}), prev flow id = {0}, updating with flow id = {1}",
|
||||
remote_flow_flow_name.flow_id(),
|
||||
flow_id,
|
||||
new_flow_info.catalog_name,
|
||||
new_flow_info.flow_name,
|
||||
),
|
||||
}.fail();
|
||||
}
|
||||
|
||||
let remote_flow =
|
||||
on_create_flow_failure(&mut set)?.with_context(|| error::UnexpectedSnafu {
|
||||
err_msg: format!(
|
||||
"Reads the empty flow during the updating flow, flow_id: {flow_id}"
|
||||
),
|
||||
})?;
|
||||
let op_name = "updating flow";
|
||||
ensure_values!(*remote_flow, new_flow_info.clone(), op_name);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn flow_metadata_keys(&self, flow_id: FlowId, flow_value: &FlowInfoValue) -> Vec<Vec<u8>> {
|
||||
let source_table_ids = flow_value.source_table_ids();
|
||||
let mut keys =
|
||||
@@ -560,4 +656,222 @@ mod tests {
|
||||
// Ensures all keys are deleted
|
||||
assert!(mem_kv.is_empty())
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_update_flow_metadata() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
let flow_metadata_manager = FlowMetadataManager::new(mem_kv.clone());
|
||||
let flow_id = 10;
|
||||
let flow_value = test_flow_info_value(
|
||||
"flow",
|
||||
[(0, 1u64), (1, 2u64)].into(),
|
||||
vec![1024, 1025, 1026],
|
||||
);
|
||||
let flow_routes = vec![
|
||||
(
|
||||
1u32,
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(1),
|
||||
},
|
||||
),
|
||||
(
|
||||
2,
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(2),
|
||||
},
|
||||
),
|
||||
];
|
||||
flow_metadata_manager
|
||||
.create_flow_metadata(flow_id, flow_value.clone(), flow_routes.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let new_flow_value = {
|
||||
let mut tmp = flow_value.clone();
|
||||
tmp.raw_sql = "new".to_string();
|
||||
tmp
|
||||
};
|
||||
|
||||
// Update flow instead
|
||||
flow_metadata_manager
|
||||
.update_flow_metadata(
|
||||
flow_id,
|
||||
&DeserializedValueWithBytes::from_inner(flow_value.clone()),
|
||||
&new_flow_value,
|
||||
flow_routes.clone(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let got = flow_metadata_manager
|
||||
.flow_info_manager()
|
||||
.get(flow_id)
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
let routes = flow_metadata_manager
|
||||
.flow_route_manager()
|
||||
.routes(flow_id)
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
routes,
|
||||
vec![
|
||||
(
|
||||
FlowRouteKey::new(flow_id, 1),
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(1),
|
||||
},
|
||||
),
|
||||
(
|
||||
FlowRouteKey::new(flow_id, 2),
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(2),
|
||||
},
|
||||
),
|
||||
]
|
||||
);
|
||||
assert_eq!(got, new_flow_value);
|
||||
let flows = flow_metadata_manager
|
||||
.flownode_flow_manager()
|
||||
.flows(1)
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(flows, vec![(flow_id, 0)]);
|
||||
for table_id in [1024, 1025, 1026] {
|
||||
let nodes = flow_metadata_manager
|
||||
.table_flow_manager()
|
||||
.flows(table_id)
|
||||
.try_collect::<Vec<_>>()
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(
|
||||
nodes,
|
||||
vec![
|
||||
(
|
||||
TableFlowKey::new(table_id, 1, flow_id, 1),
|
||||
TableFlowValue {
|
||||
peer: Peer::empty(1)
|
||||
}
|
||||
),
|
||||
(
|
||||
TableFlowKey::new(table_id, 2, flow_id, 2),
|
||||
TableFlowValue {
|
||||
peer: Peer::empty(2)
|
||||
}
|
||||
)
|
||||
]
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_update_flow_metadata_flow_replace_diff_id_err() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
let flow_metadata_manager = FlowMetadataManager::new(mem_kv);
|
||||
let flow_id = 10;
|
||||
let flow_value = test_flow_info_value("flow", [(0, 1u64)].into(), vec![1024, 1025, 1026]);
|
||||
let flow_routes = vec![
|
||||
(
|
||||
1u32,
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(1),
|
||||
},
|
||||
),
|
||||
(
|
||||
2,
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(2),
|
||||
},
|
||||
),
|
||||
];
|
||||
flow_metadata_manager
|
||||
.create_flow_metadata(flow_id, flow_value.clone(), flow_routes.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
// update again with same flow id
|
||||
flow_metadata_manager
|
||||
.update_flow_metadata(
|
||||
flow_id,
|
||||
&DeserializedValueWithBytes::from_inner(flow_value.clone()),
|
||||
&flow_value,
|
||||
flow_routes.clone(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
// update again with wrong flow id, expected error
|
||||
let err = flow_metadata_manager
|
||||
.update_flow_metadata(
|
||||
flow_id + 1,
|
||||
&DeserializedValueWithBytes::from_inner(flow_value.clone()),
|
||||
&flow_value,
|
||||
flow_routes,
|
||||
)
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, error::Error::Unexpected { .. });
|
||||
assert!(err
|
||||
.to_string()
|
||||
.contains("Reads different flow id when updating flow"));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_update_flow_metadata_unexpected_err_prev_value_diff() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
let flow_metadata_manager = FlowMetadataManager::new(mem_kv);
|
||||
let flow_id = 10;
|
||||
let catalog_name = "greptime";
|
||||
let flow_value = test_flow_info_value("flow", [(0, 1u64)].into(), vec![1024, 1025, 1026]);
|
||||
let flow_routes = vec![
|
||||
(
|
||||
1u32,
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(1),
|
||||
},
|
||||
),
|
||||
(
|
||||
2,
|
||||
FlowRouteValue {
|
||||
peer: Peer::empty(2),
|
||||
},
|
||||
),
|
||||
];
|
||||
flow_metadata_manager
|
||||
.create_flow_metadata(flow_id, flow_value.clone(), flow_routes.clone())
|
||||
.await
|
||||
.unwrap();
|
||||
// Creates again.
|
||||
let another_sink_table_name = TableName {
|
||||
catalog_name: catalog_name.to_string(),
|
||||
schema_name: "my_schema".to_string(),
|
||||
table_name: "another_sink_table".to_string(),
|
||||
};
|
||||
let flow_value = FlowInfoValue {
|
||||
catalog_name: "greptime".to_string(),
|
||||
flow_name: "flow".to_string(),
|
||||
source_table_ids: vec![1024, 1025, 1026],
|
||||
sink_table_name: another_sink_table_name,
|
||||
flownode_ids: [(0, 1u64)].into(),
|
||||
raw_sql: "raw".to_string(),
|
||||
expire_after: Some(300),
|
||||
comment: "hi".to_string(),
|
||||
options: Default::default(),
|
||||
};
|
||||
let err = flow_metadata_manager
|
||||
.update_flow_metadata(
|
||||
flow_id,
|
||||
&DeserializedValueWithBytes::from_inner(flow_value.clone()),
|
||||
&flow_value,
|
||||
flow_routes.clone(),
|
||||
)
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(
|
||||
err.to_string().contains("Reads the different value"),
|
||||
"error: {:?}",
|
||||
err
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -26,7 +26,7 @@ use crate::error::{self, Result};
|
||||
use crate::key::flow::FlowScoped;
|
||||
use crate::key::txn_helper::TxnOpGetResponseSet;
|
||||
use crate::key::{DeserializedValueWithBytes, FlowId, FlowPartitionId, MetadataKey, MetadataValue};
|
||||
use crate::kv_backend::txn::Txn;
|
||||
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
|
||||
use crate::kv_backend::KvBackendRef;
|
||||
use crate::FlownodeId;
|
||||
|
||||
@@ -196,6 +196,19 @@ impl FlowInfoManager {
|
||||
.transpose()
|
||||
}
|
||||
|
||||
/// Returns the [FlowInfoValue] with original bytes of specified `flow_id`.
|
||||
pub async fn get_raw(
|
||||
&self,
|
||||
flow_id: FlowId,
|
||||
) -> Result<Option<DeserializedValueWithBytes<FlowInfoValue>>> {
|
||||
let key = FlowInfoKey::new(flow_id).to_bytes();
|
||||
self.kv_backend
|
||||
.get(&key)
|
||||
.await?
|
||||
.map(|x| DeserializedValueWithBytes::from_inner_slice(&x.value))
|
||||
.transpose()
|
||||
}
|
||||
|
||||
/// Builds a create flow transaction.
|
||||
/// It is expected that the `__flow/info/{flow_id}` wasn't occupied.
|
||||
/// Otherwise, the transaction will retrieve existing value.
|
||||
@@ -215,6 +228,36 @@ impl FlowInfoManager {
|
||||
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
|
||||
))
|
||||
}
|
||||
|
||||
/// Builds a update flow transaction.
|
||||
/// It is expected that the `__flow/info/{flow_id}` IS ALREADY occupied and equal to `prev_flow_value`,
|
||||
/// but the new value can be the same, so to allow replace operation to happen even when the value is the same.
|
||||
/// Otherwise, the transaction will retrieve existing value and fail.
|
||||
pub(crate) fn build_update_txn(
|
||||
&self,
|
||||
flow_id: FlowId,
|
||||
current_flow_value: &DeserializedValueWithBytes<FlowInfoValue>,
|
||||
new_flow_value: &FlowInfoValue,
|
||||
) -> Result<(
|
||||
Txn,
|
||||
impl FnOnce(&mut TxnOpGetResponseSet) -> FlowInfoDecodeResult,
|
||||
)> {
|
||||
let key = FlowInfoKey::new(flow_id).to_bytes();
|
||||
let raw_value = new_flow_value.try_as_raw_value()?;
|
||||
let prev_value = current_flow_value.get_raw_bytes();
|
||||
let txn = Txn::new()
|
||||
.when(vec![
|
||||
Compare::new(key.clone(), CompareOp::NotEqual, None),
|
||||
Compare::new(key.clone(), CompareOp::Equal, Some(prev_value)),
|
||||
])
|
||||
.and_then(vec![TxnOp::Put(key.clone(), raw_value)])
|
||||
.or_else(vec![TxnOp::Get(key.clone())]);
|
||||
|
||||
Ok((
|
||||
txn,
|
||||
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
||||
@@ -26,7 +26,7 @@ use crate::key::txn_helper::TxnOpGetResponseSet;
|
||||
use crate::key::{
|
||||
BytesAdapter, DeserializedValueWithBytes, FlowId, MetadataKey, MetadataValue, NAME_PATTERN,
|
||||
};
|
||||
use crate::kv_backend::txn::Txn;
|
||||
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
|
||||
use crate::kv_backend::KvBackendRef;
|
||||
use crate::range_stream::{PaginationStream, DEFAULT_PAGE_SIZE};
|
||||
use crate::rpc::store::RangeRequest;
|
||||
@@ -237,6 +237,37 @@ impl FlowNameManager {
|
||||
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(raw_key)),
|
||||
))
|
||||
}
|
||||
|
||||
/// Builds a update flow name transaction. Which doesn't change either the name or id, just checking if they are the same.
|
||||
/// It's expected that the `__flow/name/{catalog}/{flow_name}` IS already occupied,
|
||||
/// and both flow name and flow id is the same.
|
||||
/// Otherwise, the transaction will retrieve existing value(and fail).
|
||||
pub fn build_update_txn(
|
||||
&self,
|
||||
catalog_name: &str,
|
||||
flow_name: &str,
|
||||
flow_id: FlowId,
|
||||
) -> Result<(
|
||||
Txn,
|
||||
impl FnOnce(&mut TxnOpGetResponseSet) -> FlowNameDecodeResult,
|
||||
)> {
|
||||
let key = FlowNameKey::new(catalog_name, flow_name);
|
||||
let raw_key = key.to_bytes();
|
||||
let flow_flow_name_value = FlowNameValue::new(flow_id);
|
||||
let raw_value = flow_flow_name_value.try_as_raw_value()?;
|
||||
let txn = Txn::new()
|
||||
.when(vec![Compare::new(
|
||||
raw_key.clone(),
|
||||
CompareOp::Equal,
|
||||
Some(raw_value),
|
||||
)])
|
||||
.or_else(vec![TxnOp::Get(raw_key.clone())]);
|
||||
|
||||
Ok((
|
||||
txn,
|
||||
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(raw_key)),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
||||
Reference in New Issue
Block a user