mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-23 06:30:05 +00:00
Compare commits
2 Commits
async_deco
...
release/v0
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
60852af5f8 | ||
|
|
557e850d87 |
@@ -47,7 +47,6 @@ runs:
|
||||
shell: pwsh
|
||||
run: make test sqlness-test
|
||||
env:
|
||||
RUSTUP_WINDOWS_PATH_ADD_BIN: 1 # Workaround for https://github.com/nextest-rs/nextest/issues/1493
|
||||
RUST_BACKTRACE: 1
|
||||
SQLNESS_OPTS: "--preserve-state"
|
||||
|
||||
|
||||
1
.github/workflows/nightly-ci.yml
vendored
1
.github/workflows/nightly-ci.yml
vendored
@@ -107,7 +107,6 @@ jobs:
|
||||
CARGO_BUILD_RUSTFLAGS: "-C linker=lld-link"
|
||||
RUST_BACKTRACE: 1
|
||||
CARGO_INCREMENTAL: 0
|
||||
RUSTUP_WINDOWS_PATH_ADD_BIN: 1 # Workaround for https://github.com/nextest-rs/nextest/issues/1493
|
||||
GT_S3_BUCKET: ${{ vars.AWS_CI_TEST_BUCKET }}
|
||||
GT_S3_ACCESS_KEY_ID: ${{ secrets.AWS_CI_TEST_ACCESS_KEY_ID }}
|
||||
GT_S3_ACCESS_KEY: ${{ secrets.AWS_CI_TEST_SECRET_ACCESS_KEY }}
|
||||
|
||||
3
Cargo.lock
generated
3
Cargo.lock
generated
@@ -2267,6 +2267,7 @@ version = "0.13.0"
|
||||
dependencies = [
|
||||
"async-trait",
|
||||
"common-procedure",
|
||||
"snafu 0.8.5",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -4705,7 +4706,7 @@ dependencies = [
|
||||
[[package]]
|
||||
name = "greptime-proto"
|
||||
version = "0.1.0"
|
||||
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=c5419bbd20cb42e568ec325a4d71a3c94cc327e1#c5419bbd20cb42e568ec325a4d71a3c94cc327e1"
|
||||
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=2be0f36b3264e28ab0e1c22a980d0bb634eb3a77#2be0f36b3264e28ab0e1c22a980d0bb634eb3a77"
|
||||
dependencies = [
|
||||
"prost 0.13.3",
|
||||
"serde",
|
||||
|
||||
@@ -129,7 +129,7 @@ etcd-client = "0.14"
|
||||
fst = "0.4.7"
|
||||
futures = "0.3"
|
||||
futures-util = "0.3"
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "c5419bbd20cb42e568ec325a4d71a3c94cc327e1" }
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "2be0f36b3264e28ab0e1c22a980d0bb634eb3a77" }
|
||||
hex = "0.4"
|
||||
http = "1"
|
||||
humantime = "2.1"
|
||||
|
||||
@@ -22,30 +22,31 @@ use std::vec;
|
||||
use api::v1::alter_table_expr::Kind;
|
||||
use api::v1::RenameTable;
|
||||
use async_trait::async_trait;
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_error::status_code::StatusCode;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_procedure::error::{FromJsonSnafu, Result as ProcedureResult, ToJsonSnafu};
|
||||
use common_procedure::{
|
||||
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure, Status, StringKey,
|
||||
Context as ProcedureContext, ContextProvider, Error as ProcedureError, LockKey, PoisonKey,
|
||||
PoisonKeys, Procedure, ProcedureId, Status, StringKey,
|
||||
};
|
||||
use common_telemetry::{debug, error, info};
|
||||
use futures::future;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::ResultExt;
|
||||
use snafu::{ensure, ResultExt};
|
||||
use store_api::storage::RegionId;
|
||||
use strum::AsRefStr;
|
||||
use table::metadata::{RawTableInfo, TableId, TableInfo};
|
||||
use table::table_reference::TableReference;
|
||||
|
||||
use crate::cache_invalidator::Context;
|
||||
use crate::ddl::utils::add_peer_context_if_needed;
|
||||
use crate::ddl::utils::{add_peer_context_if_needed, handle_multiple_results, MultipleResults};
|
||||
use crate::ddl::DdlContext;
|
||||
use crate::error::{Error, Result};
|
||||
use crate::error::{AbortProcedureSnafu, Error, NoLeaderSnafu, PutPoisonSnafu, Result};
|
||||
use crate::instruction::CacheIdent;
|
||||
use crate::key::table_info::TableInfoValue;
|
||||
use crate::key::{DeserializedValueWithBytes, RegionDistribution};
|
||||
use crate::lock_key::{CatalogLock, SchemaLock, TableLock, TableNameLock};
|
||||
use crate::metrics;
|
||||
use crate::poison_key::table_poison_key;
|
||||
use crate::rpc::ddl::AlterTableTask;
|
||||
use crate::rpc::router::{find_leader_regions, find_leaders, region_distribution};
|
||||
|
||||
@@ -104,7 +105,27 @@ impl AlterTableProcedure {
|
||||
Ok(Status::executing(true))
|
||||
}
|
||||
|
||||
pub async fn submit_alter_region_requests(&mut self) -> Result<Status> {
|
||||
fn table_poison_key(&self) -> PoisonKey {
|
||||
table_poison_key(self.data.table_id())
|
||||
}
|
||||
|
||||
async fn put_poison(
|
||||
&self,
|
||||
ctx_provider: &dyn ContextProvider,
|
||||
procedure_id: ProcedureId,
|
||||
) -> Result<()> {
|
||||
let poison_key = self.table_poison_key();
|
||||
ctx_provider
|
||||
.try_put_poison(&poison_key, procedure_id)
|
||||
.await
|
||||
.context(PutPoisonSnafu)
|
||||
}
|
||||
|
||||
pub async fn submit_alter_region_requests(
|
||||
&mut self,
|
||||
procedure_id: ProcedureId,
|
||||
ctx_provider: &dyn ContextProvider,
|
||||
) -> Result<Status> {
|
||||
let table_id = self.data.table_id();
|
||||
let (_, physical_table_route) = self
|
||||
.context
|
||||
@@ -127,6 +148,9 @@ impl AlterTableProcedure {
|
||||
alter_kind,
|
||||
);
|
||||
|
||||
ensure!(!leaders.is_empty(), NoLeaderSnafu { table_id });
|
||||
// Puts the poison before submitting alter region requests to datanodes.
|
||||
self.put_poison(ctx_provider, procedure_id).await?;
|
||||
for datanode in leaders {
|
||||
let requester = self.context.node_manager.datanode(&datanode).await;
|
||||
let regions = find_leader_regions(&physical_table_route.region_routes, &datanode);
|
||||
@@ -140,28 +164,51 @@ impl AlterTableProcedure {
|
||||
let requester = requester.clone();
|
||||
|
||||
alter_region_tasks.push(async move {
|
||||
if let Err(err) = requester.handle(request).await {
|
||||
if err.status_code() != StatusCode::RequestOutdated {
|
||||
// Treat request outdated as success.
|
||||
// The engine will throw this code when the schema version not match.
|
||||
// As this procedure has locked the table, the only reason for this error
|
||||
// is procedure is succeeded before and is retrying.
|
||||
return Err(add_peer_context_if_needed(datanode)(err));
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
requester
|
||||
.handle(request)
|
||||
.await
|
||||
.map_err(add_peer_context_if_needed(datanode))
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
future::join_all(alter_region_tasks)
|
||||
let results = future::join_all(alter_region_tasks)
|
||||
.await
|
||||
.into_iter()
|
||||
.collect::<Result<Vec<_>>>()?;
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
self.data.state = AlterTableState::UpdateMetadata;
|
||||
match handle_multiple_results(results) {
|
||||
MultipleResults::PartialRetryable(error) => {
|
||||
// Just returns the error, and wait for the next try.
|
||||
Err(error)
|
||||
}
|
||||
MultipleResults::PartialNonRetryable(error) => {
|
||||
error!(error; "Partial non-retryable errors occurred during alter table, table {}, table_id: {}", self.data.table_ref(), self.data.table_id());
|
||||
// No retry will be done.
|
||||
Ok(Status::poisoned(
|
||||
Some(self.table_poison_key()),
|
||||
ProcedureError::external(error),
|
||||
))
|
||||
}
|
||||
MultipleResults::AllRetryable(error) => {
|
||||
// Just returns the error, and wait for the next try.
|
||||
Err(error)
|
||||
}
|
||||
MultipleResults::Ok => {
|
||||
self.data.state = AlterTableState::UpdateMetadata;
|
||||
Ok(Status::executing_with_clean_poisons(true))
|
||||
}
|
||||
MultipleResults::AllNonRetryable(error) => {
|
||||
error!(error; "All alter requests returned non-retryable errors for table {}, table_id: {}", self.data.table_ref(), self.data.table_id());
|
||||
// It assumes the metadata on datanode is not changed.
|
||||
// Case: The alter region request is sent but not applied. (e.g., InvalidArgument)
|
||||
|
||||
Ok(Status::executing(true))
|
||||
let err = BoxedError::new(error);
|
||||
Err(err).context(AbortProcedureSnafu {
|
||||
clean_poisons: true,
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Update table metadata.
|
||||
@@ -250,10 +297,12 @@ impl Procedure for AlterTableProcedure {
|
||||
Self::TYPE_NAME
|
||||
}
|
||||
|
||||
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
|
||||
async fn execute(&mut self, ctx: &ProcedureContext) -> ProcedureResult<Status> {
|
||||
let error_handler = |e: Error| {
|
||||
if e.is_retry_later() {
|
||||
ProcedureError::retry_later(e)
|
||||
} else if e.need_clean_poisons() {
|
||||
ProcedureError::external_and_clean_poisons(e)
|
||||
} else {
|
||||
ProcedureError::external(e)
|
||||
}
|
||||
@@ -269,7 +318,10 @@ impl Procedure for AlterTableProcedure {
|
||||
|
||||
match state {
|
||||
AlterTableState::Prepare => self.on_prepare().await,
|
||||
AlterTableState::SubmitAlterRegionRequests => self.submit_alter_region_requests().await,
|
||||
AlterTableState::SubmitAlterRegionRequests => {
|
||||
self.submit_alter_region_requests(ctx.procedure_id, ctx.provider.as_ref())
|
||||
.await
|
||||
}
|
||||
AlterTableState::UpdateMetadata => self.on_update_metadata().await,
|
||||
AlterTableState::InvalidateTableCache => self.on_broadcast().await,
|
||||
}
|
||||
@@ -285,6 +337,10 @@ impl Procedure for AlterTableProcedure {
|
||||
|
||||
LockKey::new(key)
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
PoisonKeys::new(vec![self.table_poison_key()])
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize, AsRefStr)]
|
||||
|
||||
@@ -299,7 +299,9 @@ impl Procedure for CreateTableProcedure {
|
||||
.creator
|
||||
.register_opening_regions(&self.context, &x.region_routes)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
.context(ExternalSnafu {
|
||||
clean_poisons: false,
|
||||
})?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
|
||||
@@ -130,7 +130,9 @@ impl Procedure for DropDatabaseProcedure {
|
||||
self.state
|
||||
.recover(&self.runtime_context)
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)
|
||||
.context(ExternalSnafu {
|
||||
clean_poisons: false,
|
||||
})
|
||||
}
|
||||
|
||||
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
|
||||
|
||||
@@ -200,7 +200,9 @@ impl Procedure for DropTableProcedure {
|
||||
if register_operating_regions {
|
||||
self.register_dropping_regions()
|
||||
.map_err(BoxedError::new)
|
||||
.context(ExternalSnafu)?;
|
||||
.context(ExternalSnafu {
|
||||
clean_poisons: false,
|
||||
})?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
|
||||
@@ -80,7 +80,13 @@ pub async fn create_logical_table(
|
||||
let tasks = vec![test_create_logical_table_task(table_name)];
|
||||
let mut procedure = CreateLogicalTablesProcedure::new(tasks, physical_table_id, ddl_context);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
let status = procedure.on_create_metadata().await.unwrap();
|
||||
assert_matches!(status, Status::Done { .. });
|
||||
|
||||
|
||||
@@ -171,3 +171,74 @@ impl MockDatanodeHandler for NaiveDatanodeHandler {
|
||||
unreachable!()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct PartialSuccessDatanodeHandler {
|
||||
pub retryable: bool,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl MockDatanodeHandler for PartialSuccessDatanodeHandler {
|
||||
async fn handle(&self, peer: &Peer, _request: RegionRequest) -> Result<RegionResponse> {
|
||||
let success = peer.id % 2 == 0;
|
||||
if success {
|
||||
Ok(RegionResponse::new(0))
|
||||
} else if self.retryable {
|
||||
Err(Error::RetryLater {
|
||||
source: BoxedError::new(
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: "retry later",
|
||||
}
|
||||
.build(),
|
||||
),
|
||||
})
|
||||
} else {
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: "mock error",
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_query(
|
||||
&self,
|
||||
_peer: &Peer,
|
||||
_request: QueryRequest,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
unreachable!()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct AllFailureDatanodeHandler {
|
||||
pub retryable: bool,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl MockDatanodeHandler for AllFailureDatanodeHandler {
|
||||
async fn handle(&self, _peer: &Peer, _request: RegionRequest) -> Result<RegionResponse> {
|
||||
if self.retryable {
|
||||
Err(Error::RetryLater {
|
||||
source: BoxedError::new(
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: "retry later",
|
||||
}
|
||||
.build(),
|
||||
),
|
||||
})
|
||||
} else {
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: "mock error",
|
||||
}
|
||||
.fail()
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_query(
|
||||
&self,
|
||||
_peer: &Peer,
|
||||
_request: QueryRequest,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
unreachable!()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -180,7 +180,13 @@ async fn test_on_prepare() {
|
||||
|
||||
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context);
|
||||
let result = procedure.on_prepare().await;
|
||||
assert_matches!(result, Ok(Status::Executing { persist: true }));
|
||||
assert_matches!(
|
||||
result,
|
||||
Ok(Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
})
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -205,7 +211,13 @@ async fn test_on_update_metadata() {
|
||||
|
||||
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context);
|
||||
let mut status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
|
||||
let ctx = common_procedure::Context {
|
||||
procedure_id: ProcedureId::random(),
|
||||
@@ -213,10 +225,22 @@ async fn test_on_update_metadata() {
|
||||
};
|
||||
// on_submit_alter_region_requests
|
||||
status = procedure.execute(&ctx).await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
// on_update_metadata
|
||||
status = procedure.execute(&ctx).await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -237,7 +261,13 @@ async fn test_on_part_duplicate_alter_request() {
|
||||
|
||||
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context.clone());
|
||||
let mut status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
|
||||
let ctx = common_procedure::Context {
|
||||
procedure_id: ProcedureId::random(),
|
||||
@@ -245,10 +275,22 @@ async fn test_on_part_duplicate_alter_request() {
|
||||
};
|
||||
// on_submit_alter_region_requests
|
||||
status = procedure.execute(&ctx).await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
// on_update_metadata
|
||||
status = procedure.execute(&ctx).await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
|
||||
// re-alter
|
||||
let tasks = vec![
|
||||
@@ -270,7 +312,13 @@ async fn test_on_part_duplicate_alter_request() {
|
||||
|
||||
let mut procedure = AlterLogicalTablesProcedure::new(tasks, phy_id, ddl_context.clone());
|
||||
let mut status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
|
||||
let ctx = common_procedure::Context {
|
||||
procedure_id: ProcedureId::random(),
|
||||
@@ -278,10 +326,22 @@ async fn test_on_part_duplicate_alter_request() {
|
||||
};
|
||||
// on_submit_alter_region_requests
|
||||
status = procedure.execute(&ctx).await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
// on_update_metadata
|
||||
status = procedure.execute(&ctx).await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
|
||||
let table_name_keys = vec![
|
||||
TableNameKey::new(DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME, "table1"),
|
||||
|
||||
@@ -25,6 +25,9 @@ use api::v1::{
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_error::status_code::StatusCode;
|
||||
use common_procedure::store::poison_store::PoisonStore;
|
||||
use common_procedure::{ProcedureId, Status};
|
||||
use common_procedure_test::MockContextProvider;
|
||||
use store_api::storage::RegionId;
|
||||
use table::requests::TTL_KEY;
|
||||
use tokio::sync::mpsc::{self};
|
||||
@@ -33,16 +36,46 @@ use crate::ddl::alter_table::AlterTableProcedure;
|
||||
use crate::ddl::test_util::alter_table::TestAlterTableExprBuilder;
|
||||
use crate::ddl::test_util::create_table::test_create_table_task;
|
||||
use crate::ddl::test_util::datanode_handler::{
|
||||
DatanodeWatcher, RequestOutdatedErrorDatanodeHandler,
|
||||
AllFailureDatanodeHandler, DatanodeWatcher, PartialSuccessDatanodeHandler,
|
||||
RequestOutdatedErrorDatanodeHandler,
|
||||
};
|
||||
use crate::error::Error;
|
||||
use crate::key::datanode_table::DatanodeTableKey;
|
||||
use crate::key::table_name::TableNameKey;
|
||||
use crate::key::table_route::TableRouteValue;
|
||||
use crate::node_manager::NodeManagerRef;
|
||||
use crate::peer::Peer;
|
||||
use crate::poison_key::table_poison_key;
|
||||
use crate::rpc::ddl::AlterTableTask;
|
||||
use crate::rpc::router::{Region, RegionRoute};
|
||||
use crate::test_util::{new_ddl_context, MockDatanodeManager};
|
||||
|
||||
fn prepare_table_route(table_id: u32) -> TableRouteValue {
|
||||
TableRouteValue::physical(vec![
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 1)),
|
||||
leader_peer: Some(Peer::empty(1)),
|
||||
follower_peers: vec![Peer::empty(5)],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 2)),
|
||||
leader_peer: Some(Peer::empty(2)),
|
||||
follower_peers: vec![Peer::empty(4)],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 3)),
|
||||
leader_peer: Some(Peer::empty(3)),
|
||||
follower_peers: vec![],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
])
|
||||
}
|
||||
|
||||
fn test_rename_alter_table_task(table_name: &str, new_table_name: &str) -> AlterTableTask {
|
||||
let builder = TestAlterTableExprBuilder::default()
|
||||
.table_name(table_name)
|
||||
@@ -101,29 +134,7 @@ async fn test_on_submit_alter_request() {
|
||||
.table_metadata_manager
|
||||
.create_table_metadata(
|
||||
task.table_info.clone(),
|
||||
TableRouteValue::physical(vec![
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 1)),
|
||||
leader_peer: Some(Peer::empty(1)),
|
||||
follower_peers: vec![Peer::empty(5)],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 2)),
|
||||
leader_peer: Some(Peer::empty(2)),
|
||||
follower_peers: vec![Peer::empty(4)],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 3)),
|
||||
leader_peer: Some(Peer::empty(3)),
|
||||
follower_peers: vec![],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
]),
|
||||
prepare_table_route(table_id),
|
||||
HashMap::new(),
|
||||
)
|
||||
.await
|
||||
@@ -141,9 +152,15 @@ async fn test_on_submit_alter_request() {
|
||||
})),
|
||||
},
|
||||
};
|
||||
let mut procedure = AlterTableProcedure::new(table_id, alter_table_task, ddl_context).unwrap();
|
||||
let procedure_id = ProcedureId::random();
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let mut procedure =
|
||||
AlterTableProcedure::new(table_id, alter_table_task, ddl_context.clone()).unwrap();
|
||||
procedure.on_prepare().await.unwrap();
|
||||
procedure.submit_alter_region_requests().await.unwrap();
|
||||
procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let check = |peer: Peer,
|
||||
request: RegionRequest,
|
||||
@@ -185,29 +202,7 @@ async fn test_on_submit_alter_request_with_outdated_request() {
|
||||
.table_metadata_manager
|
||||
.create_table_metadata(
|
||||
task.table_info.clone(),
|
||||
TableRouteValue::physical(vec![
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 1)),
|
||||
leader_peer: Some(Peer::empty(1)),
|
||||
follower_peers: vec![Peer::empty(5)],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 2)),
|
||||
leader_peer: Some(Peer::empty(2)),
|
||||
follower_peers: vec![Peer::empty(4)],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
RegionRoute {
|
||||
region: Region::new_test(RegionId::new(table_id, 3)),
|
||||
leader_peer: Some(Peer::empty(3)),
|
||||
follower_peers: vec![],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
},
|
||||
]),
|
||||
prepare_table_route(table_id),
|
||||
HashMap::new(),
|
||||
)
|
||||
.await
|
||||
@@ -225,9 +220,15 @@ async fn test_on_submit_alter_request_with_outdated_request() {
|
||||
})),
|
||||
},
|
||||
};
|
||||
let procedure_id = ProcedureId::random();
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let mut procedure = AlterTableProcedure::new(table_id, alter_table_task, ddl_context).unwrap();
|
||||
procedure.on_prepare().await.unwrap();
|
||||
procedure.submit_alter_region_requests().await.unwrap();
|
||||
let err = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(!err.is_retry_later());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -326,9 +327,14 @@ async fn test_on_update_metadata_add_columns() {
|
||||
})),
|
||||
},
|
||||
};
|
||||
let procedure_id = ProcedureId::random();
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let mut procedure = AlterTableProcedure::new(table_id, task, ddl_context.clone()).unwrap();
|
||||
procedure.on_prepare().await.unwrap();
|
||||
procedure.submit_alter_region_requests().await.unwrap();
|
||||
procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap();
|
||||
procedure.on_update_metadata().await.unwrap();
|
||||
|
||||
let table_info = ddl_context
|
||||
@@ -387,9 +393,14 @@ async fn test_on_update_table_options() {
|
||||
})),
|
||||
},
|
||||
};
|
||||
let procedure_id = ProcedureId::random();
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let mut procedure = AlterTableProcedure::new(table_id, task, ddl_context.clone()).unwrap();
|
||||
procedure.on_prepare().await.unwrap();
|
||||
procedure.submit_alter_region_requests().await.unwrap();
|
||||
procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap();
|
||||
procedure.on_update_metadata().await.unwrap();
|
||||
|
||||
let table_info = ddl_context
|
||||
@@ -417,3 +428,156 @@ async fn test_on_update_table_options() {
|
||||
HashMap::from(&table_info.meta.options)
|
||||
);
|
||||
}
|
||||
|
||||
async fn prepare_alter_table_procedure(
|
||||
node_manager: NodeManagerRef,
|
||||
) -> (AlterTableProcedure, ProcedureId) {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let ddl_context = new_ddl_context(node_manager);
|
||||
let table_id = 1024;
|
||||
let table_name = "foo";
|
||||
let task = test_create_table_task(table_name, table_id);
|
||||
// Puts a value to table name key.
|
||||
ddl_context
|
||||
.table_metadata_manager
|
||||
.create_table_metadata(
|
||||
task.table_info.clone(),
|
||||
prepare_table_route(table_id),
|
||||
HashMap::new(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let alter_table_task = AlterTableTask {
|
||||
alter_table: AlterTableExpr {
|
||||
catalog_name: DEFAULT_CATALOG_NAME.to_string(),
|
||||
schema_name: DEFAULT_SCHEMA_NAME.to_string(),
|
||||
table_name: table_name.to_string(),
|
||||
kind: Some(Kind::DropColumns(DropColumns {
|
||||
drop_columns: vec![DropColumn {
|
||||
name: "cpu".to_string(),
|
||||
}],
|
||||
})),
|
||||
},
|
||||
};
|
||||
let procedure_id = ProcedureId::random();
|
||||
let mut procedure =
|
||||
AlterTableProcedure::new(table_id, alter_table_task, ddl_context.clone()).unwrap();
|
||||
procedure.on_prepare().await.unwrap();
|
||||
(procedure, procedure_id)
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_on_submit_alter_request_with_partial_success_retryable() {
|
||||
let node_manager = Arc::new(MockDatanodeManager::new(PartialSuccessDatanodeHandler {
|
||||
retryable: true,
|
||||
}));
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
|
||||
let result = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(result.is_retry_later());
|
||||
|
||||
// Submits again
|
||||
let result = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(result.is_retry_later());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_on_submit_alter_request_with_partial_success_non_retryable() {
|
||||
let node_manager = Arc::new(MockDatanodeManager::new(PartialSuccessDatanodeHandler {
|
||||
retryable: false,
|
||||
}));
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
|
||||
let result = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_matches!(result, Status::Poisoned { .. });
|
||||
|
||||
// submits again
|
||||
let result = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_matches!(result, Status::Poisoned { .. });
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_on_submit_alter_request_with_all_failure_retrybale() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let node_manager = Arc::new(MockDatanodeManager::new(AllFailureDatanodeHandler {
|
||||
retryable: true,
|
||||
}));
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
|
||||
let err = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(err.is_retry_later());
|
||||
// submits again
|
||||
let err = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert!(err.is_retry_later());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_on_submit_alter_request_with_all_failure_non_retrybale() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let node_manager = Arc::new(MockDatanodeManager::new(AllFailureDatanodeHandler {
|
||||
retryable: false,
|
||||
}));
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
|
||||
let err = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, Error::AbortProcedure { .. });
|
||||
assert!(!err.is_retry_later());
|
||||
assert!(err.need_clean_poisons());
|
||||
|
||||
// submits again
|
||||
let err = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, Error::AbortProcedure { .. });
|
||||
assert!(!err.is_retry_later());
|
||||
assert!(err.need_clean_poisons());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_on_submit_alter_request_with_exist_poison() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let node_manager = Arc::new(MockDatanodeManager::new(AllFailureDatanodeHandler {
|
||||
retryable: false,
|
||||
}));
|
||||
let provider = Arc::new(MockContextProvider::default());
|
||||
let (mut procedure, procedure_id) = prepare_alter_table_procedure(node_manager).await;
|
||||
|
||||
let table_id = 1024;
|
||||
let key = table_poison_key(table_id).to_string();
|
||||
let another_procedure_id = ProcedureId::random();
|
||||
provider
|
||||
.poison_manager()
|
||||
.try_put_poison(key, another_procedure_id.to_string())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
procedure.on_prepare().await.unwrap();
|
||||
let err = procedure
|
||||
.submit_alter_region_requests(procedure_id, provider.as_ref())
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, Error::PutPoison { .. });
|
||||
}
|
||||
|
||||
@@ -69,7 +69,13 @@ async fn test_on_prepare() {
|
||||
let physical_table_id = table_id;
|
||||
let mut procedure = CreateLogicalTablesProcedure::new(tasks, physical_table_id, ddl_context);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -202,7 +208,13 @@ async fn test_on_prepare_part_logical_tables_exist() {
|
||||
ddl_context,
|
||||
);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -238,7 +250,13 @@ async fn test_on_create_metadata() {
|
||||
ddl_context,
|
||||
);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
let ctx = ProcedureContext {
|
||||
procedure_id: ProcedureId::random(),
|
||||
provider: Arc::new(MockContextProvider::default()),
|
||||
@@ -294,7 +312,13 @@ async fn test_on_create_metadata_part_logical_tables_exist() {
|
||||
ddl_context,
|
||||
);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
let ctx = ProcedureContext {
|
||||
procedure_id: ProcedureId::random(),
|
||||
provider: Arc::new(MockContextProvider::default()),
|
||||
@@ -339,7 +363,13 @@ async fn test_on_create_metadata_err() {
|
||||
ddl_context.clone(),
|
||||
);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
let ctx = ProcedureContext {
|
||||
procedure_id: ProcedureId::random(),
|
||||
provider: Arc::new(MockContextProvider::default()),
|
||||
|
||||
@@ -137,7 +137,13 @@ async fn test_on_prepare_without_create_if_table_exists() {
|
||||
task.create_table.create_if_not_exists = true;
|
||||
let mut procedure = CreateTableProcedure::new(task, ddl_context);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
assert_eq!(procedure.table_id(), 1024);
|
||||
}
|
||||
|
||||
|
||||
@@ -153,7 +153,13 @@ async fn test_on_prepare_without_create_if_table_exists() {
|
||||
task.create_view.create_if_not_exists = true;
|
||||
let mut procedure = CreateViewProcedure::new(task, ddl_context);
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Executing { persist: true });
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
assert_eq!(procedure.view_id(), 1024);
|
||||
}
|
||||
|
||||
|
||||
@@ -13,10 +13,12 @@
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::fmt::Debug;
|
||||
|
||||
use common_catalog::consts::METRIC_ENGINE;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_procedure::error::Error as ProcedureError;
|
||||
use common_telemetry::{error, warn};
|
||||
use common_wal::options::WalOptions;
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use store_api::metric_engine_consts::LOGICAL_TABLE_METADATA_KEY;
|
||||
@@ -38,6 +40,7 @@ use crate::rpc::router::RegionRoute;
|
||||
/// Adds [Peer] context if the error is unretryable.
|
||||
pub fn add_peer_context_if_needed(datanode: Peer) -> impl FnOnce(Error) -> Error {
|
||||
move |err| {
|
||||
error!(err; "Failed to operate datanode, peer: {}", datanode);
|
||||
if !err.is_retry_later() {
|
||||
return Err::<(), BoxedError>(BoxedError::new(err))
|
||||
.context(OperateDatanodeSnafu { peer: datanode })
|
||||
@@ -182,6 +185,85 @@ pub fn extract_region_wal_options(
|
||||
Ok(region_wal_options)
|
||||
}
|
||||
|
||||
/// The result of multiple operations.
|
||||
///
|
||||
/// - Ok: all operations are successful.
|
||||
/// - PartialRetryable: if any operation is retryable and without non retryable error, the result is retryable.
|
||||
/// - PartialNonRetryable: if any operation is non retryable, the result is non retryable.
|
||||
/// - AllRetryable: all operations are retryable.
|
||||
/// - AllNonRetryable: all operations are not retryable.
|
||||
pub enum MultipleResults {
|
||||
Ok,
|
||||
PartialRetryable(Error),
|
||||
PartialNonRetryable(Error),
|
||||
AllRetryable(Error),
|
||||
AllNonRetryable(Error),
|
||||
}
|
||||
|
||||
/// Handles the results of alter region requests.
|
||||
///
|
||||
/// For partial success, we need to check if the errors are retryable.
|
||||
/// If all the errors are retryable, we return a retryable error.
|
||||
/// Otherwise, we return the first error.
|
||||
pub fn handle_multiple_results<T: Debug>(results: Vec<Result<T>>) -> MultipleResults {
|
||||
if results.is_empty() {
|
||||
return MultipleResults::Ok;
|
||||
}
|
||||
let num_results = results.len();
|
||||
let mut retryable_results = Vec::new();
|
||||
let mut non_retryable_results = Vec::new();
|
||||
let mut ok_results = Vec::new();
|
||||
|
||||
for result in results {
|
||||
match result {
|
||||
Ok(_) => ok_results.push(result),
|
||||
Err(err) => {
|
||||
if err.is_retry_later() {
|
||||
retryable_results.push(err);
|
||||
} else {
|
||||
non_retryable_results.push(err);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
common_telemetry::debug!(
|
||||
"retryable_results: {}, non_retryable_results: {}, ok_results: {}",
|
||||
retryable_results.len(),
|
||||
non_retryable_results.len(),
|
||||
ok_results.len()
|
||||
);
|
||||
|
||||
if retryable_results.len() == num_results {
|
||||
return MultipleResults::AllRetryable(retryable_results.into_iter().next().unwrap());
|
||||
} else if non_retryable_results.len() == num_results {
|
||||
warn!("all non retryable results: {}", non_retryable_results.len());
|
||||
for err in &non_retryable_results {
|
||||
error!(err; "non retryable error");
|
||||
}
|
||||
return MultipleResults::AllNonRetryable(non_retryable_results.into_iter().next().unwrap());
|
||||
} else if ok_results.len() == num_results {
|
||||
return MultipleResults::Ok;
|
||||
} else if !retryable_results.is_empty()
|
||||
&& !ok_results.is_empty()
|
||||
&& non_retryable_results.is_empty()
|
||||
{
|
||||
return MultipleResults::PartialRetryable(retryable_results.into_iter().next().unwrap());
|
||||
}
|
||||
|
||||
warn!(
|
||||
"partial non retryable results: {}, retryable results: {}, ok results: {}",
|
||||
non_retryable_results.len(),
|
||||
retryable_results.len(),
|
||||
ok_results.len()
|
||||
);
|
||||
for err in &non_retryable_results {
|
||||
error!(err; "non retryable error");
|
||||
}
|
||||
// non_retryable_results.len() > 0
|
||||
MultipleResults::PartialNonRetryable(non_retryable_results.into_iter().next().unwrap())
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
@@ -834,6 +834,7 @@ mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_procedure::local::LocalManager;
|
||||
use common_procedure::test_util::InMemoryPoisonStore;
|
||||
|
||||
use super::DdlManager;
|
||||
use crate::cache_invalidator::DummyCacheInvalidator;
|
||||
@@ -882,7 +883,12 @@ mod tests {
|
||||
));
|
||||
|
||||
let state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
|
||||
let procedure_manager = Arc::new(LocalManager::new(Default::default(), state_store));
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::default());
|
||||
let procedure_manager = Arc::new(LocalManager::new(
|
||||
Default::default(),
|
||||
state_store,
|
||||
poison_manager,
|
||||
));
|
||||
|
||||
let _ = DdlManager::try_new(
|
||||
DdlContext {
|
||||
|
||||
@@ -449,6 +449,14 @@ pub enum Error {
|
||||
#[snafu(display("Retry later"))]
|
||||
RetryLater { source: BoxedError },
|
||||
|
||||
#[snafu(display("Abort procedure"))]
|
||||
AbortProcedure {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
clean_poisons: bool,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Failed to encode a wal options to json string, wal_options: {:?}",
|
||||
wal_options
|
||||
@@ -748,6 +756,33 @@ pub enum Error {
|
||||
#[snafu(source)]
|
||||
error: serde_json::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("No leader found for table_id: {}", table_id))]
|
||||
NoLeader {
|
||||
table_id: TableId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Procedure poison key already exists with a different value, key: {}, value: {}",
|
||||
key,
|
||||
value
|
||||
))]
|
||||
ProcedurePoisonConflict {
|
||||
key: String,
|
||||
value: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to put poison, table metadata may be corrupted"))]
|
||||
PutPoison {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
#[snafu(source)]
|
||||
source: common_procedure::error::Error,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -766,7 +801,8 @@ impl ErrorExt for Error {
|
||||
| SerializeToJson { .. }
|
||||
| DeserializeFromJson { .. } => StatusCode::Internal,
|
||||
|
||||
ValueNotExist { .. } => StatusCode::Unexpected,
|
||||
NoLeader { .. } => StatusCode::TableUnavailable,
|
||||
ValueNotExist { .. } | ProcedurePoisonConflict { .. } => StatusCode::Unexpected,
|
||||
|
||||
Unsupported { .. } => StatusCode::Unsupported,
|
||||
|
||||
@@ -837,7 +873,9 @@ impl ErrorExt for Error {
|
||||
OperateDatanode { source, .. } => source.status_code(),
|
||||
Table { source, .. } => source.status_code(),
|
||||
RetryLater { source, .. } => source.status_code(),
|
||||
AbortProcedure { source, .. } => source.status_code(),
|
||||
ConvertAlterTableRequest { source, .. } => source.status_code(),
|
||||
PutPoison { source, .. } => source.status_code(),
|
||||
|
||||
ParseProcedureId { .. }
|
||||
| InvalidNumTopics { .. }
|
||||
@@ -908,6 +946,11 @@ impl Error {
|
||||
matches!(self, Error::RetryLater { .. })
|
||||
}
|
||||
|
||||
/// Determine whether it needs to clean poisons.
|
||||
pub fn need_clean_poisons(&self) -> bool {
|
||||
matches!(self, Error::AbortProcedure { clean_poisons, .. } if *clean_poisons)
|
||||
}
|
||||
|
||||
/// Returns true if the response exceeds the size limit.
|
||||
pub fn is_exceeded_size_limit(&self) -> bool {
|
||||
match self {
|
||||
|
||||
@@ -156,6 +156,7 @@ use crate::kv_backend::txn::{Txn, TxnOp};
|
||||
use crate::kv_backend::KvBackendRef;
|
||||
use crate::rpc::router::{region_distribution, LeaderState, RegionRoute};
|
||||
use crate::rpc::store::BatchDeleteRequest;
|
||||
use crate::state_store::PoisonValue;
|
||||
use crate::DatanodeId;
|
||||
|
||||
pub const NAME_PATTERN: &str = r"[a-zA-Z_:-][a-zA-Z0-9_:\-\.@#]*";
|
||||
@@ -1320,7 +1321,8 @@ impl_metadata_value! {
|
||||
TableFlowValue,
|
||||
NodeAddressValue,
|
||||
SchemaNameValue,
|
||||
FlowStateValue
|
||||
FlowStateValue,
|
||||
PoisonValue
|
||||
}
|
||||
|
||||
impl_optional_metadata_value! {
|
||||
|
||||
@@ -158,18 +158,18 @@ impl<'a> MySqlTemplateFactory<'a> {
|
||||
"CREATE TABLE IF NOT EXISTS {table_name}(k VARBINARY(3072) PRIMARY KEY, v BLOB);",
|
||||
),
|
||||
range_template: RangeTemplate {
|
||||
point: format!("SELECT k, v FROM {table_name} WHERE k = ?"),
|
||||
range: format!("SELECT k, v FROM {table_name} WHERE k >= ? AND k < ? ORDER BY k"),
|
||||
full: format!("SELECT k, v FROM {table_name} ? ORDER BY k"),
|
||||
left_bounded: format!("SELECT k, v FROM {table_name} WHERE k >= ? ORDER BY k"),
|
||||
prefix: format!("SELECT k, v FROM {table_name} WHERE k LIKE ? ORDER BY k"),
|
||||
point: format!("SELECT k, v FROM `{table_name}` WHERE k = ?"),
|
||||
range: format!("SELECT k, v FROM `{table_name}` WHERE k >= ? AND k < ? ORDER BY k"),
|
||||
full: format!("SELECT k, v FROM `{table_name}` ? ORDER BY k"),
|
||||
left_bounded: format!("SELECT k, v FROM `{table_name}` WHERE k >= ? ORDER BY k"),
|
||||
prefix: format!("SELECT k, v FROM `{table_name}` WHERE k LIKE ? ORDER BY k"),
|
||||
},
|
||||
delete_template: RangeTemplate {
|
||||
point: format!("DELETE FROM {table_name} WHERE k = ?;"),
|
||||
range: format!("DELETE FROM {table_name} WHERE k >= ? AND k < ?;"),
|
||||
full: format!("DELETE FROM {table_name}"),
|
||||
left_bounded: format!("DELETE FROM {table_name} WHERE k >= ?;"),
|
||||
prefix: format!("DELETE FROM {table_name} WHERE k LIKE ?;"),
|
||||
point: format!("DELETE FROM `{table_name}` WHERE k = ?;"),
|
||||
range: format!("DELETE FROM `{table_name}` WHERE k >= ? AND k < ?;"),
|
||||
full: format!("DELETE FROM `{table_name}`"),
|
||||
left_bounded: format!("DELETE FROM `{table_name}` WHERE k >= ?;"),
|
||||
prefix: format!("DELETE FROM `{table_name}` WHERE k LIKE ?;"),
|
||||
},
|
||||
}
|
||||
}
|
||||
@@ -189,14 +189,17 @@ impl MySqlTemplateSet {
|
||||
fn generate_batch_get_query(&self, key_len: usize) -> String {
|
||||
let table_name = &self.table_name;
|
||||
let in_clause = mysql_generate_in_placeholders(1, key_len).join(", ");
|
||||
format!("SELECT k, v FROM {table_name} WHERE k in ({});", in_clause)
|
||||
format!(
|
||||
"SELECT k, v FROM `{table_name}` WHERE k in ({});",
|
||||
in_clause
|
||||
)
|
||||
}
|
||||
|
||||
/// Generates the sql for batch delete.
|
||||
fn generate_batch_delete_query(&self, key_len: usize) -> String {
|
||||
let table_name = &self.table_name;
|
||||
let in_clause = mysql_generate_in_placeholders(1, key_len).join(", ");
|
||||
format!("DELETE FROM {table_name} WHERE k in ({});", in_clause)
|
||||
format!("DELETE FROM `{table_name}` WHERE k in ({});", in_clause)
|
||||
}
|
||||
|
||||
/// Generates the sql for batch upsert.
|
||||
@@ -212,9 +215,9 @@ impl MySqlTemplateSet {
|
||||
let values_clause = values_placeholders.join(", ");
|
||||
|
||||
(
|
||||
format!(r#"SELECT k, v FROM {table_name} WHERE k IN ({in_clause})"#,),
|
||||
format!(r#"SELECT k, v FROM `{table_name}` WHERE k IN ({in_clause})"#,),
|
||||
format!(
|
||||
r#"INSERT INTO {table_name} (k, v) VALUES {values_clause} ON DUPLICATE KEY UPDATE v = VALUES(v);"#,
|
||||
r#"INSERT INTO `{table_name}` (k, v) VALUES {values_clause} ON DUPLICATE KEY UPDATE v = VALUES(v);"#,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
@@ -157,21 +157,25 @@ impl<'a> PgSqlTemplateFactory<'a> {
|
||||
PgSqlTemplateSet {
|
||||
table_name: table_name.to_string(),
|
||||
create_table_statement: format!(
|
||||
"CREATE TABLE IF NOT EXISTS {table_name}(k bytea PRIMARY KEY, v bytea)",
|
||||
"CREATE TABLE IF NOT EXISTS \"{table_name}\"(k bytea PRIMARY KEY, v bytea)",
|
||||
),
|
||||
range_template: RangeTemplate {
|
||||
point: format!("SELECT k, v FROM {table_name} WHERE k = $1"),
|
||||
range: format!("SELECT k, v FROM {table_name} WHERE k >= $1 AND k < $2 ORDER BY k"),
|
||||
full: format!("SELECT k, v FROM {table_name} $1 ORDER BY k"),
|
||||
left_bounded: format!("SELECT k, v FROM {table_name} WHERE k >= $1 ORDER BY k"),
|
||||
prefix: format!("SELECT k, v FROM {table_name} WHERE k LIKE $1 ORDER BY k"),
|
||||
point: format!("SELECT k, v FROM \"{table_name}\" WHERE k = $1"),
|
||||
range: format!(
|
||||
"SELECT k, v FROM \"{table_name}\" WHERE k >= $1 AND k < $2 ORDER BY k"
|
||||
),
|
||||
full: format!("SELECT k, v FROM \"{table_name}\" $1 ORDER BY k"),
|
||||
left_bounded: format!("SELECT k, v FROM \"{table_name}\" WHERE k >= $1 ORDER BY k"),
|
||||
prefix: format!("SELECT k, v FROM \"{table_name}\" WHERE k LIKE $1 ORDER BY k"),
|
||||
},
|
||||
delete_template: RangeTemplate {
|
||||
point: format!("DELETE FROM {table_name} WHERE k = $1 RETURNING k,v;"),
|
||||
range: format!("DELETE FROM {table_name} WHERE k >= $1 AND k < $2 RETURNING k,v;"),
|
||||
full: format!("DELETE FROM {table_name} RETURNING k,v"),
|
||||
left_bounded: format!("DELETE FROM {table_name} WHERE k >= $1 RETURNING k,v;"),
|
||||
prefix: format!("DELETE FROM {table_name} WHERE k LIKE $1 RETURNING k,v;"),
|
||||
point: format!("DELETE FROM \"{table_name}\" WHERE k = $1 RETURNING k,v;"),
|
||||
range: format!(
|
||||
"DELETE FROM \"{table_name}\" WHERE k >= $1 AND k < $2 RETURNING k,v;"
|
||||
),
|
||||
full: format!("DELETE FROM \"{table_name}\" RETURNING k,v"),
|
||||
left_bounded: format!("DELETE FROM \"{table_name}\" WHERE k >= $1 RETURNING k,v;"),
|
||||
prefix: format!("DELETE FROM \"{table_name}\" WHERE k LIKE $1 RETURNING k,v;"),
|
||||
},
|
||||
}
|
||||
}
|
||||
@@ -191,7 +195,10 @@ impl PgSqlTemplateSet {
|
||||
fn generate_batch_get_query(&self, key_len: usize) -> String {
|
||||
let table_name = &self.table_name;
|
||||
let in_clause = pg_generate_in_placeholders(1, key_len).join(", ");
|
||||
format!("SELECT k, v FROM {table_name} WHERE k in ({});", in_clause)
|
||||
format!(
|
||||
"SELECT k, v FROM \"{table_name}\" WHERE k in ({});",
|
||||
in_clause
|
||||
)
|
||||
}
|
||||
|
||||
/// Generates the sql for batch delete.
|
||||
@@ -199,7 +206,7 @@ impl PgSqlTemplateSet {
|
||||
let table_name = &self.table_name;
|
||||
let in_clause = pg_generate_in_placeholders(1, key_len).join(", ");
|
||||
format!(
|
||||
"DELETE FROM {table_name} WHERE k in ({}) RETURNING k,v;",
|
||||
"DELETE FROM \"{table_name}\" WHERE k in ({}) RETURNING k,v;",
|
||||
in_clause
|
||||
)
|
||||
}
|
||||
@@ -220,9 +227,9 @@ impl PgSqlTemplateSet {
|
||||
format!(
|
||||
r#"
|
||||
WITH prev AS (
|
||||
SELECT k,v FROM {table_name} WHERE k IN ({in_clause})
|
||||
SELECT k,v FROM "{table_name}" WHERE k IN ({in_clause})
|
||||
), update AS (
|
||||
INSERT INTO {table_name} (k, v) VALUES
|
||||
INSERT INTO "{table_name}" (k, v) VALUES
|
||||
{values_clause}
|
||||
ON CONFLICT (
|
||||
k
|
||||
|
||||
@@ -37,6 +37,7 @@ pub mod metrics;
|
||||
pub mod node_expiry_listener;
|
||||
pub mod node_manager;
|
||||
pub mod peer;
|
||||
pub mod poison_key;
|
||||
pub mod range_stream;
|
||||
pub mod region_keeper;
|
||||
pub mod rpc;
|
||||
|
||||
22
src/common/meta/src/poison_key.rs
Normal file
22
src/common/meta/src/poison_key.rs
Normal file
@@ -0,0 +1,22 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use common_procedure::PoisonKey;
|
||||
use store_api::storage::TableId;
|
||||
|
||||
/// Returns the poison key for the table.
|
||||
pub fn table_poison_key(table_id: TableId) -> PoisonKey {
|
||||
let key = format!("table/{}", table_id);
|
||||
PoisonKey::new(&key)
|
||||
}
|
||||
@@ -1240,6 +1240,7 @@ impl From<QueryContext> for PbQueryContext {
|
||||
extensions,
|
||||
channel: channel as u32,
|
||||
snapshot_seqs: None,
|
||||
explain: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -81,6 +81,7 @@ pub fn procedure_state_to_pb_state(state: &ProcedureState) -> (PbProcedureStatus
|
||||
ProcedureState::RollingBack { error } => {
|
||||
(PbProcedureStatus::RollingBack, error.to_string())
|
||||
}
|
||||
ProcedureState::Poisoned { error, .. } => (PbProcedureStatus::Poisoned, error.to_string()),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -14,16 +14,23 @@
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_procedure::error::{DeleteStatesSnafu, ListStateSnafu, PutStateSnafu};
|
||||
use common_procedure::error::{
|
||||
DeletePoisonSnafu, DeleteStatesSnafu, GetPoisonSnafu, ListStateSnafu, PutPoisonSnafu,
|
||||
PutStateSnafu, Result as ProcedureResult,
|
||||
};
|
||||
use common_procedure::store::poison_store::PoisonStore;
|
||||
use common_procedure::store::state_store::{KeySet, KeyValueStream, StateStore};
|
||||
use common_procedure::store::util::multiple_value_stream;
|
||||
use common_procedure::Result as ProcedureResult;
|
||||
use futures::future::try_join_all;
|
||||
use futures::StreamExt;
|
||||
use itertools::Itertools;
|
||||
use snafu::ResultExt;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::error::{ProcedurePoisonConflictSnafu, Result, UnexpectedSnafu};
|
||||
use crate::key::txn_helper::TxnOpGetResponseSet;
|
||||
use crate::key::{DeserializedValueWithBytes, MetadataValue};
|
||||
use crate::kv_backend::txn::{Compare, CompareOp, Txn, TxnOp};
|
||||
use crate::kv_backend::KvBackendRef;
|
||||
use crate::range_stream::PaginationStream;
|
||||
use crate::rpc::store::{BatchDeleteRequest, PutRequest, RangeRequest};
|
||||
@@ -32,11 +39,16 @@ use crate::rpc::KeyValue;
|
||||
const DELIMITER: &str = "/";
|
||||
|
||||
const PROCEDURE_PREFIX: &str = "/__procedure__/";
|
||||
const PROCEDURE_POISON_KEY_PREFIX: &str = "/__procedure_poison/";
|
||||
|
||||
fn with_prefix(key: &str) -> String {
|
||||
format!("{PROCEDURE_PREFIX}{key}")
|
||||
}
|
||||
|
||||
fn with_poison_prefix(key: &str) -> String {
|
||||
format!("{}{}", PROCEDURE_POISON_KEY_PREFIX, key)
|
||||
}
|
||||
|
||||
fn strip_prefix(key: &str) -> String {
|
||||
key.trim_start_matches(PROCEDURE_PREFIX).to_string()
|
||||
}
|
||||
@@ -207,8 +219,168 @@ impl StateStore for KvStateStore {
|
||||
}
|
||||
}
|
||||
|
||||
/// The value of the poison key.
|
||||
///
|
||||
/// Each poison value contains a unique token that identifies the procedure.
|
||||
/// While multiple procedures may use the same poison key (representing the same resource),
|
||||
/// each procedure will have a distinct token value to differentiate its ownership.
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
pub struct PoisonValue {
|
||||
token: String,
|
||||
}
|
||||
|
||||
type PoisonDecodeResult = Result<Option<DeserializedValueWithBytes<PoisonValue>>>;
|
||||
|
||||
impl KvStateStore {
|
||||
/// Builds a create poison transaction,
|
||||
/// it expected the `__procedure_poison/{key}` wasn't occupied.
|
||||
fn build_create_poison_txn(
|
||||
&self,
|
||||
key: &str,
|
||||
value: &PoisonValue,
|
||||
) -> Result<(
|
||||
Txn,
|
||||
impl FnOnce(&mut TxnOpGetResponseSet) -> PoisonDecodeResult,
|
||||
)> {
|
||||
let key = key.as_bytes().to_vec();
|
||||
let value = value.try_as_raw_value()?;
|
||||
let txn = Txn::put_if_not_exists(key.clone(), value);
|
||||
|
||||
Ok((
|
||||
txn,
|
||||
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
|
||||
))
|
||||
}
|
||||
|
||||
/// Builds a delete poison transaction,
|
||||
/// it expected the `__procedure_poison/{key}` was occupied.
|
||||
fn build_delete_poison_txn(
|
||||
&self,
|
||||
key: &str,
|
||||
value: PoisonValue,
|
||||
) -> Result<(
|
||||
Txn,
|
||||
impl FnOnce(&mut TxnOpGetResponseSet) -> PoisonDecodeResult,
|
||||
)> {
|
||||
let key = key.as_bytes().to_vec();
|
||||
let value = value.try_as_raw_value()?;
|
||||
|
||||
let txn = Txn::new()
|
||||
.when(vec![Compare::with_value(
|
||||
key.clone(),
|
||||
CompareOp::Equal,
|
||||
value,
|
||||
)])
|
||||
.and_then(vec![TxnOp::Delete(key.clone())])
|
||||
.or_else(vec![TxnOp::Get(key.clone())]);
|
||||
|
||||
Ok((
|
||||
txn,
|
||||
TxnOpGetResponseSet::decode_with(TxnOpGetResponseSet::filter(key)),
|
||||
))
|
||||
}
|
||||
|
||||
async fn get_poison_inner(&self, key: &str) -> Result<Option<PoisonValue>> {
|
||||
let key = with_poison_prefix(key);
|
||||
let value = self.kv_backend.get(key.as_bytes()).await?;
|
||||
value
|
||||
.map(|v| PoisonValue::try_from_raw_value(&v.value))
|
||||
.transpose()
|
||||
}
|
||||
|
||||
/// Put the poison.
|
||||
///
|
||||
/// If the poison is already put by other procedure, it will return an error.
|
||||
async fn set_poison_inner(&self, key: &str, token: &str) -> Result<()> {
|
||||
let key = with_poison_prefix(key);
|
||||
let (txn, on_failure) = self.build_create_poison_txn(
|
||||
&key,
|
||||
&PoisonValue {
|
||||
token: token.to_string(),
|
||||
},
|
||||
)?;
|
||||
|
||||
let mut resp = self.kv_backend.txn(txn).await?;
|
||||
|
||||
if !resp.succeeded {
|
||||
let mut set = TxnOpGetResponseSet::from(&mut resp.responses);
|
||||
let remote_value = on_failure(&mut set)?
|
||||
.context(UnexpectedSnafu {
|
||||
err_msg: "Reads the empty poison value in comparing operation of the put consistency poison",
|
||||
})?
|
||||
.into_inner();
|
||||
|
||||
ensure!(
|
||||
remote_value.token == token,
|
||||
ProcedurePoisonConflictSnafu {
|
||||
key: &key,
|
||||
value: &remote_value.token,
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Deletes the poison.
|
||||
///
|
||||
/// If the poison is not put by the procedure, it will return an error.
|
||||
async fn delete_poison_inner(&self, key: &str, token: &str) -> Result<()> {
|
||||
let key = with_poison_prefix(key);
|
||||
let (txn, on_failure) = self.build_delete_poison_txn(
|
||||
&key,
|
||||
PoisonValue {
|
||||
token: token.to_string(),
|
||||
},
|
||||
)?;
|
||||
|
||||
let mut resp = self.kv_backend.txn(txn).await?;
|
||||
|
||||
if !resp.succeeded {
|
||||
let mut set = TxnOpGetResponseSet::from(&mut resp.responses);
|
||||
let remote_value = on_failure(&mut set)?;
|
||||
|
||||
ensure!(
|
||||
remote_value.is_none(),
|
||||
ProcedurePoisonConflictSnafu {
|
||||
key: &key,
|
||||
value: &remote_value.unwrap().into_inner().token,
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
impl PoisonStore for KvStateStore {
|
||||
async fn try_put_poison(&self, key: String, token: String) -> ProcedureResult<()> {
|
||||
self.set_poison_inner(&key, &token)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(PutPoisonSnafu { key, token })
|
||||
}
|
||||
|
||||
async fn delete_poison(&self, key: String, token: String) -> ProcedureResult<()> {
|
||||
self.delete_poison_inner(&key, &token)
|
||||
.await
|
||||
.map_err(BoxedError::new)
|
||||
.context(DeletePoisonSnafu { key, token })
|
||||
}
|
||||
|
||||
async fn get_poison(&self, key: &str) -> ProcedureResult<Option<String>> {
|
||||
self.get_poison_inner(key)
|
||||
.await
|
||||
.map(|v| v.map(|v| v.token))
|
||||
.map_err(BoxedError::new)
|
||||
.context(GetPoisonSnafu { key })
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
use std::env;
|
||||
use std::sync::Arc;
|
||||
|
||||
@@ -219,6 +391,7 @@ mod tests {
|
||||
use uuid::Uuid;
|
||||
|
||||
use super::*;
|
||||
use crate::error::Error;
|
||||
use crate::kv_backend::chroot::ChrootKvBackend;
|
||||
use crate::kv_backend::etcd::EtcdStore;
|
||||
use crate::kv_backend::memory::MemoryKvBackend;
|
||||
@@ -397,4 +570,73 @@ mod tests {
|
||||
)
|
||||
.await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_poison() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
let poison_manager = KvStateStore::new(mem_kv.clone());
|
||||
|
||||
let key = "table/1";
|
||||
|
||||
let token = "expected_token";
|
||||
|
||||
poison_manager.set_poison_inner(key, token).await.unwrap();
|
||||
|
||||
// Put again, should be ok.
|
||||
poison_manager.set_poison_inner(key, token).await.unwrap();
|
||||
|
||||
// Delete, should be ok.
|
||||
poison_manager
|
||||
.delete_poison_inner(key, token)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
// Delete again, should be ok.
|
||||
poison_manager
|
||||
.delete_poison_inner(key, token)
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_consistency_poison_failed() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
let poison_manager = KvStateStore::new(mem_kv.clone());
|
||||
|
||||
let key = "table/1";
|
||||
|
||||
let token = "expected_token";
|
||||
let token2 = "expected_token2";
|
||||
|
||||
poison_manager.set_poison_inner(key, token).await.unwrap();
|
||||
|
||||
let err = poison_manager
|
||||
.set_poison_inner(key, token2)
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, Error::ProcedurePoisonConflict { .. });
|
||||
|
||||
let err = poison_manager
|
||||
.delete_poison_inner(key, token2)
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_matches!(err, Error::ProcedurePoisonConflict { .. });
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_serialize_deserialize() {
|
||||
let key = "table/1";
|
||||
let value = PoisonValue {
|
||||
token: "expected_token".to_string(),
|
||||
};
|
||||
|
||||
let serialized_key = with_poison_prefix(key).as_bytes().to_vec();
|
||||
let serialized_value = value.try_as_raw_value().unwrap();
|
||||
|
||||
let expected_key = "/__procedure_poison/table/1";
|
||||
let expected_value = r#"{"token":"expected_token"}"#;
|
||||
|
||||
assert_eq!(expected_key.as_bytes(), serialized_key);
|
||||
assert_eq!(expected_value.as_bytes(), serialized_value);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -9,4 +9,5 @@ workspace = true
|
||||
|
||||
[dependencies]
|
||||
async-trait.workspace = true
|
||||
common-procedure.workspace = true
|
||||
common-procedure = { workspace = true, features = ["testing"] }
|
||||
snafu.workspace = true
|
||||
|
||||
@@ -18,21 +18,32 @@ use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
use common_procedure::store::poison_store::PoisonStore;
|
||||
use common_procedure::test_util::InMemoryPoisonStore;
|
||||
use common_procedure::{
|
||||
Context, ContextProvider, Output, Procedure, ProcedureId, ProcedureState, ProcedureWithId,
|
||||
Result, Status,
|
||||
Context, ContextProvider, Output, PoisonKey, Procedure, ProcedureId, ProcedureState,
|
||||
ProcedureWithId, Result, Status,
|
||||
};
|
||||
|
||||
/// A Mock [ContextProvider].
|
||||
#[derive(Default)]
|
||||
pub struct MockContextProvider {
|
||||
states: HashMap<ProcedureId, ProcedureState>,
|
||||
poison_manager: InMemoryPoisonStore,
|
||||
}
|
||||
|
||||
impl MockContextProvider {
|
||||
/// Returns a new provider.
|
||||
pub fn new(states: HashMap<ProcedureId, ProcedureState>) -> MockContextProvider {
|
||||
MockContextProvider { states }
|
||||
MockContextProvider {
|
||||
states,
|
||||
poison_manager: InMemoryPoisonStore::default(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a reference to the poison manager.
|
||||
pub fn poison_manager(&self) -> &InMemoryPoisonStore {
|
||||
&self.poison_manager
|
||||
}
|
||||
}
|
||||
|
||||
@@ -41,6 +52,12 @@ impl ContextProvider for MockContextProvider {
|
||||
async fn procedure_state(&self, procedure_id: ProcedureId) -> Result<Option<ProcedureState>> {
|
||||
Ok(self.states.get(&procedure_id).cloned())
|
||||
}
|
||||
|
||||
async fn try_put_poison(&self, key: &PoisonKey, procedure_id: ProcedureId) -> Result<()> {
|
||||
self.poison_manager
|
||||
.try_put_poison(key.to_string(), procedure_id.to_string())
|
||||
.await
|
||||
}
|
||||
}
|
||||
|
||||
/// Executes a procedure until it returns [Status::Done].
|
||||
@@ -61,6 +78,7 @@ pub async fn execute_procedure_until_done(procedure: &mut dyn Procedure) -> Opti
|
||||
"Executing subprocedure is unsupported"
|
||||
),
|
||||
Status::Done { output } => return output,
|
||||
Status::Poisoned { .. } => return None,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -88,6 +106,7 @@ pub async fn execute_procedure_once(
|
||||
false
|
||||
}
|
||||
Status::Done { .. } => true,
|
||||
Status::Poisoned { .. } => false,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -109,6 +128,7 @@ pub async fn execute_until_suspended_or_done(
|
||||
Status::Executing { .. } => (),
|
||||
Status::Suspended { subprocedures, .. } => return Some(subprocedures),
|
||||
Status::Done { .. } => break,
|
||||
Status::Poisoned { .. } => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -21,14 +21,21 @@ use common_macro::stack_trace_debug;
|
||||
use snafu::{Location, Snafu};
|
||||
|
||||
use crate::procedure::ProcedureId;
|
||||
use crate::PoisonKey;
|
||||
|
||||
/// Procedure error.
|
||||
#[derive(Snafu)]
|
||||
#[snafu(visibility(pub))]
|
||||
#[stack_trace_debug]
|
||||
pub enum Error {
|
||||
#[snafu(display("Failed to execute procedure due to external error"))]
|
||||
External { source: BoxedError },
|
||||
#[snafu(display(
|
||||
"Failed to execute procedure due to external error, clean poisons: {}",
|
||||
clean_poisons
|
||||
))]
|
||||
External {
|
||||
source: BoxedError,
|
||||
clean_poisons: bool,
|
||||
},
|
||||
|
||||
#[snafu(display("Loader {} is already registered", name))]
|
||||
LoaderConflict {
|
||||
@@ -66,6 +73,32 @@ pub enum Error {
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to put poison, key: '{key}', token: '{token}'"))]
|
||||
PutPoison {
|
||||
key: String,
|
||||
token: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to get poison, key: '{key}'"))]
|
||||
GetPoison {
|
||||
key: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to delete poison, key: '{key}', token: '{token}'"))]
|
||||
DeletePoison {
|
||||
key: String,
|
||||
token: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to delete {}", key))]
|
||||
DeleteState {
|
||||
key: String,
|
||||
@@ -175,6 +208,21 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Procedure not found, procedure_id: {}", procedure_id))]
|
||||
ProcedureNotFound {
|
||||
procedure_id: ProcedureId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Poison key not defined, key: '{key}', procedure_id: '{procedure_id}'"))]
|
||||
PoisonKeyNotDefined {
|
||||
key: PoisonKey,
|
||||
procedure_id: ProcedureId,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -185,14 +233,18 @@ impl ErrorExt for Error {
|
||||
Error::External { source, .. }
|
||||
| Error::PutState { source, .. }
|
||||
| Error::DeleteStates { source, .. }
|
||||
| Error::ListState { source, .. } => source.status_code(),
|
||||
| Error::ListState { source, .. }
|
||||
| Error::PutPoison { source, .. }
|
||||
| Error::DeletePoison { source, .. }
|
||||
| Error::GetPoison { source, .. } => source.status_code(),
|
||||
|
||||
Error::ToJson { .. }
|
||||
| Error::DeleteState { .. }
|
||||
| Error::FromJson { .. }
|
||||
| Error::WaitWatcher { .. }
|
||||
| Error::RetryLater { .. }
|
||||
| Error::RollbackProcedureRecovered { .. } => StatusCode::Internal,
|
||||
| Error::RollbackProcedureRecovered { .. }
|
||||
| Error::PoisonKeyNotDefined { .. } => StatusCode::Internal,
|
||||
|
||||
Error::RetryTimesExceeded { .. }
|
||||
| Error::RollbackTimesExceeded { .. }
|
||||
@@ -204,7 +256,8 @@ impl ErrorExt for Error {
|
||||
}
|
||||
Error::ProcedurePanic { .. }
|
||||
| Error::ParseSegmentKey { .. }
|
||||
| Error::Unexpected { .. } => StatusCode::Unexpected,
|
||||
| Error::Unexpected { .. }
|
||||
| &Error::ProcedureNotFound { .. } => StatusCode::Unexpected,
|
||||
Error::ProcedureExec { source, .. } => source.status_code(),
|
||||
Error::StartRemoveOutdatedMetaTask { source, .. }
|
||||
| Error::StopRemoveOutdatedMetaTask { source, .. } => source.status_code(),
|
||||
@@ -221,6 +274,15 @@ impl Error {
|
||||
pub fn external<E: ErrorExt + Send + Sync + 'static>(err: E) -> Error {
|
||||
Error::External {
|
||||
source: BoxedError::new(err),
|
||||
clean_poisons: false,
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates a new [Error::External] error from source `err` and clean poisons.
|
||||
pub fn external_and_clean_poisons<E: ErrorExt + Send + Sync + 'static>(err: E) -> Error {
|
||||
Error::External {
|
||||
source: BoxedError::new(err),
|
||||
clean_poisons: true,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -236,6 +298,11 @@ impl Error {
|
||||
matches!(self, Error::RetryLater { .. })
|
||||
}
|
||||
|
||||
/// Determine whether it needs to clean poisons.
|
||||
pub fn need_clean_poisons(&self) -> bool {
|
||||
matches!(self, Error::External { clean_poisons, .. } if *clean_poisons)
|
||||
}
|
||||
|
||||
/// Creates a new [Error::RetryLater] or [Error::External] error from source `err` according
|
||||
/// to its [StatusCode].
|
||||
pub fn from_error_ext<E: ErrorExt + Send + Sync + 'static>(err: E) -> Self {
|
||||
|
||||
@@ -23,10 +23,13 @@ mod procedure;
|
||||
pub mod store;
|
||||
pub mod watcher;
|
||||
|
||||
#[cfg(any(test, feature = "testing"))]
|
||||
pub mod test_util;
|
||||
|
||||
pub use crate::error::{Error, Result};
|
||||
pub use crate::procedure::{
|
||||
BoxedProcedure, BoxedProcedureLoader, Context, ContextProvider, LockKey, Output, ParseIdError,
|
||||
Procedure, ProcedureId, ProcedureInfo, ProcedureManager, ProcedureManagerRef, ProcedureState,
|
||||
ProcedureWithId, Status, StringKey,
|
||||
PoisonKey, PoisonKeys, Procedure, ProcedureId, ProcedureInfo, ProcedureManager,
|
||||
ProcedureManagerRef, ProcedureState, ProcedureWithId, Status, StringKey,
|
||||
};
|
||||
pub use crate::watcher::Watcher;
|
||||
|
||||
@@ -25,21 +25,23 @@ use backon::ExponentialBuilder;
|
||||
use common_runtime::{RepeatedTask, TaskFunction};
|
||||
use common_telemetry::tracing_context::{FutureExt, TracingContext};
|
||||
use common_telemetry::{error, info, tracing};
|
||||
use snafu::{ensure, ResultExt};
|
||||
use snafu::{ensure, OptionExt, ResultExt};
|
||||
use tokio::sync::watch::{self, Receiver, Sender};
|
||||
use tokio::sync::{Mutex as TokioMutex, Notify};
|
||||
|
||||
use self::rwlock::KeyRwLock;
|
||||
use crate::error::{
|
||||
self, DuplicateProcedureSnafu, Error, LoaderConflictSnafu, ManagerNotStartSnafu, Result,
|
||||
StartRemoveOutdatedMetaTaskSnafu, StopRemoveOutdatedMetaTaskSnafu,
|
||||
self, DuplicateProcedureSnafu, Error, LoaderConflictSnafu, ManagerNotStartSnafu,
|
||||
PoisonKeyNotDefinedSnafu, ProcedureNotFoundSnafu, Result, StartRemoveOutdatedMetaTaskSnafu,
|
||||
StopRemoveOutdatedMetaTaskSnafu,
|
||||
};
|
||||
use crate::local::runner::Runner;
|
||||
use crate::procedure::{BoxedProcedureLoader, InitProcedureState, ProcedureInfo};
|
||||
use crate::procedure::{BoxedProcedureLoader, InitProcedureState, PoisonKeys, ProcedureInfo};
|
||||
use crate::store::poison_store::PoisonStoreRef;
|
||||
use crate::store::{ProcedureMessage, ProcedureMessages, ProcedureStore, StateStoreRef};
|
||||
use crate::{
|
||||
BoxedProcedure, ContextProvider, LockKey, ProcedureId, ProcedureManager, ProcedureState,
|
||||
ProcedureWithId, Watcher,
|
||||
BoxedProcedure, ContextProvider, LockKey, PoisonKey, ProcedureId, ProcedureManager,
|
||||
ProcedureState, ProcedureWithId, Watcher,
|
||||
};
|
||||
|
||||
/// The expired time of a procedure's metadata.
|
||||
@@ -65,6 +67,8 @@ pub(crate) struct ProcedureMeta {
|
||||
child_notify: Notify,
|
||||
/// Lock required by this procedure.
|
||||
lock_key: LockKey,
|
||||
/// Poison keys that may cause this procedure to become poisoned during execution.
|
||||
poison_keys: PoisonKeys,
|
||||
/// Sender to notify the procedure state.
|
||||
state_sender: Sender<ProcedureState>,
|
||||
/// Receiver to watch the procedure state.
|
||||
@@ -83,6 +87,7 @@ impl ProcedureMeta {
|
||||
procedure_state: ProcedureState,
|
||||
parent_id: Option<ProcedureId>,
|
||||
lock_key: LockKey,
|
||||
poison_keys: PoisonKeys,
|
||||
type_name: &str,
|
||||
) -> ProcedureMeta {
|
||||
let (state_sender, state_receiver) = watch::channel(procedure_state);
|
||||
@@ -91,6 +96,7 @@ impl ProcedureMeta {
|
||||
parent_id,
|
||||
child_notify: Notify::new(),
|
||||
lock_key,
|
||||
poison_keys,
|
||||
state_sender,
|
||||
state_receiver,
|
||||
children: Mutex::new(Vec::new()),
|
||||
@@ -163,6 +169,8 @@ pub(crate) struct ManagerContext {
|
||||
finished_procedures: Mutex<VecDeque<(ProcedureId, Instant)>>,
|
||||
/// Running flag.
|
||||
running: Arc<AtomicBool>,
|
||||
/// Poison manager.
|
||||
poison_manager: PoisonStoreRef,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
@@ -170,11 +178,33 @@ impl ContextProvider for ManagerContext {
|
||||
async fn procedure_state(&self, procedure_id: ProcedureId) -> Result<Option<ProcedureState>> {
|
||||
Ok(self.state(procedure_id))
|
||||
}
|
||||
|
||||
async fn try_put_poison(&self, key: &PoisonKey, procedure_id: ProcedureId) -> Result<()> {
|
||||
{
|
||||
// validate the procedure exists
|
||||
let procedures = self.procedures.read().unwrap();
|
||||
let procedure = procedures
|
||||
.get(&procedure_id)
|
||||
.context(ProcedureNotFoundSnafu { procedure_id })?;
|
||||
|
||||
// validate the poison key is defined
|
||||
ensure!(
|
||||
procedure.poison_keys.contains(key),
|
||||
PoisonKeyNotDefinedSnafu {
|
||||
key: key.clone(),
|
||||
procedure_id
|
||||
}
|
||||
);
|
||||
}
|
||||
let key = key.to_string();
|
||||
let procedure_id = procedure_id.to_string();
|
||||
self.poison_manager.try_put_poison(key, procedure_id).await
|
||||
}
|
||||
}
|
||||
|
||||
impl ManagerContext {
|
||||
/// Returns a new [ManagerContext].
|
||||
fn new() -> ManagerContext {
|
||||
fn new(poison_manager: PoisonStoreRef) -> ManagerContext {
|
||||
ManagerContext {
|
||||
key_lock: KeyRwLock::new(),
|
||||
loaders: Mutex::new(HashMap::new()),
|
||||
@@ -182,6 +212,7 @@ impl ManagerContext {
|
||||
messages: Mutex::new(HashMap::new()),
|
||||
finished_procedures: Mutex::new(VecDeque::new()),
|
||||
running: Arc::new(AtomicBool::new(false)),
|
||||
poison_manager,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -437,8 +468,12 @@ pub struct LocalManager {
|
||||
|
||||
impl LocalManager {
|
||||
/// Create a new [LocalManager] with specific `config`.
|
||||
pub fn new(config: ManagerConfig, state_store: StateStoreRef) -> LocalManager {
|
||||
let manager_ctx = Arc::new(ManagerContext::new());
|
||||
pub fn new(
|
||||
config: ManagerConfig,
|
||||
state_store: StateStoreRef,
|
||||
poison_store: PoisonStoreRef,
|
||||
) -> LocalManager {
|
||||
let manager_ctx = Arc::new(ManagerContext::new(poison_store));
|
||||
|
||||
LocalManager {
|
||||
manager_ctx,
|
||||
@@ -476,6 +511,7 @@ impl LocalManager {
|
||||
procedure_state,
|
||||
None,
|
||||
procedure.lock_key(),
|
||||
procedure.poison_keys(),
|
||||
procedure.type_name(),
|
||||
));
|
||||
let runner = Runner {
|
||||
@@ -718,6 +754,7 @@ pub(crate) mod test_util {
|
||||
ProcedureState::Running,
|
||||
None,
|
||||
LockKey::default(),
|
||||
PoisonKeys::default(),
|
||||
"ProcedureAdapter",
|
||||
)
|
||||
}
|
||||
@@ -741,11 +778,17 @@ mod tests {
|
||||
use super::*;
|
||||
use crate::error::{self, Error};
|
||||
use crate::store::state_store::ObjectStateStore;
|
||||
use crate::test_util::InMemoryPoisonStore;
|
||||
use crate::{Context, Procedure, Status};
|
||||
|
||||
fn new_test_manager_context() -> ManagerContext {
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::default());
|
||||
ManagerContext::new(poison_manager)
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_manager_context() {
|
||||
let ctx = ManagerContext::new();
|
||||
let ctx = new_test_manager_context();
|
||||
let meta = Arc::new(test_util::procedure_meta_for_test());
|
||||
|
||||
assert!(!ctx.contains_procedure(meta.id));
|
||||
@@ -761,7 +804,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_manager_context_insert_duplicate() {
|
||||
let ctx = ManagerContext::new();
|
||||
let ctx = new_test_manager_context();
|
||||
let meta = Arc::new(test_util::procedure_meta_for_test());
|
||||
|
||||
assert!(ctx.try_insert_procedure(meta.clone()));
|
||||
@@ -783,7 +826,7 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_procedures_in_tree() {
|
||||
let ctx = ManagerContext::new();
|
||||
let ctx = new_test_manager_context();
|
||||
let root = Arc::new(test_util::procedure_meta_for_test());
|
||||
assert!(ctx.try_insert_procedure(root.clone()));
|
||||
|
||||
@@ -807,6 +850,7 @@ mod tests {
|
||||
struct ProcedureToLoad {
|
||||
content: String,
|
||||
lock_key: LockKey,
|
||||
poison_keys: PoisonKeys,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
@@ -826,6 +870,10 @@ mod tests {
|
||||
fn lock_key(&self) -> LockKey {
|
||||
self.lock_key.clone()
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
self.poison_keys.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl ProcedureToLoad {
|
||||
@@ -833,6 +881,7 @@ mod tests {
|
||||
ProcedureToLoad {
|
||||
content: content.to_string(),
|
||||
lock_key: LockKey::default(),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -855,7 +904,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.manager_ctx.start();
|
||||
|
||||
manager
|
||||
@@ -879,7 +929,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(object_store.clone()));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.manager_ctx.start();
|
||||
|
||||
manager
|
||||
@@ -932,7 +983,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.manager_ctx.start();
|
||||
|
||||
let procedure_id = ProcedureId::random();
|
||||
@@ -983,7 +1035,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.manager_ctx.start();
|
||||
|
||||
#[derive(Debug)]
|
||||
@@ -1022,6 +1075,10 @@ mod tests {
|
||||
fn lock_key(&self) -> LockKey {
|
||||
LockKey::single_exclusive("test.submit")
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
PoisonKeys::default()
|
||||
}
|
||||
}
|
||||
|
||||
let check_procedure = |procedure| async {
|
||||
@@ -1059,7 +1116,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
|
||||
let mut procedure = ProcedureToLoad::new("submit");
|
||||
procedure.lock_key = LockKey::single_exclusive("test.submit");
|
||||
@@ -1086,7 +1144,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
|
||||
manager.start().await.unwrap();
|
||||
manager.stop().await.unwrap();
|
||||
@@ -1121,7 +1180,8 @@ mod tests {
|
||||
remove_outdated_meta_ttl: Duration::from_millis(1),
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(object_store.clone()));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.manager_ctx.set_running();
|
||||
|
||||
let mut procedure = ProcedureToLoad::new("submit");
|
||||
@@ -1196,6 +1256,7 @@ mod tests {
|
||||
content: String,
|
||||
lock_key: LockKey,
|
||||
notify: Option<Arc<Notify>>,
|
||||
poison_keys: PoisonKeys,
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
@@ -1220,6 +1281,10 @@ mod tests {
|
||||
self.notify.as_ref().unwrap().notify_one();
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
self.poison_keys.clone()
|
||||
}
|
||||
}
|
||||
|
||||
impl ProcedureToRecover {
|
||||
@@ -1227,6 +1292,7 @@ mod tests {
|
||||
ProcedureToRecover {
|
||||
content: content.to_string(),
|
||||
lock_key: LockKey::default(),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
notify: None,
|
||||
}
|
||||
}
|
||||
@@ -1236,6 +1302,7 @@ mod tests {
|
||||
let procedure = ProcedureToRecover {
|
||||
content: json.to_string(),
|
||||
lock_key: LockKey::default(),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
notify: Some(notify.clone()),
|
||||
};
|
||||
Ok(Box::new(procedure) as _)
|
||||
@@ -1256,7 +1323,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(object_store.clone()));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::new());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.manager_ctx.start();
|
||||
|
||||
let notify = Arc::new(Notify::new());
|
||||
|
||||
@@ -238,11 +238,34 @@ impl Runner {
|
||||
}
|
||||
ProcedureState::Done { .. } => return,
|
||||
ProcedureState::Failed { .. } => return,
|
||||
ProcedureState::Poisoned { .. } => return,
|
||||
}
|
||||
self.execute_once(ctx).await;
|
||||
}
|
||||
}
|
||||
|
||||
async fn clean_poisons(&mut self) -> Result<()> {
|
||||
let mut error = None;
|
||||
for key in self.meta.poison_keys.iter() {
|
||||
let key = key.to_string();
|
||||
if let Err(e) = self
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.delete_poison(key, self.meta.id.to_string())
|
||||
.await
|
||||
{
|
||||
error!(e; "Failed to clean poisons for procedure: {}", self.meta.id);
|
||||
error = Some(e);
|
||||
}
|
||||
}
|
||||
|
||||
// returns the last error if any.
|
||||
if let Some(e) = error {
|
||||
return Err(e);
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn rollback(&mut self, ctx: &Context, err: Arc<Error>) {
|
||||
if self.procedure.rollback_supported() {
|
||||
if let Err(e) = self.procedure.rollback(ctx).await {
|
||||
@@ -255,7 +278,7 @@ impl Runner {
|
||||
}
|
||||
|
||||
async fn prepare_rollback(&mut self, err: Arc<Error>) {
|
||||
if let Err(e) = self.write_procedure_state(err.to_string()).await {
|
||||
if let Err(e) = self.write_rollback_procedure_state(err.to_string()).await {
|
||||
self.meta
|
||||
.set_state(ProcedureState::prepare_rollback(Arc::new(e)));
|
||||
return;
|
||||
@@ -288,26 +311,48 @@ impl Runner {
|
||||
return;
|
||||
}
|
||||
|
||||
// Cleans poisons before persist.
|
||||
if status.need_clean_poisons() {
|
||||
if let Err(e) = self.clean_poisons().await {
|
||||
error!(e; "Failed to clean poison for procedure: {}", self.meta.id);
|
||||
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if status.need_persist() {
|
||||
if let Err(err) = self.persist_procedure().await {
|
||||
self.meta.set_state(ProcedureState::retrying(Arc::new(err)));
|
||||
if let Err(e) = self.persist_procedure().await {
|
||||
error!(e; "Failed to persist procedure: {}", self.meta.id);
|
||||
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
match status {
|
||||
Status::Executing { .. } => (),
|
||||
Status::Executing { .. } => {}
|
||||
Status::Suspended { subprocedures, .. } => {
|
||||
self.on_suspended(subprocedures).await;
|
||||
}
|
||||
Status::Done { output } => {
|
||||
if let Err(e) = self.commit_procedure().await {
|
||||
error!(e; "Failed to commit procedure: {}", self.meta.id);
|
||||
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
|
||||
return;
|
||||
}
|
||||
|
||||
self.done(output);
|
||||
}
|
||||
Status::Poisoned { error, keys } => {
|
||||
error!(
|
||||
error;
|
||||
"Procedure {}-{} is poisoned, keys: {:?}",
|
||||
self.procedure.type_name(),
|
||||
self.meta.id,
|
||||
keys,
|
||||
);
|
||||
self.meta
|
||||
.set_state(ProcedureState::poisoned(keys, Arc::new(error)));
|
||||
}
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
@@ -327,6 +372,14 @@ impl Runner {
|
||||
return;
|
||||
}
|
||||
|
||||
if e.need_clean_poisons() {
|
||||
if let Err(e) = self.clean_poisons().await {
|
||||
error!(e; "Failed to clean poison for procedure: {}", self.meta.id);
|
||||
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
if e.is_retry_later() {
|
||||
self.meta.set_state(ProcedureState::retrying(Arc::new(e)));
|
||||
return;
|
||||
@@ -339,7 +392,9 @@ impl Runner {
|
||||
}
|
||||
ProcedureState::PrepareRollback { error } => self.prepare_rollback(error).await,
|
||||
ProcedureState::RollingBack { error } => self.rollback(ctx, error).await,
|
||||
ProcedureState::Failed { .. } | ProcedureState::Done { .. } => (),
|
||||
ProcedureState::Failed { .. }
|
||||
| ProcedureState::Done { .. }
|
||||
| ProcedureState::Poisoned { .. } => (),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -372,6 +427,7 @@ impl Runner {
|
||||
procedure_state,
|
||||
Some(self.meta.id),
|
||||
procedure.lock_key(),
|
||||
procedure.poison_keys(),
|
||||
procedure.type_name(),
|
||||
));
|
||||
let runner = Runner {
|
||||
@@ -494,7 +550,7 @@ impl Runner {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn write_procedure_state(&mut self, error: String) -> Result<()> {
|
||||
async fn write_rollback_procedure_state(&mut self, error: String) -> Result<()> {
|
||||
// Persists procedure state
|
||||
let type_name = self.procedure.type_name().to_string();
|
||||
let data = self.procedure.dump()?;
|
||||
@@ -549,8 +605,10 @@ mod tests {
|
||||
|
||||
use super::*;
|
||||
use crate::local::test_util;
|
||||
use crate::procedure::PoisonKeys;
|
||||
use crate::store::proc_path;
|
||||
use crate::{ContextProvider, Error, LockKey, Procedure};
|
||||
use crate::test_util::InMemoryPoisonStore;
|
||||
use crate::{ContextProvider, Error, LockKey, PoisonKey, Procedure};
|
||||
|
||||
const ROOT_ID: &str = "9f805a1f-05f7-490c-9f91-bd56e3cc54c1";
|
||||
|
||||
@@ -562,7 +620,9 @@ mod tests {
|
||||
Runner {
|
||||
meta,
|
||||
procedure,
|
||||
manager_ctx: Arc::new(ManagerContext::new()),
|
||||
manager_ctx: Arc::new(ManagerContext::new(
|
||||
Arc::new(InMemoryPoisonStore::default()),
|
||||
)),
|
||||
step: 0,
|
||||
exponential_builder: ExponentialBuilder::default(),
|
||||
store,
|
||||
@@ -587,6 +647,16 @@ mod tests {
|
||||
assert_eq!(files, files_in_dir);
|
||||
}
|
||||
|
||||
fn context_with_provider(
|
||||
procedure_id: ProcedureId,
|
||||
provider: Arc<dyn ContextProvider>,
|
||||
) -> Context {
|
||||
Context {
|
||||
procedure_id,
|
||||
provider,
|
||||
}
|
||||
}
|
||||
|
||||
fn context_without_provider(procedure_id: ProcedureId) -> Context {
|
||||
struct MockProvider;
|
||||
|
||||
@@ -598,6 +668,14 @@ mod tests {
|
||||
) -> Result<Option<ProcedureState>> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
async fn try_put_poison(
|
||||
&self,
|
||||
_key: &PoisonKey,
|
||||
_procedure_id: ProcedureId,
|
||||
) -> Result<()> {
|
||||
unimplemented!()
|
||||
}
|
||||
}
|
||||
|
||||
Context {
|
||||
@@ -611,6 +689,7 @@ mod tests {
|
||||
struct ProcedureAdapter<F> {
|
||||
data: String,
|
||||
lock_key: LockKey,
|
||||
poison_keys: PoisonKeys,
|
||||
exec_fn: F,
|
||||
rollback_fn: Option<RollbackFn>,
|
||||
}
|
||||
@@ -620,6 +699,7 @@ mod tests {
|
||||
let mut meta = test_util::procedure_meta_for_test();
|
||||
meta.id = ProcedureId::parse_str(uuid).unwrap();
|
||||
meta.lock_key = self.lock_key.clone();
|
||||
meta.poison_keys = self.poison_keys.clone();
|
||||
|
||||
Arc::new(meta)
|
||||
}
|
||||
@@ -657,6 +737,10 @@ mod tests {
|
||||
fn lock_key(&self) -> LockKey {
|
||||
self.lock_key.clone()
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
self.poison_keys.clone()
|
||||
}
|
||||
}
|
||||
|
||||
async fn execute_once_normal(persist: bool, first_files: &[&str], second_files: &[&str]) {
|
||||
@@ -665,7 +749,7 @@ mod tests {
|
||||
times += 1;
|
||||
async move {
|
||||
if times == 1 {
|
||||
Ok(Status::Executing { persist })
|
||||
Ok(Status::executing(persist))
|
||||
} else {
|
||||
Ok(Status::done())
|
||||
}
|
||||
@@ -675,6 +759,7 @@ mod tests {
|
||||
let normal = ProcedureAdapter {
|
||||
data: "normal".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -739,6 +824,7 @@ mod tests {
|
||||
let suspend = ProcedureAdapter {
|
||||
data: "suspend".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -763,7 +849,7 @@ mod tests {
|
||||
async move {
|
||||
if times == 1 {
|
||||
time::sleep(Duration::from_millis(200)).await;
|
||||
Ok(Status::Executing { persist: true })
|
||||
Ok(Status::executing(true))
|
||||
} else {
|
||||
Ok(Status::done())
|
||||
}
|
||||
@@ -773,6 +859,7 @@ mod tests {
|
||||
let child = ProcedureAdapter {
|
||||
data: "child".to_string(),
|
||||
lock_key: LockKey::new_exclusive(keys.iter().map(|k| k.to_string())),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -842,6 +929,7 @@ mod tests {
|
||||
let parent = ProcedureAdapter {
|
||||
data: "parent".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -853,7 +941,8 @@ mod tests {
|
||||
let object_store = test_util::new_object_store(&dir);
|
||||
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
|
||||
let mut runner = new_runner(meta.clone(), Box::new(parent), procedure_store.clone());
|
||||
let manager_ctx = Arc::new(ManagerContext::new());
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::default());
|
||||
let manager_ctx = Arc::new(ManagerContext::new(poison_manager));
|
||||
manager_ctx.start();
|
||||
// Manually add this procedure to the manager ctx.
|
||||
assert!(manager_ctx.try_insert_procedure(meta));
|
||||
@@ -885,10 +974,11 @@ mod tests {
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_running_is_stopped() {
|
||||
let exec_fn = move |_| async move { Ok(Status::Executing { persist: true }) }.boxed();
|
||||
let exec_fn = move |_| async move { Ok(Status::executing(true)) }.boxed();
|
||||
let normal = ProcedureAdapter {
|
||||
data: "normal".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -933,6 +1023,7 @@ mod tests {
|
||||
let normal = ProcedureAdapter {
|
||||
data: "fail".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -959,6 +1050,7 @@ mod tests {
|
||||
let fail = ProcedureAdapter {
|
||||
data: "fail".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -995,6 +1087,7 @@ mod tests {
|
||||
let fail = ProcedureAdapter {
|
||||
data: "fail".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: Some(Box::new(rollback_fn)),
|
||||
};
|
||||
@@ -1046,6 +1139,7 @@ mod tests {
|
||||
let retry_later = ProcedureAdapter {
|
||||
data: "retry_later".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -1082,6 +1176,7 @@ mod tests {
|
||||
let exceed_max_retry_later = ProcedureAdapter {
|
||||
data: "exceed_max_retry_later".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -1117,6 +1212,7 @@ mod tests {
|
||||
let exceed_max_retry_later = ProcedureAdapter {
|
||||
data: "exceed_max_rollback".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: Some(Box::new(rollback_fn)),
|
||||
};
|
||||
@@ -1159,6 +1255,7 @@ mod tests {
|
||||
let retry_later = ProcedureAdapter {
|
||||
data: "rollback_after_retry_fail".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: Some(Box::new(rollback_fn)),
|
||||
};
|
||||
@@ -1203,6 +1300,7 @@ mod tests {
|
||||
let fail = ProcedureAdapter {
|
||||
data: "fail".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table.region-0"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -1238,6 +1336,7 @@ mod tests {
|
||||
let parent = ProcedureAdapter {
|
||||
data: "parent".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::default(),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
@@ -1248,7 +1347,8 @@ mod tests {
|
||||
let object_store = test_util::new_object_store(&dir);
|
||||
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
|
||||
let mut runner = new_runner(meta.clone(), Box::new(parent), procedure_store);
|
||||
let manager_ctx = Arc::new(ManagerContext::new());
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::default());
|
||||
let manager_ctx = Arc::new(ManagerContext::new(poison_manager));
|
||||
manager_ctx.start();
|
||||
// Manually add this procedure to the manager ctx.
|
||||
assert!(manager_ctx.try_insert_procedure(meta.clone()));
|
||||
@@ -1261,4 +1361,327 @@ mod tests {
|
||||
let err = meta.state().error().unwrap().output_msg();
|
||||
assert!(err.contains("subprocedure failed"), "{err}");
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_execute_with_clean_poisons() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut times = 0;
|
||||
let poison_key = PoisonKey::new("table/1024");
|
||||
let moved_poison_key = poison_key.clone();
|
||||
let exec_fn = move |ctx: Context| {
|
||||
times += 1;
|
||||
let poison_key = moved_poison_key.clone();
|
||||
async move {
|
||||
if times == 1 {
|
||||
// Put the poison to the context.
|
||||
ctx.provider
|
||||
.try_put_poison(&poison_key, ctx.procedure_id)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
Ok(Status::executing(true))
|
||||
} else {
|
||||
Ok(Status::executing_with_clean_poisons(true))
|
||||
}
|
||||
}
|
||||
.boxed()
|
||||
};
|
||||
let poison = ProcedureAdapter {
|
||||
data: "poison".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
|
||||
let dir = create_temp_dir("clean_poisons");
|
||||
let meta = poison.new_meta(ROOT_ID);
|
||||
|
||||
let object_store = test_util::new_object_store(&dir);
|
||||
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
|
||||
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
|
||||
|
||||
// Use the manager ctx as the context provider.
|
||||
let ctx = context_with_provider(
|
||||
meta.id,
|
||||
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
|
||||
);
|
||||
// Manually add this procedure to the manager ctx.
|
||||
runner
|
||||
.manager_ctx
|
||||
.procedures
|
||||
.write()
|
||||
.unwrap()
|
||||
.insert(meta.id, runner.meta.clone());
|
||||
|
||||
runner.manager_ctx.start();
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_running(), "{state:?}");
|
||||
|
||||
let procedure_id = runner
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.get_poison(&poison_key.to_string())
|
||||
.await
|
||||
.unwrap();
|
||||
// poison key should be exist.
|
||||
assert!(procedure_id.is_some());
|
||||
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_running(), "{state:?}");
|
||||
|
||||
let procedure_id = runner
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.get_poison(&poison_key.to_string())
|
||||
.await
|
||||
.unwrap();
|
||||
// poison key should be deleted.
|
||||
assert!(procedure_id.is_none());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_execute_error_with_clean_poisons() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut times = 0;
|
||||
let poison_key = PoisonKey::new("table/1024");
|
||||
let moved_poison_key = poison_key.clone();
|
||||
let exec_fn = move |ctx: Context| {
|
||||
times += 1;
|
||||
let poison_key = moved_poison_key.clone();
|
||||
async move {
|
||||
if times == 1 {
|
||||
// Put the poison to the context.
|
||||
ctx.provider
|
||||
.try_put_poison(&poison_key, ctx.procedure_id)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
Ok(Status::executing(true))
|
||||
} else {
|
||||
Err(Error::external_and_clean_poisons(MockError::new(
|
||||
StatusCode::Unexpected,
|
||||
)))
|
||||
}
|
||||
}
|
||||
.boxed()
|
||||
};
|
||||
let poison = ProcedureAdapter {
|
||||
data: "poison".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
|
||||
let dir = create_temp_dir("error_with_clean_poisons");
|
||||
let meta = poison.new_meta(ROOT_ID);
|
||||
|
||||
let object_store = test_util::new_object_store(&dir);
|
||||
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
|
||||
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
|
||||
|
||||
// Use the manager ctx as the context provider.
|
||||
let ctx = context_with_provider(
|
||||
meta.id,
|
||||
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
|
||||
);
|
||||
// Manually add this procedure to the manager ctx.
|
||||
runner
|
||||
.manager_ctx
|
||||
.procedures
|
||||
.write()
|
||||
.unwrap()
|
||||
.insert(meta.id, runner.meta.clone());
|
||||
|
||||
runner.manager_ctx.start();
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_running(), "{state:?}");
|
||||
|
||||
let procedure_id = runner
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.get_poison(&poison_key.to_string())
|
||||
.await
|
||||
.unwrap();
|
||||
// poison key should be exist.
|
||||
assert!(procedure_id.is_some());
|
||||
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_prepare_rollback(), "{state:?}");
|
||||
|
||||
let procedure_id = runner
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.get_poison(&poison_key.to_string())
|
||||
.await
|
||||
.unwrap();
|
||||
// poison key should be deleted.
|
||||
assert!(procedure_id.is_none());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_execute_failed_after_set_poison() {
|
||||
let mut times = 0;
|
||||
let poison_key = PoisonKey::new("table/1024");
|
||||
let moved_poison_key = poison_key.clone();
|
||||
let exec_fn = move |ctx: Context| {
|
||||
times += 1;
|
||||
let poison_key = moved_poison_key.clone();
|
||||
async move {
|
||||
if times == 1 {
|
||||
Ok(Status::executing(true))
|
||||
} else {
|
||||
// Put the poison to the context.
|
||||
ctx.provider
|
||||
.try_put_poison(&poison_key, ctx.procedure_id)
|
||||
.await
|
||||
.unwrap();
|
||||
Err(Error::external(MockError::new(StatusCode::Unexpected)))
|
||||
}
|
||||
}
|
||||
.boxed()
|
||||
};
|
||||
let poison = ProcedureAdapter {
|
||||
data: "poison".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
|
||||
let dir = create_temp_dir("poison");
|
||||
let meta = poison.new_meta(ROOT_ID);
|
||||
|
||||
let object_store = test_util::new_object_store(&dir);
|
||||
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
|
||||
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
|
||||
|
||||
// Use the manager ctx as the context provider.
|
||||
let ctx = context_with_provider(
|
||||
meta.id,
|
||||
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
|
||||
);
|
||||
// Manually add this procedure to the manager ctx.
|
||||
runner
|
||||
.manager_ctx
|
||||
.procedures
|
||||
.write()
|
||||
.unwrap()
|
||||
.insert(meta.id, runner.meta.clone());
|
||||
|
||||
runner.manager_ctx.start();
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_running(), "{state:?}");
|
||||
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_prepare_rollback(), "{state:?}");
|
||||
assert!(meta.state().is_prepare_rollback());
|
||||
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_failed(), "{state:?}");
|
||||
assert!(meta.state().is_failed());
|
||||
|
||||
// Check the poison is set.
|
||||
let procedure_id = runner
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.get_poison(&poison_key.to_string())
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
// If the procedure is poisoned, the poison key shouldn't be deleted.
|
||||
assert_eq!(&procedure_id.to_string(), ROOT_ID);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_execute_poisoned() {
|
||||
let mut times = 0;
|
||||
let poison_key = PoisonKey::new("table/1024");
|
||||
let moved_poison_key = poison_key.clone();
|
||||
let exec_fn = move |ctx: Context| {
|
||||
times += 1;
|
||||
let poison_key = moved_poison_key.clone();
|
||||
async move {
|
||||
if times == 1 {
|
||||
Ok(Status::executing(true))
|
||||
} else {
|
||||
// Put the poison to the context.
|
||||
ctx.provider
|
||||
.try_put_poison(&poison_key, ctx.procedure_id)
|
||||
.await
|
||||
.unwrap();
|
||||
Ok(Status::Poisoned {
|
||||
keys: PoisonKeys::new(vec![poison_key.clone()]),
|
||||
error: Error::external(MockError::new(StatusCode::Unexpected)),
|
||||
})
|
||||
}
|
||||
}
|
||||
.boxed()
|
||||
};
|
||||
let poison = ProcedureAdapter {
|
||||
data: "poison".to_string(),
|
||||
lock_key: LockKey::single_exclusive("catalog.schema.table"),
|
||||
poison_keys: PoisonKeys::new(vec![poison_key.clone()]),
|
||||
exec_fn,
|
||||
rollback_fn: None,
|
||||
};
|
||||
|
||||
let dir = create_temp_dir("poison");
|
||||
let meta = poison.new_meta(ROOT_ID);
|
||||
|
||||
let object_store = test_util::new_object_store(&dir);
|
||||
let procedure_store = Arc::new(ProcedureStore::from_object_store(object_store.clone()));
|
||||
let mut runner = new_runner(meta.clone(), Box::new(poison), procedure_store.clone());
|
||||
|
||||
// Use the manager ctx as the context provider.
|
||||
let ctx = context_with_provider(
|
||||
meta.id,
|
||||
runner.manager_ctx.clone() as Arc<dyn ContextProvider>,
|
||||
);
|
||||
// Manually add this procedure to the manager ctx.
|
||||
runner
|
||||
.manager_ctx
|
||||
.procedures
|
||||
.write()
|
||||
.unwrap()
|
||||
.insert(meta.id, runner.meta.clone());
|
||||
|
||||
runner.manager_ctx.start();
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_running(), "{state:?}");
|
||||
|
||||
runner.execute_once(&ctx).await;
|
||||
let state = runner.meta.state();
|
||||
assert!(state.is_poisoned(), "{state:?}");
|
||||
assert!(meta.state().is_poisoned());
|
||||
check_files(
|
||||
&object_store,
|
||||
&procedure_store,
|
||||
ctx.procedure_id,
|
||||
&["0000000000.step"],
|
||||
)
|
||||
.await;
|
||||
|
||||
// Check the poison is set.
|
||||
let procedure_id = runner
|
||||
.manager_ctx
|
||||
.poison_manager
|
||||
.get_poison(&poison_key.to_string())
|
||||
.await
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
// If the procedure is poisoned, the poison key shouldn't be deleted.
|
||||
assert_eq!(procedure_id, ROOT_ID);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -14,6 +14,7 @@
|
||||
|
||||
use std::any::Any;
|
||||
use std::fmt;
|
||||
use std::fmt::Display;
|
||||
use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
|
||||
@@ -35,6 +36,8 @@ pub enum Status {
|
||||
Executing {
|
||||
/// Whether the framework needs to persist the procedure.
|
||||
persist: bool,
|
||||
/// Whether the framework needs to clean the poisons.
|
||||
clean_poisons: bool,
|
||||
},
|
||||
/// The procedure has suspended itself and is waiting for subprocedures.
|
||||
Suspended {
|
||||
@@ -42,14 +45,40 @@ pub enum Status {
|
||||
/// Whether the framework needs to persist the procedure.
|
||||
persist: bool,
|
||||
},
|
||||
/// The procedure is poisoned.
|
||||
Poisoned {
|
||||
/// The keys that cause the procedure to be poisoned.
|
||||
keys: PoisonKeys,
|
||||
/// The error that cause the procedure to be poisoned.
|
||||
error: Error,
|
||||
},
|
||||
/// the procedure is done.
|
||||
Done { output: Option<Output> },
|
||||
}
|
||||
|
||||
impl Status {
|
||||
/// Returns a [Status::Poisoned] with given `keys` and `error`.
|
||||
pub fn poisoned(keys: impl IntoIterator<Item = PoisonKey>, error: Error) -> Status {
|
||||
Status::Poisoned {
|
||||
keys: PoisonKeys::new(keys),
|
||||
error,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a [Status::Executing] with given `persist` flag.
|
||||
pub fn executing(persist: bool) -> Status {
|
||||
Status::Executing { persist }
|
||||
Status::Executing {
|
||||
persist,
|
||||
clean_poisons: false,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a [Status::Executing] with given `persist` flag and clean poisons.
|
||||
pub fn executing_with_clean_poisons(persist: bool) -> Status {
|
||||
Status::Executing {
|
||||
persist,
|
||||
clean_poisons: true,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns a [Status::Done] without output.
|
||||
@@ -86,11 +115,20 @@ impl Status {
|
||||
|
||||
/// Returns `true` if the procedure needs the framework to persist its intermediate state.
|
||||
pub fn need_persist(&self) -> bool {
|
||||
// If the procedure is done, the framework doesn't need to persist the procedure
|
||||
// anymore. It only needs to mark the procedure as committed.
|
||||
match self {
|
||||
Status::Executing { persist } | Status::Suspended { persist, .. } => *persist,
|
||||
Status::Done { .. } => false,
|
||||
// If the procedure is done/poisoned, the framework doesn't need to persist the procedure
|
||||
// anymore. It only needs to mark the procedure as committed.
|
||||
Status::Executing { persist, .. } | Status::Suspended { persist, .. } => *persist,
|
||||
Status::Done { .. } | Status::Poisoned { .. } => false,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns `true` if the framework needs to clean the poisons.
|
||||
pub fn need_clean_poisons(&self) -> bool {
|
||||
match self {
|
||||
Status::Executing { clean_poisons, .. } => *clean_poisons,
|
||||
Status::Done { .. } => true,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -100,6 +138,12 @@ impl Status {
|
||||
pub trait ContextProvider: Send + Sync {
|
||||
/// Query the procedure state.
|
||||
async fn procedure_state(&self, procedure_id: ProcedureId) -> Result<Option<ProcedureState>>;
|
||||
|
||||
/// Try to put a poison key for a procedure.
|
||||
///
|
||||
/// This method is used to mark a resource as being operated on by a procedure.
|
||||
/// If the poison key already exists with a different value, the operation will fail.
|
||||
async fn try_put_poison(&self, key: &PoisonKey, procedure_id: ProcedureId) -> Result<()>;
|
||||
}
|
||||
|
||||
/// Reference-counted pointer to [ContextProvider].
|
||||
@@ -147,6 +191,11 @@ pub trait Procedure: Send {
|
||||
|
||||
/// Returns the [LockKey] that this procedure needs to acquire.
|
||||
fn lock_key(&self) -> LockKey;
|
||||
|
||||
/// Returns the [PoisonKeys] that may cause this procedure to become poisoned during execution.
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
PoisonKeys::default()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait]
|
||||
@@ -174,6 +223,54 @@ impl<T: Procedure + ?Sized> Procedure for Box<T> {
|
||||
fn lock_key(&self) -> LockKey {
|
||||
(**self).lock_key()
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
(**self).poison_keys()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)]
|
||||
pub struct PoisonKey(String);
|
||||
|
||||
impl Display for PoisonKey {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
write!(f, "{}", self.0)
|
||||
}
|
||||
}
|
||||
|
||||
impl PoisonKey {
|
||||
/// Creates a new [PoisonKey] from a [String].
|
||||
pub fn new(key: impl Into<String>) -> Self {
|
||||
Self(key.into())
|
||||
}
|
||||
}
|
||||
|
||||
/// A collection of [PoisonKey]s.
|
||||
///
|
||||
/// This type is used to represent the keys that may cause the procedure to become poisoned during execution.
|
||||
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, Default)]
|
||||
pub struct PoisonKeys(SmallVec<[PoisonKey; 2]>);
|
||||
|
||||
impl PoisonKeys {
|
||||
/// Creates a new [PoisonKeys] from a [String].
|
||||
pub fn single(key: impl Into<String>) -> Self {
|
||||
Self(smallvec![PoisonKey::new(key)])
|
||||
}
|
||||
|
||||
/// Creates a new [PoisonKeys] from a [PoisonKey].
|
||||
pub fn new(keys: impl IntoIterator<Item = PoisonKey>) -> Self {
|
||||
Self(keys.into_iter().collect())
|
||||
}
|
||||
|
||||
/// Returns `true` if the [PoisonKeys] contains the given [PoisonKey].
|
||||
pub fn contains(&self, key: &PoisonKey) -> bool {
|
||||
self.0.contains(key)
|
||||
}
|
||||
|
||||
/// Returns an iterator over the [PoisonKey]s.
|
||||
pub fn iter(&self) -> impl Iterator<Item = &PoisonKey> {
|
||||
self.0.iter()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone, Debug, PartialEq, Eq, Hash, PartialOrd, Ord)]
|
||||
@@ -325,6 +422,8 @@ pub enum ProcedureState {
|
||||
RollingBack { error: Arc<Error> },
|
||||
/// The procedure is failed and cannot proceed anymore.
|
||||
Failed { error: Arc<Error> },
|
||||
/// The procedure is poisoned.
|
||||
Poisoned { keys: PoisonKeys, error: Arc<Error> },
|
||||
}
|
||||
|
||||
impl ProcedureState {
|
||||
@@ -348,6 +447,11 @@ impl ProcedureState {
|
||||
ProcedureState::Retrying { error }
|
||||
}
|
||||
|
||||
/// Returns a [ProcedureState] with poisoned state.
|
||||
pub fn poisoned(keys: PoisonKeys, error: Arc<Error>) -> ProcedureState {
|
||||
ProcedureState::Poisoned { keys, error }
|
||||
}
|
||||
|
||||
/// Returns true if the procedure state is running.
|
||||
pub fn is_running(&self) -> bool {
|
||||
matches!(self, ProcedureState::Running)
|
||||
@@ -358,6 +462,11 @@ impl ProcedureState {
|
||||
matches!(self, ProcedureState::Done { .. })
|
||||
}
|
||||
|
||||
/// Returns true if the procedure state is poisoned.
|
||||
pub fn is_poisoned(&self) -> bool {
|
||||
matches!(self, ProcedureState::Poisoned { .. })
|
||||
}
|
||||
|
||||
/// Returns true if the procedure state failed.
|
||||
pub fn is_failed(&self) -> bool {
|
||||
matches!(self, ProcedureState::Failed { .. })
|
||||
@@ -384,6 +493,7 @@ impl ProcedureState {
|
||||
ProcedureState::Failed { error } => Some(error),
|
||||
ProcedureState::Retrying { error } => Some(error),
|
||||
ProcedureState::RollingBack { error } => Some(error),
|
||||
ProcedureState::Poisoned { error, .. } => Some(error),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
@@ -397,6 +507,7 @@ impl ProcedureState {
|
||||
ProcedureState::Failed { .. } => "Failed",
|
||||
ProcedureState::PrepareRollback { .. } => "PrepareRollback",
|
||||
ProcedureState::RollingBack { .. } => "RollingBack",
|
||||
ProcedureState::Poisoned { .. } => "Poisoned",
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -470,12 +581,18 @@ mod tests {
|
||||
|
||||
#[test]
|
||||
fn test_status() {
|
||||
let status = Status::Executing { persist: false };
|
||||
let status = Status::executing(false);
|
||||
assert!(!status.need_persist());
|
||||
|
||||
let status = Status::Executing { persist: true };
|
||||
let status = Status::executing(true);
|
||||
assert!(status.need_persist());
|
||||
|
||||
let status = Status::executing_with_clean_poisons(false);
|
||||
assert!(status.need_clean_poisons());
|
||||
|
||||
let status = Status::executing_with_clean_poisons(true);
|
||||
assert!(status.need_clean_poisons());
|
||||
|
||||
let status = Status::Suspended {
|
||||
subprocedures: Vec::new(),
|
||||
persist: false,
|
||||
@@ -490,6 +607,7 @@ mod tests {
|
||||
|
||||
let status = Status::done();
|
||||
assert!(!status.need_persist());
|
||||
assert!(status.need_clean_poisons());
|
||||
}
|
||||
|
||||
#[test]
|
||||
|
||||
@@ -24,6 +24,7 @@ use crate::error::{Result, ToJsonSnafu};
|
||||
pub(crate) use crate::store::state_store::StateStoreRef;
|
||||
use crate::ProcedureId;
|
||||
|
||||
pub mod poison_store;
|
||||
pub mod state_store;
|
||||
pub mod util;
|
||||
|
||||
@@ -341,6 +342,7 @@ mod tests {
|
||||
|
||||
use object_store::ObjectStore;
|
||||
|
||||
use crate::procedure::PoisonKeys;
|
||||
use crate::store::state_store::ObjectStateStore;
|
||||
use crate::BoxedProcedure;
|
||||
|
||||
@@ -503,6 +505,10 @@ mod tests {
|
||||
fn lock_key(&self) -> LockKey {
|
||||
LockKey::default()
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
PoisonKeys::default()
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
|
||||
59
src/common/procedure/src/store/poison_store.rs
Normal file
59
src/common/procedure/src/store/poison_store.rs
Normal file
@@ -0,0 +1,59 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use async_trait::async_trait;
|
||||
|
||||
use crate::error::Result;
|
||||
|
||||
pub type PoisonStoreRef = Arc<dyn PoisonStore>;
|
||||
|
||||
/// Poison store.
|
||||
///
|
||||
/// This trait is used to manage the state of operations on resources, particularly
|
||||
/// when an operation encounters an unrecoverable error, potentially leading to
|
||||
/// metadata inconsistency. In such cases, manual intervention is required to
|
||||
/// resolve the issue before any further operations can be performed on the resource.
|
||||
///
|
||||
/// ## Behavior:
|
||||
/// - **Insertion**: When an operation begins on a resource, a "poison" key is inserted
|
||||
/// into the state store to indicate the operation is in progress.
|
||||
/// - **Deletion**: If the operation completes successfully or
|
||||
/// other cases can ensure the resource is in a consistent state, the poison key is removed
|
||||
/// from the state store, indicating the resource is in a consistent state.
|
||||
/// - **Failure Handling**:
|
||||
/// - If the operation fails or other cases may lead to metadata inconsistency,
|
||||
/// the poison key remains in the state store.
|
||||
/// - The presence of this key indicates that the resource has encountered an
|
||||
/// unrecoverable error and the metadata may be inconsistent.
|
||||
/// - New operations on the same resource are rejected until the resource is
|
||||
/// manually recovered and the poison key is removed.
|
||||
#[async_trait]
|
||||
pub trait PoisonStore: Send + Sync {
|
||||
/// Try to put the poison key.
|
||||
///
|
||||
/// If the poison key already exists with a different value, the operation will fail.
|
||||
async fn try_put_poison(&self, key: String, token: String) -> Result<()>;
|
||||
|
||||
/// Delete the poison key.
|
||||
///
|
||||
/// If the poison key exists with a different value, the operation will fail.
|
||||
async fn delete_poison(&self, key: String, token: String) -> Result<()>;
|
||||
|
||||
/// Get the poison key.
|
||||
///
|
||||
/// If the poison key does not exist, the operation will return `None`.
|
||||
async fn get_poison(&self, key: &str) -> Result<Option<String>>;
|
||||
}
|
||||
85
src/common/procedure/src/test_util.rs
Normal file
85
src/common/procedure/src/test_util.rs
Normal file
@@ -0,0 +1,85 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::hash_map::Entry;
|
||||
use std::collections::HashMap;
|
||||
use std::sync::{Arc, RwLock};
|
||||
|
||||
use snafu::ensure;
|
||||
|
||||
use super::*;
|
||||
use crate::error;
|
||||
use crate::store::poison_store::PoisonStore;
|
||||
|
||||
/// A poison store that uses an in-memory map to store the poison state.
|
||||
#[derive(Debug, Default)]
|
||||
pub struct InMemoryPoisonStore {
|
||||
map: Arc<RwLock<HashMap<String, String>>>,
|
||||
}
|
||||
|
||||
impl InMemoryPoisonStore {
|
||||
/// Create a new in-memory poison manager.
|
||||
pub fn new() -> Self {
|
||||
Self::default()
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl PoisonStore for InMemoryPoisonStore {
|
||||
async fn try_put_poison(&self, key: String, token: String) -> Result<()> {
|
||||
let mut map = self.map.write().unwrap();
|
||||
match map.entry(key) {
|
||||
Entry::Vacant(v) => {
|
||||
v.insert(token.to_string());
|
||||
}
|
||||
Entry::Occupied(o) => {
|
||||
let value = o.get();
|
||||
ensure!(
|
||||
value == &token,
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: format!("The poison is already set by other token {}", value)
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn delete_poison(&self, key: String, token: String) -> Result<()> {
|
||||
let mut map = self.map.write().unwrap();
|
||||
match map.entry(key) {
|
||||
Entry::Vacant(_) => {
|
||||
// do nothing
|
||||
}
|
||||
Entry::Occupied(o) => {
|
||||
let value = o.get();
|
||||
ensure!(
|
||||
value == &token,
|
||||
error::UnexpectedSnafu {
|
||||
err_msg: format!("The poison is not set by the token {}", value)
|
||||
}
|
||||
);
|
||||
|
||||
o.remove();
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn get_poison(&self, key: &str) -> Result<Option<String>> {
|
||||
let map = self.map.read().unwrap();
|
||||
let key = key.to_string();
|
||||
Ok(map.get(&key).cloned())
|
||||
}
|
||||
}
|
||||
@@ -43,6 +43,10 @@ pub async fn wait(watcher: &mut Watcher) -> Result<Option<Output>> {
|
||||
ProcedureState::PrepareRollback { error } => {
|
||||
debug!("commit rollback, source: {}", error)
|
||||
}
|
||||
ProcedureState::Poisoned { error, .. } => {
|
||||
debug!("poisoned, source: {}", error);
|
||||
return Err(error.clone()).context(ProcedureExecSnafu);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -61,7 +65,9 @@ mod tests {
|
||||
use super::*;
|
||||
use crate::error::Error;
|
||||
use crate::local::{test_util, LocalManager, ManagerConfig};
|
||||
use crate::procedure::PoisonKeys;
|
||||
use crate::store::state_store::ObjectStateStore;
|
||||
use crate::test_util::InMemoryPoisonStore;
|
||||
use crate::{
|
||||
Context, LockKey, Procedure, ProcedureId, ProcedureManager, ProcedureWithId, Status,
|
||||
};
|
||||
@@ -76,7 +82,8 @@ mod tests {
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = Arc::new(ObjectStateStore::new(test_util::new_object_store(&dir)));
|
||||
let manager = LocalManager::new(config, state_store);
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::default());
|
||||
let manager = LocalManager::new(config, state_store, poison_manager);
|
||||
manager.start().await.unwrap();
|
||||
|
||||
#[derive(Debug)]
|
||||
@@ -106,6 +113,10 @@ mod tests {
|
||||
fn lock_key(&self) -> LockKey {
|
||||
LockKey::single_exclusive("test.submit")
|
||||
}
|
||||
|
||||
fn poison_keys(&self) -> PoisonKeys {
|
||||
PoisonKeys::default()
|
||||
}
|
||||
}
|
||||
|
||||
let procedure_id = ProcedureId::random();
|
||||
|
||||
@@ -468,6 +468,7 @@ mod test {
|
||||
&[GrantedRegion {
|
||||
region_id: region_id.as_u64(),
|
||||
role: api::v1::meta::RegionRole::Leader.into(),
|
||||
extensions: HashMap::new(),
|
||||
}],
|
||||
Instant::now() + Duration::from_millis(200),
|
||||
)
|
||||
|
||||
@@ -20,7 +20,7 @@ use api::v1::{ColumnDataType, ColumnDataTypeExtension, CreateTableExpr, Semantic
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::key::table_info::TableInfoValue;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema};
|
||||
use itertools::Itertools;
|
||||
use operator::expr_helper;
|
||||
use session::context::QueryContextBuilder;
|
||||
@@ -174,7 +174,15 @@ pub fn table_info_value_to_relation_desc(
|
||||
let default_values = raw_schema
|
||||
.column_schemas
|
||||
.iter()
|
||||
.map(|c| c.default_constraint().cloned())
|
||||
.map(|c| {
|
||||
c.default_constraint().cloned().or_else(|| {
|
||||
if c.is_nullable() {
|
||||
Some(ColumnDefaultConstraint::null_value())
|
||||
} else {
|
||||
None
|
||||
}
|
||||
})
|
||||
})
|
||||
.collect_vec();
|
||||
|
||||
Ok(TableDesc::new(relation_desc, default_values))
|
||||
|
||||
@@ -151,12 +151,12 @@ impl ScalarExpr {
|
||||
|
||||
/// apply optimization to the expression, like flatten variadic function
|
||||
pub fn optimize(&mut self) {
|
||||
self.flatten_varidic_fn();
|
||||
self.flatten_variadic_fn();
|
||||
}
|
||||
|
||||
/// Because Substrait's `And`/`Or` function is binary, but FlowPlan's
|
||||
/// `And`/`Or` function is variadic, we need to flatten the `And` function if multiple `And`/`Or` functions are nested.
|
||||
fn flatten_varidic_fn(&mut self) {
|
||||
fn flatten_variadic_fn(&mut self) {
|
||||
if let ScalarExpr::CallVariadic { func, exprs } = self {
|
||||
let mut new_exprs = vec![];
|
||||
for expr in std::mem::take(exprs) {
|
||||
@@ -167,7 +167,7 @@ impl ScalarExpr {
|
||||
{
|
||||
if *func == inner_func {
|
||||
for inner_expr in inner_exprs.iter_mut() {
|
||||
inner_expr.flatten_varidic_fn();
|
||||
inner_expr.flatten_variadic_fn();
|
||||
}
|
||||
new_exprs.extend(inner_exprs);
|
||||
}
|
||||
|
||||
@@ -145,14 +145,18 @@ impl Instance {
|
||||
.context(error::OpenRaftEngineBackendSnafu)?;
|
||||
|
||||
let kv_backend = Arc::new(kv_backend);
|
||||
let state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
|
||||
let kv_state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
|
||||
|
||||
let manager_config = ManagerConfig {
|
||||
max_retry_times: procedure_config.max_retry_times,
|
||||
retry_delay: procedure_config.retry_delay,
|
||||
..Default::default()
|
||||
};
|
||||
let procedure_manager = Arc::new(LocalManager::new(manager_config, state_store));
|
||||
let procedure_manager = Arc::new(LocalManager::new(
|
||||
manager_config,
|
||||
kv_state_store.clone(),
|
||||
kv_state_store,
|
||||
));
|
||||
|
||||
Ok((kv_backend, procedure_manager))
|
||||
}
|
||||
|
||||
@@ -126,9 +126,9 @@ impl<'a> ElectionSqlFactory<'a> {
|
||||
fn put_value_with_lease_sql(&self) -> String {
|
||||
format!(
|
||||
r#"WITH prev AS (
|
||||
SELECT k, v FROM {} WHERE k = $1
|
||||
SELECT k, v FROM "{}" WHERE k = $1
|
||||
), insert AS (
|
||||
INSERT INTO {}
|
||||
INSERT INTO "{}"
|
||||
VALUES($1, convert_to($2 || '{}' || TO_CHAR(CURRENT_TIMESTAMP + INTERVAL '1 second' * $3, 'YYYY-MM-DD HH24:MI:SS.MS'), 'UTF8'))
|
||||
ON CONFLICT (k) DO NOTHING
|
||||
)
|
||||
@@ -140,7 +140,7 @@ impl<'a> ElectionSqlFactory<'a> {
|
||||
|
||||
fn update_value_with_lease_sql(&self) -> String {
|
||||
format!(
|
||||
r#"UPDATE {}
|
||||
r#"UPDATE "{}"
|
||||
SET v = convert_to($3 || '{}' || TO_CHAR(CURRENT_TIMESTAMP + INTERVAL '1 second' * $4, 'YYYY-MM-DD HH24:MI:SS.MS'), 'UTF8')
|
||||
WHERE k = $1 AND v = $2"#,
|
||||
self.table_name, LEASE_SEP
|
||||
@@ -149,21 +149,21 @@ impl<'a> ElectionSqlFactory<'a> {
|
||||
|
||||
fn get_value_with_lease_sql(&self) -> String {
|
||||
format!(
|
||||
r#"SELECT v, TO_CHAR(CURRENT_TIMESTAMP, 'YYYY-MM-DD HH24:MI:SS.MS') FROM {} WHERE k = $1"#,
|
||||
r#"SELECT v, TO_CHAR(CURRENT_TIMESTAMP, 'YYYY-MM-DD HH24:MI:SS.MS') FROM "{}" WHERE k = $1"#,
|
||||
self.table_name
|
||||
)
|
||||
}
|
||||
|
||||
fn get_value_with_lease_by_prefix_sql(&self) -> String {
|
||||
format!(
|
||||
r#"SELECT v, TO_CHAR(CURRENT_TIMESTAMP, 'YYYY-MM-DD HH24:MI:SS.MS') FROM {} WHERE k LIKE $1"#,
|
||||
r#"SELECT v, TO_CHAR(CURRENT_TIMESTAMP, 'YYYY-MM-DD HH24:MI:SS.MS') FROM "{}" WHERE k LIKE $1"#,
|
||||
self.table_name
|
||||
)
|
||||
}
|
||||
|
||||
fn delete_value_sql(&self) -> String {
|
||||
format!(
|
||||
"DELETE FROM {} WHERE k = $1 RETURNING k,v;",
|
||||
"DELETE FROM \"{}\" WHERE k = $1 RETURNING k,v;",
|
||||
self.table_name
|
||||
)
|
||||
}
|
||||
@@ -285,7 +285,6 @@ impl Election for PgElection {
|
||||
.is_ok()
|
||||
}
|
||||
|
||||
/// TODO(CookiePie): Split the candidate registration and keep alive logic into separate methods, so that upper layers can call them separately.
|
||||
async fn register_candidate(&self, node_info: &MetasrvNodeInfo) -> Result<()> {
|
||||
let key = self.candidate_key();
|
||||
let node_info =
|
||||
@@ -747,7 +746,7 @@ mod tests {
|
||||
});
|
||||
if let Some(table_name) = table_name {
|
||||
let create_table_sql = format!(
|
||||
"CREATE TABLE IF NOT EXISTS {}(k bytea PRIMARY KEY, v bytea);",
|
||||
"CREATE TABLE IF NOT EXISTS \"{}\"(k bytea PRIMARY KEY, v bytea);",
|
||||
table_name
|
||||
);
|
||||
client.execute(&create_table_sql, &[]).await.unwrap();
|
||||
@@ -756,7 +755,7 @@ mod tests {
|
||||
}
|
||||
|
||||
async fn drop_table(client: &Client, table_name: &str) {
|
||||
let sql = format!("DROP TABLE IF EXISTS {};", table_name);
|
||||
let sql = format!("DROP TABLE IF EXISTS \"{}\";", table_name);
|
||||
client.execute(&sql, &[]).await.unwrap();
|
||||
}
|
||||
|
||||
|
||||
231
src/meta-srv/src/handler/collect_leader_region_handler.rs
Normal file
231
src/meta-srv/src/handler/collect_leader_region_handler.rs
Normal file
@@ -0,0 +1,231 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use api::v1::meta::{HeartbeatRequest, Role};
|
||||
use common_meta::region_registry::LeaderRegion;
|
||||
use store_api::region_engine::RegionRole;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::handler::{HandleControl, HeartbeatAccumulator, HeartbeatHandler};
|
||||
use crate::metasrv::Context;
|
||||
|
||||
pub struct CollectLeaderRegionHandler;
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl HeartbeatHandler for CollectLeaderRegionHandler {
|
||||
fn is_acceptable(&self, role: Role) -> bool {
|
||||
role == Role::Datanode
|
||||
}
|
||||
|
||||
async fn handle(
|
||||
&self,
|
||||
_req: &HeartbeatRequest,
|
||||
ctx: &mut Context,
|
||||
acc: &mut HeartbeatAccumulator,
|
||||
) -> Result<HandleControl> {
|
||||
let Some(current_stat) = acc.stat.as_ref() else {
|
||||
return Ok(HandleControl::Continue);
|
||||
};
|
||||
|
||||
let mut key_values = Vec::with_capacity(current_stat.region_stats.len());
|
||||
for stat in current_stat.region_stats.iter() {
|
||||
if stat.role != RegionRole::Leader {
|
||||
continue;
|
||||
}
|
||||
|
||||
let manifest = stat.region_manifest.into();
|
||||
let value = LeaderRegion {
|
||||
datanode_id: current_stat.id,
|
||||
manifest,
|
||||
};
|
||||
key_values.push((stat.id, value));
|
||||
}
|
||||
ctx.leader_region_registry.batch_put(key_values);
|
||||
|
||||
Ok(HandleControl::Continue)
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_meta::cache_invalidator::DummyCacheInvalidator;
|
||||
use common_meta::datanode::{RegionManifestInfo, RegionStat, Stat};
|
||||
use common_meta::key::TableMetadataManager;
|
||||
use common_meta::kv_backend::memory::MemoryKvBackend;
|
||||
use common_meta::region_registry::{LeaderRegionManifestInfo, LeaderRegionRegistry};
|
||||
use common_meta::sequence::SequenceBuilder;
|
||||
use store_api::region_engine::RegionRole;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use super::*;
|
||||
use crate::cluster::MetaPeerClientBuilder;
|
||||
use crate::handler::{HeartbeatMailbox, Pushers};
|
||||
use crate::service::store::cached_kv::LeaderCachedKvBackend;
|
||||
|
||||
fn mock_ctx() -> Context {
|
||||
let in_memory = Arc::new(MemoryKvBackend::new());
|
||||
let kv_backend = Arc::new(MemoryKvBackend::new());
|
||||
let leader_cached_kv_backend = Arc::new(LeaderCachedKvBackend::with_always_leader(
|
||||
kv_backend.clone(),
|
||||
));
|
||||
let seq = SequenceBuilder::new("test_seq", kv_backend.clone()).build();
|
||||
let mailbox = HeartbeatMailbox::create(Pushers::default(), seq);
|
||||
let meta_peer_client = MetaPeerClientBuilder::default()
|
||||
.election(None)
|
||||
.in_memory(in_memory.clone())
|
||||
.build()
|
||||
.map(Arc::new)
|
||||
// Safety: all required fields set at initialization
|
||||
.unwrap();
|
||||
Context {
|
||||
server_addr: "127.0.0.1:0000".to_string(),
|
||||
in_memory,
|
||||
kv_backend: kv_backend.clone(),
|
||||
leader_cached_kv_backend,
|
||||
meta_peer_client,
|
||||
mailbox,
|
||||
election: None,
|
||||
is_infancy: false,
|
||||
table_metadata_manager: Arc::new(TableMetadataManager::new(kv_backend.clone())),
|
||||
cache_invalidator: Arc::new(DummyCacheInvalidator),
|
||||
leader_region_registry: Arc::new(LeaderRegionRegistry::new()),
|
||||
}
|
||||
}
|
||||
|
||||
fn new_region_stat(id: RegionId, manifest_version: u64, role: RegionRole) -> RegionStat {
|
||||
RegionStat {
|
||||
id,
|
||||
region_manifest: RegionManifestInfo::Mito {
|
||||
manifest_version,
|
||||
flushed_entry_id: 0,
|
||||
},
|
||||
rcus: 0,
|
||||
wcus: 0,
|
||||
approximate_bytes: 0,
|
||||
engine: "mito".to_string(),
|
||||
role,
|
||||
num_rows: 0,
|
||||
memtable_size: 0,
|
||||
manifest_size: 0,
|
||||
sst_size: 0,
|
||||
index_size: 0,
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_handle_collect_leader_region() {
|
||||
let mut ctx = mock_ctx();
|
||||
|
||||
let mut acc = HeartbeatAccumulator {
|
||||
stat: Some(Stat {
|
||||
id: 1,
|
||||
region_stats: vec![
|
||||
new_region_stat(RegionId::new(1, 1), 1, RegionRole::Leader),
|
||||
new_region_stat(RegionId::new(1, 2), 2, RegionRole::Follower),
|
||||
],
|
||||
addr: "127.0.0.1:0000".to_string(),
|
||||
region_num: 2,
|
||||
..Default::default()
|
||||
}),
|
||||
..Default::default()
|
||||
};
|
||||
|
||||
let handler = CollectLeaderRegionHandler;
|
||||
let control = handler
|
||||
.handle(&HeartbeatRequest::default(), &mut ctx, &mut acc)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(control, HandleControl::Continue);
|
||||
let regions = ctx
|
||||
.leader_region_registry
|
||||
.batch_get(vec![RegionId::new(1, 1), RegionId::new(1, 2)].into_iter());
|
||||
assert_eq!(regions.len(), 1);
|
||||
assert_eq!(
|
||||
regions.get(&RegionId::new(1, 1)),
|
||||
Some(&LeaderRegion {
|
||||
datanode_id: 1,
|
||||
manifest: LeaderRegionManifestInfo::Mito {
|
||||
manifest_version: 1,
|
||||
flushed_entry_id: 0,
|
||||
},
|
||||
})
|
||||
);
|
||||
|
||||
// New heartbeat with new manifest version
|
||||
acc.stat = Some(Stat {
|
||||
id: 1,
|
||||
region_stats: vec![new_region_stat(RegionId::new(1, 1), 2, RegionRole::Leader)],
|
||||
timestamp_millis: 0,
|
||||
addr: "127.0.0.1:0000".to_string(),
|
||||
region_num: 1,
|
||||
node_epoch: 0,
|
||||
..Default::default()
|
||||
});
|
||||
let control = handler
|
||||
.handle(&HeartbeatRequest::default(), &mut ctx, &mut acc)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(control, HandleControl::Continue);
|
||||
let regions = ctx
|
||||
.leader_region_registry
|
||||
.batch_get(vec![RegionId::new(1, 1)].into_iter());
|
||||
assert_eq!(regions.len(), 1);
|
||||
assert_eq!(
|
||||
regions.get(&RegionId::new(1, 1)),
|
||||
Some(&LeaderRegion {
|
||||
datanode_id: 1,
|
||||
manifest: LeaderRegionManifestInfo::Mito {
|
||||
manifest_version: 2,
|
||||
flushed_entry_id: 0,
|
||||
},
|
||||
})
|
||||
);
|
||||
|
||||
// New heartbeat with old manifest version
|
||||
acc.stat = Some(Stat {
|
||||
id: 1,
|
||||
region_stats: vec![new_region_stat(RegionId::new(1, 1), 1, RegionRole::Leader)],
|
||||
timestamp_millis: 0,
|
||||
addr: "127.0.0.1:0000".to_string(),
|
||||
region_num: 1,
|
||||
node_epoch: 0,
|
||||
..Default::default()
|
||||
});
|
||||
let control = handler
|
||||
.handle(&HeartbeatRequest::default(), &mut ctx, &mut acc)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(control, HandleControl::Continue);
|
||||
let regions = ctx
|
||||
.leader_region_registry
|
||||
.batch_get(vec![RegionId::new(1, 1)].into_iter());
|
||||
assert_eq!(regions.len(), 1);
|
||||
assert_eq!(
|
||||
regions.get(&RegionId::new(1, 1)),
|
||||
// The manifest version is not updated
|
||||
Some(&LeaderRegion {
|
||||
datanode_id: 1,
|
||||
manifest: LeaderRegionManifestInfo::Mito {
|
||||
manifest_version: 2,
|
||||
flushed_entry_id: 0,
|
||||
},
|
||||
})
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -447,13 +447,20 @@ fn build_procedure_manager(
|
||||
retry_delay: options.procedure.retry_delay,
|
||||
..Default::default()
|
||||
};
|
||||
let state_store = KvStateStore::new(kv_backend.clone()).with_max_value_size(
|
||||
options
|
||||
.procedure
|
||||
.max_metadata_value_size
|
||||
.map(|v| v.as_bytes() as usize),
|
||||
let kv_state_store = Arc::new(
|
||||
KvStateStore::new(kv_backend.clone()).with_max_value_size(
|
||||
options
|
||||
.procedure
|
||||
.max_metadata_value_size
|
||||
.map(|v| v.as_bytes() as usize),
|
||||
),
|
||||
);
|
||||
Arc::new(LocalManager::new(manager_config, Arc::new(state_store)))
|
||||
|
||||
Arc::new(LocalManager::new(
|
||||
manager_config,
|
||||
kv_state_store.clone(),
|
||||
kv_state_store,
|
||||
))
|
||||
}
|
||||
|
||||
impl Default for MetasrvBuilder {
|
||||
|
||||
@@ -55,7 +55,11 @@ impl TestingEnv {
|
||||
let mailbox_ctx = MailboxContext::new(mailbox_sequence);
|
||||
|
||||
let state_store = Arc::new(KvStateStore::new(kv_backend));
|
||||
let procedure_manager = Arc::new(LocalManager::new(ManagerConfig::default(), state_store));
|
||||
let procedure_manager = Arc::new(LocalManager::new(
|
||||
ManagerConfig::default(),
|
||||
state_store.clone(),
|
||||
state_store,
|
||||
));
|
||||
|
||||
Self {
|
||||
table_metadata_manager,
|
||||
|
||||
@@ -31,6 +31,7 @@ use common_meta::sequence::SequenceBuilder;
|
||||
use common_meta::state_store::KvStateStore;
|
||||
use common_meta::DatanodeId;
|
||||
use common_procedure::local::{LocalManager, ManagerConfig};
|
||||
use common_procedure::test_util::InMemoryPoisonStore;
|
||||
use common_procedure::{Context as ProcedureContext, ProcedureId, ProcedureManagerRef, Status};
|
||||
use common_procedure_test::MockContextProvider;
|
||||
use common_telemetry::debug;
|
||||
@@ -85,7 +86,12 @@ impl TestingEnv {
|
||||
let opening_region_keeper = Arc::new(MemoryRegionKeeper::default());
|
||||
|
||||
let state_store = Arc::new(KvStateStore::new(kv_backend.clone()));
|
||||
let procedure_manager = Arc::new(LocalManager::new(ManagerConfig::default(), state_store));
|
||||
let poison_manager = Arc::new(InMemoryPoisonStore::default());
|
||||
let procedure_manager = Arc::new(LocalManager::new(
|
||||
ManagerConfig::default(),
|
||||
state_store,
|
||||
poison_manager,
|
||||
));
|
||||
|
||||
Self {
|
||||
table_metadata_manager,
|
||||
|
||||
@@ -224,7 +224,13 @@ async fn test_on_datanode_create_regions() {
|
||||
});
|
||||
|
||||
let status = procedure.on_datanode_create_regions().await.unwrap();
|
||||
assert!(matches!(status, Status::Executing { persist: true }));
|
||||
assert!(matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false,
|
||||
}
|
||||
));
|
||||
assert!(matches!(
|
||||
procedure.creator.data.state,
|
||||
CreateTableState::CreateMetadata
|
||||
@@ -291,7 +297,13 @@ async fn test_on_datanode_create_logical_regions() {
|
||||
|
||||
procedure.check_tables_already_exist().await.unwrap();
|
||||
let status = procedure.on_datanode_create_regions().await.unwrap();
|
||||
assert!(matches!(status, Status::Executing { persist: true }));
|
||||
assert!(matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false,
|
||||
}
|
||||
));
|
||||
assert!(matches!(
|
||||
procedure.data.state(),
|
||||
&CreateTablesState::CreateMetadata
|
||||
|
||||
440
src/meta-srv/src/procedure/wal_prune.rs
Normal file
440
src/meta-srv/src/procedure/wal_prune.rs
Normal file
@@ -0,0 +1,440 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::key::TableMetadataManagerRef;
|
||||
use common_meta::lock_key::RemoteWalLock;
|
||||
use common_meta::region_registry::LeaderRegionRegistryRef;
|
||||
use common_procedure::error::ToJsonSnafu;
|
||||
use common_procedure::{
|
||||
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
|
||||
Result as ProcedureResult, Status, StringKey,
|
||||
};
|
||||
use common_telemetry::warn;
|
||||
use log_store::kafka::DEFAULT_PARTITION;
|
||||
use rskafka::client::partition::UnknownTopicHandling;
|
||||
use rskafka::client::Client;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::ResultExt;
|
||||
use store_api::logstore::EntryId;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{self, BuildPartitionClientSnafu, DeleteRecordSnafu, TableMetadataManagerSnafu};
|
||||
use crate::Result;
|
||||
|
||||
type KafkaClientRef = Arc<Client>;
|
||||
|
||||
const TIMEOUT: i32 = 1000;
|
||||
|
||||
/// The state of WAL pruning.
|
||||
#[derive(Debug, Serialize, Deserialize)]
|
||||
pub enum WalPruneState {
|
||||
Prepare,
|
||||
Prune,
|
||||
}
|
||||
|
||||
pub struct Context {
|
||||
/// The Kafka client.
|
||||
client: KafkaClientRef,
|
||||
/// The table metadata manager.
|
||||
table_metadata_manager: TableMetadataManagerRef,
|
||||
leader_region_registry: LeaderRegionRegistryRef,
|
||||
}
|
||||
|
||||
/// The data of WAL pruning.
|
||||
#[derive(Serialize, Deserialize)]
|
||||
pub struct WalPruneData {
|
||||
/// The topic name to prune.
|
||||
pub topic: String,
|
||||
/// The minimum flush entry id for topic, which is used to prune the WAL.
|
||||
/// If the topic has no region, the value is set to `None`.
|
||||
pub min_flushed_entry_id: EntryId,
|
||||
/// The state.
|
||||
pub state: WalPruneState,
|
||||
}
|
||||
|
||||
/// The procedure to prune WAL.
|
||||
pub struct WalPruneProcedure {
|
||||
pub data: WalPruneData,
|
||||
pub context: Context,
|
||||
}
|
||||
|
||||
impl WalPruneProcedure {
|
||||
const TYPE_NAME: &'static str = "metasrv-procedure::WalPrune";
|
||||
|
||||
pub fn new(topic: String, context: Context) -> Self {
|
||||
Self {
|
||||
data: WalPruneData {
|
||||
topic,
|
||||
min_flushed_entry_id: 0,
|
||||
state: WalPruneState::Prepare,
|
||||
},
|
||||
context,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn from_json(json: &str, context: Context) -> ProcedureResult<Self> {
|
||||
let data: WalPruneData = serde_json::from_str(json).context(ToJsonSnafu)?;
|
||||
Ok(Self { data, context })
|
||||
}
|
||||
|
||||
/// Calculate the last entry id to prune for each topic.
|
||||
pub async fn on_prepare(&mut self) -> Result<Status> {
|
||||
let region_ids = self
|
||||
.context
|
||||
.table_metadata_manager
|
||||
.topic_region_manager()
|
||||
.regions(&self.data.topic)
|
||||
.await
|
||||
.context(TableMetadataManagerSnafu)
|
||||
.map_err(BoxedError::new)
|
||||
.with_context(|_| error::RetryLaterWithSourceSnafu {
|
||||
reason: "Failed to get topic-region map",
|
||||
})?;
|
||||
let flush_entry_ids_map: HashMap<_, _> = self
|
||||
.context
|
||||
.leader_region_registry
|
||||
.batch_get(region_ids.iter().cloned())
|
||||
.into_iter()
|
||||
.map(|(region_id, region)| {
|
||||
let flushed_entry_id = region.manifest.min_flushed_entry_id();
|
||||
(region_id, flushed_entry_id)
|
||||
})
|
||||
.collect();
|
||||
|
||||
if region_ids.is_empty() {
|
||||
// No regions to prune.
|
||||
return Ok(Status::done());
|
||||
}
|
||||
// Check if the `flush_entry_ids_map` contains all region ids.
|
||||
let non_collected_region_ids =
|
||||
check_heartbeat_collected_region_ids(®ion_ids, &flush_entry_ids_map);
|
||||
if !non_collected_region_ids.is_empty() {
|
||||
// The heartbeat collected region ids do not contain all region ids in the topic-region map.
|
||||
// In this case, we should not prune the WAL.
|
||||
warn!("The heartbeat collected region ids do not contain all region ids in the topic-region map. Aborting the WAL prune procedure.
|
||||
topic: {}, non-collected region ids: {:?}", self.data.topic, non_collected_region_ids);
|
||||
return Ok(Status::done());
|
||||
}
|
||||
|
||||
// Safety: `flush_entry_ids_map` are not empty.
|
||||
self.data.min_flushed_entry_id = *(flush_entry_ids_map.values().min().unwrap());
|
||||
self.data.state = WalPruneState::Prune;
|
||||
Ok(Status::executing(true))
|
||||
}
|
||||
|
||||
/// Prune the WAL.
|
||||
pub async fn on_prune(&mut self) -> Result<Status> {
|
||||
// Safety: last_entry_ids are loaded in on_prepare.
|
||||
let partition_client = self
|
||||
.context
|
||||
.client
|
||||
.partition_client(
|
||||
self.data.topic.clone(),
|
||||
DEFAULT_PARTITION,
|
||||
UnknownTopicHandling::Retry,
|
||||
)
|
||||
.await
|
||||
.context(BuildPartitionClientSnafu {
|
||||
topic: self.data.topic.clone(),
|
||||
partition: DEFAULT_PARTITION,
|
||||
})?;
|
||||
|
||||
partition_client
|
||||
.delete_records(self.data.min_flushed_entry_id as i64, TIMEOUT)
|
||||
.await
|
||||
.context(DeleteRecordSnafu {
|
||||
topic: self.data.topic.clone(),
|
||||
partition: DEFAULT_PARTITION,
|
||||
offset: self.data.min_flushed_entry_id,
|
||||
})
|
||||
.map_err(BoxedError::new)
|
||||
.with_context(|_| error::RetryLaterWithSourceSnafu {
|
||||
reason: "Failed to delete records",
|
||||
})?;
|
||||
|
||||
// TODO(CookiePie): Persist the minimum flushed entry id to the table metadata manager.
|
||||
Ok(Status::done())
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl Procedure for WalPruneProcedure {
|
||||
fn type_name(&self) -> &str {
|
||||
Self::TYPE_NAME
|
||||
}
|
||||
|
||||
fn rollback_supported(&self) -> bool {
|
||||
false
|
||||
}
|
||||
|
||||
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
|
||||
let state = &self.data.state;
|
||||
|
||||
match state {
|
||||
WalPruneState::Prepare => self.on_prepare().await,
|
||||
WalPruneState::Prune => self.on_prune().await,
|
||||
}
|
||||
.map_err(|e| {
|
||||
if e.is_retryable() {
|
||||
ProcedureError::retry_later(e)
|
||||
} else {
|
||||
ProcedureError::external(e)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
fn dump(&self) -> ProcedureResult<String> {
|
||||
serde_json::to_string(&self.data).context(ToJsonSnafu)
|
||||
}
|
||||
|
||||
/// WAL prune procedure will read the topic-region map from the table metadata manager,
|
||||
/// which are modified by `DROP [TABLE|DATABASE]` and `CREATE [TABLE]` operations.
|
||||
/// But the modifications are atomic, so it does not conflict with the procedure.
|
||||
/// It only abort the procedure sometimes since the `check_heartbeat_collected_region_ids` fails.
|
||||
fn lock_key(&self) -> LockKey {
|
||||
let lock_key: StringKey = RemoteWalLock::Write(self.data.topic.clone()).into();
|
||||
LockKey::new(vec![lock_key])
|
||||
}
|
||||
}
|
||||
|
||||
/// Check if the heartbeat collected region ids contains all region ids in the topic-region map.
|
||||
fn check_heartbeat_collected_region_ids(
|
||||
region_ids: &[RegionId],
|
||||
heartbeat_collected_region_ids: &HashMap<RegionId, u64>,
|
||||
) -> Vec<RegionId> {
|
||||
let mut non_collected_region_ids = Vec::new();
|
||||
for region_id in region_ids {
|
||||
if !heartbeat_collected_region_ids.contains_key(region_id) {
|
||||
non_collected_region_ids.push(*region_id);
|
||||
}
|
||||
}
|
||||
non_collected_region_ids
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::assert_matches::assert_matches;
|
||||
|
||||
use common_meta::key::TableMetadataManager;
|
||||
use common_meta::kv_backend::memory::MemoryKvBackend;
|
||||
use common_meta::region_registry::LeaderRegionRegistry;
|
||||
use common_meta::wal_options_allocator::build_kafka_topic_creator;
|
||||
use common_wal::config::kafka::common::{KafkaConnectionConfig, KafkaTopicConfig};
|
||||
use common_wal::config::kafka::MetasrvKafkaConfig;
|
||||
use common_wal::test_util::run_test_with_kafka_wal;
|
||||
use rskafka::record::Record;
|
||||
|
||||
use super::*;
|
||||
use crate::procedure::test_util::new_wal_prune_metadata;
|
||||
|
||||
struct TestEnv {
|
||||
table_metadata_manager: TableMetadataManagerRef,
|
||||
leader_region_registry: LeaderRegionRegistryRef,
|
||||
}
|
||||
|
||||
impl TestEnv {
|
||||
fn new() -> Self {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::new());
|
||||
let table_metadata_manager = Arc::new(TableMetadataManager::new(kv_backend.clone()));
|
||||
let leader_region_registry = Arc::new(LeaderRegionRegistry::new());
|
||||
Self {
|
||||
table_metadata_manager,
|
||||
leader_region_registry,
|
||||
}
|
||||
}
|
||||
|
||||
fn table_metadata_manager(&self) -> &TableMetadataManagerRef {
|
||||
&self.table_metadata_manager
|
||||
}
|
||||
|
||||
fn leader_region_registry(&self) -> &LeaderRegionRegistryRef {
|
||||
&self.leader_region_registry
|
||||
}
|
||||
}
|
||||
|
||||
/// Mock a test env for testing.
|
||||
/// Including:
|
||||
/// 1. Create a test env with a mailbox, a table metadata manager and a in-memory kv backend.
|
||||
/// 2. Prepare some data in the table metadata manager and in-memory kv backend.
|
||||
/// 3. Generate a `WalPruneProcedure` with the test env.
|
||||
/// 4. Return the test env, the procedure, the minimum last entry id to prune and the regions to flush.
|
||||
async fn mock_test_env(
|
||||
topic: String,
|
||||
broker_endpoints: Vec<String>,
|
||||
env: &TestEnv,
|
||||
) -> (WalPruneProcedure, u64, Vec<RegionId>) {
|
||||
// Creates a topic manager.
|
||||
let kafka_topic = KafkaTopicConfig {
|
||||
replication_factor: broker_endpoints.len() as i16,
|
||||
..Default::default()
|
||||
};
|
||||
let config = MetasrvKafkaConfig {
|
||||
connection: KafkaConnectionConfig {
|
||||
broker_endpoints,
|
||||
..Default::default()
|
||||
},
|
||||
kafka_topic,
|
||||
..Default::default()
|
||||
};
|
||||
let topic_creator = build_kafka_topic_creator(&config).await.unwrap();
|
||||
let table_metadata_manager = env.table_metadata_manager().clone();
|
||||
let leader_region_registry = env.leader_region_registry().clone();
|
||||
let offsets = mock_wal_entries(topic_creator.client().clone(), &topic, 10).await;
|
||||
|
||||
let (min_last_entry_id, regions_to_flush) = new_wal_prune_metadata(
|
||||
table_metadata_manager.clone(),
|
||||
leader_region_registry.clone(),
|
||||
10,
|
||||
5,
|
||||
&offsets,
|
||||
10,
|
||||
topic.clone(),
|
||||
)
|
||||
.await;
|
||||
|
||||
let context = Context {
|
||||
client: topic_creator.client().clone(),
|
||||
table_metadata_manager,
|
||||
leader_region_registry,
|
||||
};
|
||||
|
||||
let wal_prune_procedure = WalPruneProcedure::new(topic, context);
|
||||
(wal_prune_procedure, min_last_entry_id, regions_to_flush)
|
||||
}
|
||||
|
||||
fn record(i: usize) -> Record {
|
||||
let key = format!("key_{i}");
|
||||
let value = format!("value_{i}");
|
||||
Record {
|
||||
key: Some(key.into()),
|
||||
value: Some(value.into()),
|
||||
timestamp: chrono::Utc::now(),
|
||||
headers: Default::default(),
|
||||
}
|
||||
}
|
||||
|
||||
async fn mock_wal_entries(
|
||||
client: KafkaClientRef,
|
||||
topic_name: &str,
|
||||
n_entries: usize,
|
||||
) -> Vec<i64> {
|
||||
let controller_client = client.controller_client().unwrap();
|
||||
let _ = controller_client
|
||||
.create_topic(topic_name, 1, 1, 5_000)
|
||||
.await;
|
||||
let partition_client = client
|
||||
.partition_client(topic_name, 0, UnknownTopicHandling::Retry)
|
||||
.await
|
||||
.unwrap();
|
||||
let mut offsets = Vec::with_capacity(n_entries);
|
||||
for i in 0..n_entries {
|
||||
let record = vec![record(i)];
|
||||
let offset = partition_client
|
||||
.produce(
|
||||
record,
|
||||
rskafka::client::partition::Compression::NoCompression,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
offsets.extend(offset);
|
||||
}
|
||||
offsets
|
||||
}
|
||||
|
||||
async fn check_entry_id_existence(
|
||||
client: KafkaClientRef,
|
||||
topic_name: &str,
|
||||
entry_id: i64,
|
||||
) -> bool {
|
||||
let partition_client = client
|
||||
.partition_client(topic_name, 0, UnknownTopicHandling::Retry)
|
||||
.await
|
||||
.unwrap();
|
||||
let (records, _high_watermark) = partition_client
|
||||
.fetch_records(entry_id, 0..10001, 5_000)
|
||||
.await
|
||||
.unwrap();
|
||||
!records.is_empty()
|
||||
}
|
||||
|
||||
async fn delete_topic(client: KafkaClientRef, topic_name: &str) {
|
||||
let controller_client = client.controller_client().unwrap();
|
||||
controller_client
|
||||
.delete_topic(topic_name, 5_000)
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_procedure_execution() {
|
||||
run_test_with_kafka_wal(|broker_endpoints| {
|
||||
Box::pin(async {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let topic_name = "greptime_test_topic".to_string();
|
||||
let env = TestEnv::new();
|
||||
let (mut procedure, min_last_entry_id, _) =
|
||||
mock_test_env(topic_name.clone(), broker_endpoints, &env).await;
|
||||
|
||||
// Step 1: Test `on_prepare`.
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(
|
||||
status,
|
||||
Status::Executing {
|
||||
persist: true,
|
||||
clean_poisons: false
|
||||
}
|
||||
);
|
||||
assert_matches!(procedure.data.state, WalPruneState::Prune);
|
||||
assert_eq!(procedure.data.min_flushed_entry_id, min_last_entry_id);
|
||||
|
||||
// Step 2: Test `on_prune`.
|
||||
let status = procedure.on_prune().await.unwrap();
|
||||
assert_matches!(status, Status::Done { output: None });
|
||||
// Check if the entry ids after `min_flushed_entry_id` still exist.
|
||||
assert!(
|
||||
check_entry_id_existence(
|
||||
procedure.context.client.clone(),
|
||||
&topic_name,
|
||||
procedure.data.min_flushed_entry_id as i64,
|
||||
)
|
||||
.await
|
||||
);
|
||||
// Check if the entry s before `min_flushed_entry_id` are deleted.
|
||||
assert!(
|
||||
procedure.data.min_flushed_entry_id == 0
|
||||
|| !check_entry_id_existence(
|
||||
procedure.context.client.clone(),
|
||||
&topic_name,
|
||||
procedure.data.min_flushed_entry_id as i64 - 1,
|
||||
)
|
||||
.await
|
||||
);
|
||||
|
||||
// `check_heartbeat_collected_region_ids` fails.
|
||||
// Should log a warning and return `Status::Done`.
|
||||
procedure.context.leader_region_registry.reset();
|
||||
let status = procedure.on_prepare().await.unwrap();
|
||||
assert_matches!(status, Status::Done { output: None });
|
||||
|
||||
// Clean up the topic.
|
||||
delete_topic(procedure.context.client, &topic_name).await;
|
||||
})
|
||||
})
|
||||
.await;
|
||||
}
|
||||
}
|
||||
@@ -103,7 +103,6 @@ impl DataRegion {
|
||||
.get_metadata(region_id)
|
||||
.await
|
||||
.context(MitoReadOperationSnafu)?;
|
||||
let version = region_metadata.schema_version;
|
||||
|
||||
// find the max column id
|
||||
let new_column_id_start = 1 + region_metadata
|
||||
@@ -166,7 +165,6 @@ impl DataRegion {
|
||||
debug!("Adding (Column id assigned) columns {new_columns:?} to region {region_id:?}");
|
||||
// assemble alter request
|
||||
let alter_request = RegionRequest::Alter(RegionAlterRequest {
|
||||
schema_version: version,
|
||||
kind: AlterKind::AddColumns {
|
||||
columns: new_columns,
|
||||
},
|
||||
|
||||
@@ -234,7 +234,6 @@ mod test {
|
||||
// alter physical region
|
||||
let physical_region_id = env.default_physical_region_id();
|
||||
let request = RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::AddColumns {
|
||||
columns: vec![AddColumn {
|
||||
column_metadata: ColumnMetadata {
|
||||
@@ -262,7 +261,6 @@ mod test {
|
||||
|
||||
// alter physical region's option should work
|
||||
let alter_region_option_request = RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::SetRegionOptions {
|
||||
options: vec![SetRegionOption::Ttl(Some(Duration::from_secs(500).into()))],
|
||||
},
|
||||
|
||||
@@ -177,7 +177,6 @@ pub fn alter_logical_region_add_tag_columns(
|
||||
});
|
||||
}
|
||||
RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::AddColumns {
|
||||
columns: new_columns,
|
||||
},
|
||||
|
||||
@@ -12,6 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::assert_matches::assert_matches;
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
@@ -19,7 +20,6 @@ use std::time::Duration;
|
||||
use api::v1::value::ValueData;
|
||||
use api::v1::{ColumnDataType, Row, Rows, SemanticType};
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_error::status_code::StatusCode;
|
||||
use common_recordbatch::RecordBatches;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::{ColumnSchema, FulltextAnalyzer, FulltextOptions};
|
||||
@@ -34,6 +34,7 @@ use store_api::storage::{RegionId, ScanRequest};
|
||||
use crate::config::MitoConfig;
|
||||
use crate::engine::listener::{AlterFlushListener, NotifyRegionChangeResultListener};
|
||||
use crate::engine::MitoEngine;
|
||||
use crate::error;
|
||||
use crate::test_util::{
|
||||
build_rows, build_rows_for_key, flush_region, put_rows, rows_schema, CreateRequestBuilder,
|
||||
TestEnv,
|
||||
@@ -51,7 +52,6 @@ async fn scan_check_after_alter(engine: &MitoEngine, region_id: RegionId, expect
|
||||
|
||||
fn add_tag1() -> RegionAlterRequest {
|
||||
RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::AddColumns {
|
||||
columns: vec![AddColumn {
|
||||
column_metadata: ColumnMetadata {
|
||||
@@ -71,7 +71,6 @@ fn add_tag1() -> RegionAlterRequest {
|
||||
|
||||
fn alter_column_inverted_index() -> RegionAlterRequest {
|
||||
RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Inverted {
|
||||
column_name: "tag_0".to_string(),
|
||||
@@ -82,7 +81,6 @@ fn alter_column_inverted_index() -> RegionAlterRequest {
|
||||
|
||||
fn alter_column_fulltext_options() -> RegionAlterRequest {
|
||||
RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::SetIndex {
|
||||
options: ApiSetIndexOptions::Fulltext {
|
||||
column_name: "tag_0".to_string(),
|
||||
@@ -357,7 +355,8 @@ async fn test_alter_region_retry() {
|
||||
.handle_request(region_id, RegionRequest::Alter(request))
|
||||
.await
|
||||
.unwrap_err();
|
||||
assert_eq!(err.status_code(), StatusCode::RequestOutdated);
|
||||
let err = err.as_any().downcast_ref::<error::Error>().unwrap();
|
||||
assert_matches!(err, &error::Error::InvalidRegionRequest { .. });
|
||||
|
||||
let expected = "\
|
||||
+-------+-------+---------+---------------------+
|
||||
@@ -731,7 +730,6 @@ async fn test_alter_region_ttl_options() {
|
||||
.unwrap();
|
||||
let engine_cloned = engine.clone();
|
||||
let alter_ttl_request = RegionAlterRequest {
|
||||
schema_version: 0,
|
||||
kind: AlterKind::SetRegionOptions {
|
||||
options: vec![SetRegionOption::Ttl(Some(Duration::from_secs(500).into()))],
|
||||
},
|
||||
|
||||
@@ -535,7 +535,6 @@ async fn test_change_region_compaction_window() {
|
||||
|
||||
// Change compaction window.
|
||||
let request = RegionRequest::Alter(RegionAlterRequest {
|
||||
schema_version: region.metadata().schema_version,
|
||||
kind: SetRegionOptions {
|
||||
options: vec![SetRegionOption::Twsc(
|
||||
"compaction.twcs.time_window".to_string(),
|
||||
|
||||
246
src/mito2/src/engine/sync_test.rs
Normal file
246
src/mito2/src/engine/sync_test.rs
Normal file
@@ -0,0 +1,246 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::assert_matches::assert_matches;
|
||||
|
||||
use api::v1::{Rows, SemanticType};
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_recordbatch::RecordBatches;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
use store_api::metadata::ColumnMetadata;
|
||||
use store_api::region_engine::{RegionEngine, RegionManifestInfo};
|
||||
use store_api::region_request::{
|
||||
AddColumn, AddColumnLocation, AlterKind, RegionAlterRequest, RegionOpenRequest, RegionRequest,
|
||||
};
|
||||
use store_api::storage::{RegionId, ScanRequest};
|
||||
|
||||
use super::MitoEngine;
|
||||
use crate::config::MitoConfig;
|
||||
use crate::error::Error;
|
||||
use crate::test_util::{
|
||||
build_rows, flush_region, put_rows, rows_schema, CreateRequestBuilder, TestEnv,
|
||||
};
|
||||
|
||||
fn add_tag1() -> RegionAlterRequest {
|
||||
RegionAlterRequest {
|
||||
kind: AlterKind::AddColumns {
|
||||
columns: vec![AddColumn {
|
||||
column_metadata: ColumnMetadata {
|
||||
column_schema: ColumnSchema::new(
|
||||
"tag_1",
|
||||
ConcreteDataType::string_datatype(),
|
||||
true,
|
||||
),
|
||||
semantic_type: SemanticType::Tag,
|
||||
column_id: 3,
|
||||
},
|
||||
location: Some(AddColumnLocation::First),
|
||||
}],
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
async fn scan_check(
|
||||
engine: &MitoEngine,
|
||||
region_id: RegionId,
|
||||
expected: &str,
|
||||
num_memtable: usize,
|
||||
num_files: usize,
|
||||
) {
|
||||
let request = ScanRequest::default();
|
||||
let scanner = engine.scanner(region_id, request).unwrap();
|
||||
assert_eq!(num_memtable, scanner.num_memtables());
|
||||
assert_eq!(num_files, scanner.num_files());
|
||||
let stream = scanner.scan().await.unwrap();
|
||||
let batches = RecordBatches::try_collect(stream).await.unwrap();
|
||||
assert_eq!(expected, batches.pretty_print().unwrap());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_sync_after_flush_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let mut env = TestEnv::new();
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
let region_id = RegionId::new(1, 1);
|
||||
env.get_schema_metadata_manager()
|
||||
.register_region_table_info(
|
||||
region_id.table_id(),
|
||||
"test_table",
|
||||
"test_catalog",
|
||||
"test_schema",
|
||||
None,
|
||||
env.get_kv_backend(),
|
||||
)
|
||||
.await;
|
||||
|
||||
let request = CreateRequestBuilder::new().build();
|
||||
let region_dir = request.region_dir.clone();
|
||||
let column_schemas = rows_schema(&request);
|
||||
engine
|
||||
.handle_request(region_id, RegionRequest::Create(request))
|
||||
.await
|
||||
.unwrap();
|
||||
let rows = Rows {
|
||||
schema: column_schemas,
|
||||
rows: build_rows(0, 3),
|
||||
};
|
||||
put_rows(&engine, region_id, rows).await;
|
||||
|
||||
// Open the region on the follower engine
|
||||
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
|
||||
follower_engine
|
||||
.handle_request(
|
||||
region_id,
|
||||
RegionRequest::Open(RegionOpenRequest {
|
||||
engine: String::new(),
|
||||
region_dir,
|
||||
options: Default::default(),
|
||||
// Ensure the region is not replayed from the WAL.
|
||||
skip_wal_replay: true,
|
||||
}),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
flush_region(&engine, region_id, None).await;
|
||||
|
||||
// Scan the region on the leader engine
|
||||
let expected = "\
|
||||
+-------+---------+---------------------+
|
||||
| tag_0 | field_0 | ts |
|
||||
+-------+---------+---------------------+
|
||||
| 0 | 0.0 | 1970-01-01T00:00:00 |
|
||||
| 1 | 1.0 | 1970-01-01T00:00:01 |
|
||||
| 2 | 2.0 | 1970-01-01T00:00:02 |
|
||||
+-------+---------+---------------------+";
|
||||
scan_check(&engine, region_id, expected, 0, 1).await;
|
||||
|
||||
common_telemetry::info!("Scan the region on the follower engine");
|
||||
// Scan the region on the follower engine
|
||||
let expected = "++\n++";
|
||||
scan_check(&follower_engine, region_id, expected, 0, 0).await;
|
||||
|
||||
// Returns error since the max manifest is 1
|
||||
let manifest_info = RegionManifestInfo::mito(2, 0);
|
||||
let err = follower_engine
|
||||
.sync_region(region_id, manifest_info)
|
||||
.await
|
||||
.unwrap_err();
|
||||
let err = err.as_any().downcast_ref::<Error>().unwrap();
|
||||
assert_matches!(err, Error::InstallManifestTo { .. });
|
||||
|
||||
let manifest_info = RegionManifestInfo::mito(1, 0);
|
||||
follower_engine
|
||||
.sync_region(region_id, manifest_info)
|
||||
.await
|
||||
.unwrap();
|
||||
common_telemetry::info!("Scan the region on the follower engine after sync");
|
||||
// Scan the region on the follower engine
|
||||
let expected = "\
|
||||
+-------+---------+---------------------+
|
||||
| tag_0 | field_0 | ts |
|
||||
+-------+---------+---------------------+
|
||||
| 0 | 0.0 | 1970-01-01T00:00:00 |
|
||||
| 1 | 1.0 | 1970-01-01T00:00:01 |
|
||||
| 2 | 2.0 | 1970-01-01T00:00:02 |
|
||||
+-------+---------+---------------------+";
|
||||
scan_check(&follower_engine, region_id, expected, 0, 1).await;
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_sync_after_alter_region() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
let mut env = TestEnv::new();
|
||||
let engine = env.create_engine(MitoConfig::default()).await;
|
||||
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let request = CreateRequestBuilder::new().build();
|
||||
|
||||
env.get_schema_metadata_manager()
|
||||
.register_region_table_info(
|
||||
region_id.table_id(),
|
||||
"test_table",
|
||||
"test_catalog",
|
||||
"test_schema",
|
||||
None,
|
||||
env.get_kv_backend(),
|
||||
)
|
||||
.await;
|
||||
|
||||
let column_schemas = rows_schema(&request);
|
||||
let region_dir = request.region_dir.clone();
|
||||
engine
|
||||
.handle_request(region_id, RegionRequest::Create(request))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let rows = Rows {
|
||||
schema: column_schemas,
|
||||
rows: build_rows(0, 3),
|
||||
};
|
||||
put_rows(&engine, region_id, rows).await;
|
||||
|
||||
// Open the region on the follower engine
|
||||
let follower_engine = env.create_follower_engine(MitoConfig::default()).await;
|
||||
follower_engine
|
||||
.handle_request(
|
||||
region_id,
|
||||
RegionRequest::Open(RegionOpenRequest {
|
||||
engine: String::new(),
|
||||
region_dir,
|
||||
options: Default::default(),
|
||||
// Ensure the region is not replayed from the WAL.
|
||||
skip_wal_replay: true,
|
||||
}),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let request = add_tag1();
|
||||
engine
|
||||
.handle_request(region_id, RegionRequest::Alter(request))
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let expected = "\
|
||||
+-------+-------+---------+---------------------+
|
||||
| tag_1 | tag_0 | field_0 | ts |
|
||||
+-------+-------+---------+---------------------+
|
||||
| | 0 | 0.0 | 1970-01-01T00:00:00 |
|
||||
| | 1 | 1.0 | 1970-01-01T00:00:01 |
|
||||
| | 2 | 2.0 | 1970-01-01T00:00:02 |
|
||||
+-------+-------+---------+---------------------+";
|
||||
|
||||
scan_check(&engine, region_id, expected, 0, 1).await;
|
||||
let expected = "++\n++";
|
||||
scan_check(&follower_engine, region_id, expected, 0, 0).await;
|
||||
|
||||
// Sync the region from the leader engine to the follower engine
|
||||
let manifest_info = RegionManifestInfo::mito(2, 0);
|
||||
follower_engine
|
||||
.sync_region(region_id, manifest_info)
|
||||
.await
|
||||
.unwrap();
|
||||
let expected = "\
|
||||
+-------+-------+---------+---------------------+
|
||||
| tag_1 | tag_0 | field_0 | ts |
|
||||
+-------+-------+---------+---------------------+
|
||||
| | 0 | 0.0 | 1970-01-01T00:00:00 |
|
||||
| | 1 | 1.0 | 1970-01-01T00:00:01 |
|
||||
| | 2 | 2.0 | 1970-01-01T00:00:02 |
|
||||
+-------+-------+---------+---------------------+";
|
||||
scan_check(&follower_engine, region_id, expected, 0, 1).await;
|
||||
}
|
||||
@@ -482,14 +482,6 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Schema version doesn't match. Expect {} but gives {}", expect, actual))]
|
||||
InvalidRegionRequestSchemaVersion {
|
||||
expect: u64,
|
||||
actual: u64,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Region {} is in {:?} state, which does not permit manifest updates.",
|
||||
region_id,
|
||||
@@ -1026,8 +1018,6 @@ impl ErrorExt for Error {
|
||||
| PartitionOutOfRange { .. }
|
||||
| ParseJobId { .. } => StatusCode::InvalidArguments,
|
||||
|
||||
InvalidRegionRequestSchemaVersion { .. } => StatusCode::RequestOutdated,
|
||||
|
||||
RegionMetadataNotFound { .. }
|
||||
| Join { .. }
|
||||
| WorkerStopped { .. }
|
||||
|
||||
@@ -18,7 +18,8 @@ use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_base::readable_size::ReadableSize;
|
||||
use common_telemetry::{debug, info};
|
||||
use common_telemetry::info;
|
||||
use common_telemetry::tracing::warn;
|
||||
use humantime_serde::re::humantime;
|
||||
use snafu::ResultExt;
|
||||
use store_api::metadata::{
|
||||
@@ -29,9 +30,7 @@ use store_api::mito_engine_options;
|
||||
use store_api::region_request::{AlterKind, RegionAlterRequest, SetRegionOption};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{
|
||||
InvalidMetadataSnafu, InvalidRegionRequestSchemaVersionSnafu, InvalidRegionRequestSnafu, Result,
|
||||
};
|
||||
use crate::error::{InvalidMetadataSnafu, InvalidRegionRequestSnafu, Result};
|
||||
use crate::flush::FlushReason;
|
||||
use crate::manifest::action::RegionChange;
|
||||
use crate::region::options::CompactionOptions::Twcs;
|
||||
@@ -83,22 +82,6 @@ impl<S> RegionWorkerLoop<S> {
|
||||
_ => {}
|
||||
}
|
||||
|
||||
if version.metadata.schema_version != request.schema_version {
|
||||
// This is possible if we retry the request.
|
||||
debug!(
|
||||
"Ignores alter request, region id:{}, region schema version {} is not equal to request schema version {}",
|
||||
region_id, version.metadata.schema_version, request.schema_version
|
||||
);
|
||||
// Returns an error.
|
||||
sender.send(
|
||||
InvalidRegionRequestSchemaVersionSnafu {
|
||||
expect: version.metadata.schema_version,
|
||||
actual: request.schema_version,
|
||||
}
|
||||
.fail(),
|
||||
);
|
||||
return;
|
||||
}
|
||||
// Validate request.
|
||||
if let Err(e) = request.validate(&version.metadata) {
|
||||
// Invalid request.
|
||||
@@ -108,7 +91,7 @@ impl<S> RegionWorkerLoop<S> {
|
||||
|
||||
// Checks whether we need to alter the region.
|
||||
if !request.need_alter(&version.metadata) {
|
||||
debug!(
|
||||
warn!(
|
||||
"Ignores alter request as it alters nothing, region_id: {}, request: {:?}",
|
||||
region_id, request
|
||||
);
|
||||
@@ -219,7 +202,6 @@ fn metadata_after_alteration(
|
||||
.context(InvalidRegionRequestSnafu)?
|
||||
.bump_version();
|
||||
let new_meta = builder.build().context(InvalidMetadataSnafu)?;
|
||||
assert_eq!(request.schema_version + 1, new_meta.schema_version);
|
||||
|
||||
Ok(Arc::new(new_meta))
|
||||
}
|
||||
|
||||
@@ -21,8 +21,8 @@ use api::v1::column_def::options_from_column_schema;
|
||||
use api::v1::{
|
||||
set_index, unset_index, AddColumn, AddColumns, AlterDatabaseExpr, AlterTableExpr, Analyzer,
|
||||
ColumnDataType, ColumnDataTypeExtension, CreateFlowExpr, CreateTableExpr, CreateViewExpr,
|
||||
DropColumn, DropColumns, ExpireAfter, ModifyColumnType, ModifyColumnTypes, RenameTable,
|
||||
SemanticType, SetDatabaseOptions, SetFulltext, SetIndex, SetInverted, SetSkipping,
|
||||
DropColumn, DropColumns, ExpireAfter, FulltextBackend, ModifyColumnType, ModifyColumnTypes,
|
||||
RenameTable, SemanticType, SetDatabaseOptions, SetFulltext, SetIndex, SetInverted, SetSkipping,
|
||||
SetTableOptions, SkippingIndexType as PbSkippingIndexType, TableName, UnsetDatabaseOptions,
|
||||
UnsetFulltext, UnsetIndex, UnsetInverted, UnsetSkipping, UnsetTableOptions,
|
||||
};
|
||||
@@ -581,6 +581,7 @@ pub(crate) fn to_alter_table_expr(
|
||||
FulltextAnalyzer::Chinese => Analyzer::Chinese.into(),
|
||||
},
|
||||
case_sensitive: options.case_sensitive,
|
||||
backend: FulltextBackend::Tantivy.into(),
|
||||
})),
|
||||
},
|
||||
sql::statements::alter::SetIndexOperation::Inverted { column_name } => SetIndex {
|
||||
|
||||
@@ -193,8 +193,8 @@ pub enum Error {
|
||||
impl ErrorExt for Error {
|
||||
fn status_code(&self) -> StatusCode {
|
||||
match self {
|
||||
Error::GetCache { .. } | Error::FindLeader { .. } => StatusCode::StorageUnavailable,
|
||||
Error::FindRegionRoutes { .. } => StatusCode::RegionNotReady,
|
||||
Error::GetCache { .. } => StatusCode::StorageUnavailable,
|
||||
Error::FindLeader { .. } => StatusCode::TableUnavailable,
|
||||
|
||||
Error::ConjunctExprWithNonExpr { .. }
|
||||
| Error::UnclosedValue { .. }
|
||||
@@ -211,9 +211,10 @@ impl ErrorExt for Error {
|
||||
| Error::SerializeJson { .. }
|
||||
| Error::DeserializeJson { .. } => StatusCode::Internal,
|
||||
|
||||
Error::Unexpected { .. } => StatusCode::Unexpected,
|
||||
Error::InvalidTableRouteData { .. } => StatusCode::TableUnavailable,
|
||||
Error::FindTableRoutes { .. } => StatusCode::TableUnavailable,
|
||||
Error::Unexpected { .. }
|
||||
| Error::InvalidTableRouteData { .. }
|
||||
| Error::FindTableRoutes { .. }
|
||||
| Error::FindRegionRoutes { .. } => StatusCode::Unexpected,
|
||||
Error::TableRouteNotFound { .. } => StatusCode::TableNotFound,
|
||||
Error::TableRouteManager { source, .. } => source.status_code(),
|
||||
Error::UnexpectedLogicalRouteTable { source, .. } => source.status_code(),
|
||||
|
||||
@@ -32,8 +32,8 @@ use datafusion_expr::execution_props::ExecutionProps;
|
||||
use datafusion_expr::expr::WildcardOptions;
|
||||
use datafusion_expr::simplify::SimplifyContext;
|
||||
use datafusion_expr::{
|
||||
Aggregate, Analyze, Explain, Expr, ExprSchemable, Extension, LogicalPlan, LogicalPlanBuilder,
|
||||
Projection,
|
||||
Aggregate, Analyze, Cast, Explain, Expr, ExprSchemable, Extension, LogicalPlan,
|
||||
LogicalPlanBuilder, Projection,
|
||||
};
|
||||
use datafusion_optimizer::simplify_expressions::ExprSimplifier;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
@@ -548,12 +548,29 @@ fn have_range_in_exprs(exprs: &[Expr]) -> bool {
|
||||
fn interval_only_in_expr(expr: &Expr) -> bool {
|
||||
let mut all_interval = true;
|
||||
let _ = expr.apply(|expr| {
|
||||
// A cast expression for an interval.
|
||||
if matches!(
|
||||
expr,
|
||||
Expr::Cast(Cast{
|
||||
expr,
|
||||
data_type: DataType::Interval(_)
|
||||
}) if matches!(&**expr, Expr::Literal(ScalarValue::Utf8(_)))
|
||||
) {
|
||||
// Stop checking the sub `expr`,
|
||||
// which is a `Utf8` type and has already been tested above.
|
||||
return Ok(TreeNodeRecursion::Stop);
|
||||
}
|
||||
|
||||
if !matches!(
|
||||
expr,
|
||||
Expr::Literal(ScalarValue::IntervalDayTime(_))
|
||||
| Expr::Literal(ScalarValue::IntervalMonthDayNano(_))
|
||||
| Expr::Literal(ScalarValue::IntervalYearMonth(_))
|
||||
| Expr::BinaryExpr(_)
|
||||
| Expr::Cast(Cast {
|
||||
data_type: DataType::Interval(_),
|
||||
..
|
||||
})
|
||||
) {
|
||||
all_interval = false;
|
||||
Ok(TreeNodeRecursion::Stop)
|
||||
@@ -561,6 +578,7 @@ fn interval_only_in_expr(expr: &Expr) -> bool {
|
||||
Ok(TreeNodeRecursion::Continue)
|
||||
}
|
||||
});
|
||||
|
||||
all_interval
|
||||
}
|
||||
|
||||
@@ -569,6 +587,7 @@ mod test {
|
||||
|
||||
use std::error::Error;
|
||||
|
||||
use arrow::datatypes::IntervalUnit;
|
||||
use catalog::memory::MemoryCatalogManager;
|
||||
use catalog::RegisterTableRequest;
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
@@ -844,6 +863,15 @@ mod test {
|
||||
parse_duration_expr(&args, 0).unwrap(),
|
||||
parse_duration("1y4w").unwrap()
|
||||
);
|
||||
// test cast expression
|
||||
let args = vec![Expr::Cast(Cast {
|
||||
expr: Box::new(Expr::Literal(ScalarValue::Utf8(Some("15 minutes".into())))),
|
||||
data_type: DataType::Interval(IntervalUnit::MonthDayNano),
|
||||
})];
|
||||
assert_eq!(
|
||||
parse_duration_expr(&args, 0).unwrap(),
|
||||
parse_duration("15m").unwrap()
|
||||
);
|
||||
// test index err
|
||||
assert!(parse_duration_expr(&args, 10).is_err());
|
||||
// test evaluate expr
|
||||
@@ -958,5 +986,37 @@ mod test {
|
||||
)))),
|
||||
});
|
||||
assert!(interval_only_in_expr(&expr));
|
||||
|
||||
let expr = Expr::BinaryExpr(BinaryExpr {
|
||||
left: Box::new(Expr::Cast(Cast {
|
||||
expr: Box::new(Expr::Literal(ScalarValue::Utf8(Some(
|
||||
"15 minute".to_string(),
|
||||
)))),
|
||||
data_type: DataType::Interval(IntervalUnit::MonthDayNano),
|
||||
})),
|
||||
op: Operator::Minus,
|
||||
right: Box::new(Expr::Literal(ScalarValue::IntervalDayTime(Some(
|
||||
IntervalDayTime::new(10, 0).into(),
|
||||
)))),
|
||||
});
|
||||
assert!(interval_only_in_expr(&expr));
|
||||
|
||||
let expr = Expr::Cast(Cast {
|
||||
expr: Box::new(Expr::BinaryExpr(BinaryExpr {
|
||||
left: Box::new(Expr::Cast(Cast {
|
||||
expr: Box::new(Expr::Literal(ScalarValue::Utf8(Some(
|
||||
"15 minute".to_string(),
|
||||
)))),
|
||||
data_type: DataType::Interval(IntervalUnit::MonthDayNano),
|
||||
})),
|
||||
op: Operator::Minus,
|
||||
right: Box::new(Expr::Literal(ScalarValue::IntervalDayTime(Some(
|
||||
IntervalDayTime::new(10, 0).into(),
|
||||
)))),
|
||||
})),
|
||||
data_type: DataType::Interval(IntervalUnit::MonthDayNano),
|
||||
});
|
||||
|
||||
assert!(interval_only_in_expr(&expr));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -12,12 +12,14 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::io::BufRead;
|
||||
use std::str::FromStr;
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
use api::v1::RowInsertRequests;
|
||||
use async_trait::async_trait;
|
||||
use axum::body::Bytes;
|
||||
use axum::extract::{FromRequest, Multipart, Path, Query, Request, State};
|
||||
use axum::http::header::CONTENT_TYPE;
|
||||
use axum::http::{HeaderMap, StatusCode};
|
||||
@@ -389,8 +391,8 @@ pub struct PipelineDryrunParams {
|
||||
/// Check if the payload is valid json
|
||||
/// Check if the payload contains pipeline or pipeline_name and data
|
||||
/// Return Some if valid, None if invalid
|
||||
fn check_pipeline_dryrun_params_valid(payload: &str) -> Option<PipelineDryrunParams> {
|
||||
match serde_json::from_str::<PipelineDryrunParams>(payload) {
|
||||
fn check_pipeline_dryrun_params_valid(payload: &Bytes) -> Option<PipelineDryrunParams> {
|
||||
match serde_json::from_slice::<PipelineDryrunParams>(payload) {
|
||||
// payload with pipeline or pipeline_name and data is array
|
||||
Ok(params) if params.pipeline.is_some() || params.pipeline_name.is_some() => Some(params),
|
||||
// because of the pipeline_name or pipeline is required
|
||||
@@ -432,7 +434,7 @@ pub async fn pipeline_dryrun(
|
||||
Query(query_params): Query<LogIngesterQueryParams>,
|
||||
Extension(mut query_ctx): Extension<QueryContext>,
|
||||
TypedHeader(content_type): TypedHeader<ContentType>,
|
||||
payload: String,
|
||||
payload: Bytes,
|
||||
) -> Result<Response> {
|
||||
let handler = log_state.log_handler;
|
||||
|
||||
@@ -514,7 +516,7 @@ pub async fn log_ingester(
|
||||
Extension(mut query_ctx): Extension<QueryContext>,
|
||||
TypedHeader(content_type): TypedHeader<ContentType>,
|
||||
headers: HeaderMap,
|
||||
payload: String,
|
||||
payload: Bytes,
|
||||
) -> Result<HttpResponse> {
|
||||
// validate source and payload
|
||||
let source = query_params.source.as_deref();
|
||||
@@ -565,40 +567,45 @@ pub async fn log_ingester(
|
||||
|
||||
fn extract_pipeline_value_by_content_type(
|
||||
content_type: ContentType,
|
||||
payload: String,
|
||||
payload: Bytes,
|
||||
ignore_errors: bool,
|
||||
) -> Result<Vec<Value>> {
|
||||
Ok(match content_type {
|
||||
ct if ct == *JSON_CONTENT_TYPE => transform_ndjson_array_factory(
|
||||
Deserializer::from_str(&payload).into_iter(),
|
||||
Deserializer::from_slice(&payload).into_iter(),
|
||||
ignore_errors,
|
||||
)?,
|
||||
ct if ct == *NDJSON_CONTENT_TYPE => {
|
||||
let mut result = Vec::with_capacity(1000);
|
||||
for (index, line) in payload.lines().enumerate() {
|
||||
match serde_json::from_str(line) {
|
||||
Ok(v) => {
|
||||
result.push(v);
|
||||
}
|
||||
Err(_) => {
|
||||
if !ignore_errors {
|
||||
warn!(
|
||||
"invalid json item in array, index: {:?}, value: {:?}",
|
||||
index, line
|
||||
);
|
||||
return InvalidParameterSnafu {
|
||||
reason: format!("invalid item:{} in array", line),
|
||||
}
|
||||
.fail();
|
||||
let line = match line {
|
||||
Ok(line) if !line.is_empty() => line,
|
||||
Ok(_) => continue, // Skip empty lines
|
||||
Err(_) if ignore_errors => continue,
|
||||
Err(e) => {
|
||||
warn!(e; "invalid string at index: {}", index);
|
||||
return InvalidParameterSnafu {
|
||||
reason: format!("invalid line at index: {}", index),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
};
|
||||
|
||||
if let Ok(v) = serde_json::from_str(&line) {
|
||||
result.push(v);
|
||||
} else if !ignore_errors {
|
||||
warn!("invalid JSON at index: {}, content: {:?}", index, line);
|
||||
return InvalidParameterSnafu {
|
||||
reason: format!("invalid JSON at index: {}", index),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
}
|
||||
result
|
||||
}
|
||||
ct if ct == *TEXT_CONTENT_TYPE || ct == *TEXT_UTF8_CONTENT_TYPE => payload
|
||||
.lines()
|
||||
.filter(|line| !line.is_empty())
|
||||
.filter_map(|line| line.ok().filter(|line| !line.is_empty()))
|
||||
.map(|line| json!({"message": line}))
|
||||
.collect(),
|
||||
_ => UnsupportedContentTypeSnafu { content_type }.fail()?,
|
||||
@@ -677,7 +684,8 @@ pub(crate) async fn ingest_logs_inner(
|
||||
pub trait LogValidator: Send + Sync {
|
||||
/// validate payload by source before processing
|
||||
/// Return a `Some` result to indicate validation failure.
|
||||
async fn validate(&self, source: Option<&str>, payload: &str) -> Option<Result<HttpResponse>>;
|
||||
async fn validate(&self, source: Option<&str>, payload: &Bytes)
|
||||
-> Option<Result<HttpResponse>>;
|
||||
}
|
||||
|
||||
pub type LogValidatorRef = Arc<dyn LogValidator + 'static>;
|
||||
@@ -731,17 +739,17 @@ mod tests {
|
||||
{"a": 1}
|
||||
{"b": 2"}
|
||||
{"c": 1}
|
||||
"#;
|
||||
"#
|
||||
.as_bytes();
|
||||
let payload = Bytes::from_static(payload);
|
||||
|
||||
let fail_rest =
|
||||
extract_pipeline_value_by_content_type(ContentType::json(), payload.to_string(), true);
|
||||
extract_pipeline_value_by_content_type(ContentType::json(), payload.clone(), true);
|
||||
assert!(fail_rest.is_ok());
|
||||
assert_eq!(fail_rest.unwrap(), vec![json!({"a": 1})]);
|
||||
|
||||
let fail_only_wrong = extract_pipeline_value_by_content_type(
|
||||
NDJSON_CONTENT_TYPE.clone(),
|
||||
payload.to_string(),
|
||||
true,
|
||||
);
|
||||
let fail_only_wrong =
|
||||
extract_pipeline_value_by_content_type(NDJSON_CONTENT_TYPE.clone(), payload, true);
|
||||
assert!(fail_only_wrong.is_ok());
|
||||
assert_eq!(
|
||||
fail_only_wrong.unwrap(),
|
||||
|
||||
@@ -32,6 +32,7 @@ use datatypes::scalars::ScalarVector;
|
||||
use datatypes::vectors::Float64Vector;
|
||||
use futures::future::join_all;
|
||||
use futures::StreamExt;
|
||||
use itertools::Itertools;
|
||||
use promql_parser::label::{MatchOp, Matcher, Matchers, METRIC_NAME};
|
||||
use promql_parser::parser::value::ValueType;
|
||||
use promql_parser::parser::{
|
||||
@@ -1003,18 +1004,19 @@ pub async fn label_values_query(
|
||||
|
||||
for query in queries {
|
||||
let promql_expr = try_call_return_response!(promql_parser::parser::parse(&query));
|
||||
let PromqlExpr::VectorSelector(VectorSelector { name, matchers, .. }) = promql_expr else {
|
||||
let PromqlExpr::VectorSelector(mut vector_selector) = promql_expr else {
|
||||
return PrometheusJsonResponse::error(
|
||||
StatusCode::InvalidArguments,
|
||||
"expected vector selector",
|
||||
);
|
||||
};
|
||||
let Some(name) = name else {
|
||||
let Some(name) = take_metric_name(&mut vector_selector) else {
|
||||
return PrometheusJsonResponse::error(
|
||||
StatusCode::InvalidArguments,
|
||||
"expected metric name",
|
||||
);
|
||||
};
|
||||
let VectorSelector { matchers, .. } = vector_selector;
|
||||
// Only use and filter matchers.
|
||||
let matchers = matchers.matchers;
|
||||
let result = handler
|
||||
@@ -1048,6 +1050,25 @@ pub async fn label_values_query(
|
||||
PrometheusJsonResponse::success(PrometheusResponse::LabelValues(label_values))
|
||||
}
|
||||
|
||||
/// Take metric name from the [VectorSelector].
|
||||
/// It takes the name in the selector or removes the name matcher.
|
||||
fn take_metric_name(selector: &mut VectorSelector) -> Option<String> {
|
||||
if let Some(name) = selector.name.take() {
|
||||
return Some(name);
|
||||
}
|
||||
|
||||
let (pos, matcher) = selector
|
||||
.matchers
|
||||
.matchers
|
||||
.iter()
|
||||
.find_position(|matcher| matcher.name == "__name__" && matcher.op == MatchOp::Equal)?;
|
||||
let name = matcher.value.clone();
|
||||
// We need to remove the name matcher to avoid using it as a filter in query.
|
||||
selector.matchers.matchers.remove(pos);
|
||||
|
||||
Some(name)
|
||||
}
|
||||
|
||||
async fn retrieve_field_names(
|
||||
query_ctx: &QueryContext,
|
||||
manager: CatalogManagerRef,
|
||||
|
||||
@@ -47,7 +47,7 @@ use tokio::io::AsyncWrite;
|
||||
use crate::error::{self, DataFrameSnafu, InvalidPrepareStatementSnafu, Result};
|
||||
use crate::metrics::METRIC_AUTH_FAILURE;
|
||||
use crate::mysql::helper::{
|
||||
self, format_placeholder, replace_placeholders, transform_placeholders,
|
||||
self, fix_placeholder_types, format_placeholder, replace_placeholders, transform_placeholders,
|
||||
};
|
||||
use crate::mysql::writer;
|
||||
use crate::mysql::writer::{create_mysql_column, handle_err};
|
||||
@@ -183,7 +183,7 @@ impl MysqlInstanceShim {
|
||||
let describe_result = self
|
||||
.do_describe(statement.clone(), query_ctx.clone())
|
||||
.await?;
|
||||
let (plan, schema) = if let Some(DescribeResult {
|
||||
let (mut plan, schema) = if let Some(DescribeResult {
|
||||
logical_plan,
|
||||
schema,
|
||||
}) = describe_result
|
||||
@@ -193,7 +193,9 @@ impl MysqlInstanceShim {
|
||||
(None, None)
|
||||
};
|
||||
|
||||
let params = if let Some(plan) = &plan {
|
||||
let params = if let Some(plan) = &mut plan {
|
||||
fix_placeholder_types(plan)?;
|
||||
debug!("Plan after fix placeholder types: {:#?}", plan);
|
||||
prepared_params(
|
||||
&plan
|
||||
.get_parameter_types()
|
||||
@@ -258,7 +260,8 @@ impl MysqlInstanceShim {
|
||||
};
|
||||
|
||||
let outputs = match sql_plan.plan {
|
||||
Some(plan) => {
|
||||
Some(mut plan) => {
|
||||
fix_placeholder_types(&mut plan)?;
|
||||
let param_types = plan
|
||||
.get_parameter_types()
|
||||
.context(DataFrameSnafu)?
|
||||
@@ -295,6 +298,10 @@ impl MysqlInstanceShim {
|
||||
}
|
||||
Params::CliParams(params) => params.iter().map(|x| x.to_string()).collect(),
|
||||
};
|
||||
debug!(
|
||||
"do_execute Replacing with Params: {:?}, Original Query: {}",
|
||||
param_strs, sql_plan.query
|
||||
);
|
||||
let query = replace_params(param_strs, sql_plan.query);
|
||||
debug!("Mysql execute replaced query: {}", query);
|
||||
self.do_query(&query, query_ctx.clone()).await
|
||||
@@ -412,6 +419,7 @@ impl<W: AsyncWrite + Send + Sync + Unpin> AsyncMysqlShim<W> for MysqlInstanceShi
|
||||
let (params, columns) = self
|
||||
.do_prepare(raw_query, query_ctx.clone(), stmt_key)
|
||||
.await?;
|
||||
debug!("on_prepare: Params: {:?}, Columns: {:?}", params, columns);
|
||||
w.reply(stmt_id, ¶ms, &columns).await?;
|
||||
crate::metrics::METRIC_MYSQL_PREPARED_COUNT
|
||||
.with_label_values(&[query_ctx.get_db_string().as_str()])
|
||||
@@ -641,12 +649,13 @@ fn replace_params_with_values(
|
||||
debug_assert_eq!(param_types.len(), params.len());
|
||||
|
||||
debug!(
|
||||
"replace_params_with_values(param_types: {:#?}, params: {:#?})",
|
||||
"replace_params_with_values(param_types: {:#?}, params: {:#?}, plan: {:#?})",
|
||||
param_types,
|
||||
params
|
||||
.iter()
|
||||
.map(|x| format!("({:?}, {:?})", x.value, x.coltype))
|
||||
.join(", ")
|
||||
.join(", "),
|
||||
plan
|
||||
);
|
||||
|
||||
let mut values = Vec::with_capacity(params.len());
|
||||
@@ -672,9 +681,10 @@ fn replace_params_with_exprs(
|
||||
debug_assert_eq!(param_types.len(), params.len());
|
||||
|
||||
debug!(
|
||||
"replace_params_with_exprs(param_types: {:#?}, params: {:#?})",
|
||||
"replace_params_with_exprs(param_types: {:#?}, params: {:#?}, plan: {:#?})",
|
||||
param_types,
|
||||
params.iter().map(|x| format!("({:?})", x)).join(", ")
|
||||
params.iter().map(|x| format!("({:?})", x)).join(", "),
|
||||
plan
|
||||
);
|
||||
|
||||
let mut values = Vec::with_capacity(params.len());
|
||||
|
||||
@@ -18,6 +18,8 @@ use std::time::Duration;
|
||||
use chrono::NaiveDate;
|
||||
use common_query::prelude::ScalarValue;
|
||||
use common_time::Timestamp;
|
||||
use datafusion_common::tree_node::{Transformed, TreeNode};
|
||||
use datafusion_expr::LogicalPlan;
|
||||
use datatypes::prelude::ConcreteDataType;
|
||||
use datatypes::types::TimestampType;
|
||||
use datatypes::value::{self, Value};
|
||||
@@ -28,7 +30,7 @@ use sql::ast::{visit_expressions_mut, Expr, Value as ValueExpr, VisitMut};
|
||||
use sql::statements::sql_value_to_value;
|
||||
use sql::statements::statement::Statement;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::error::{self, DataFusionSnafu, Result};
|
||||
|
||||
/// Returns the placeholder string "$i".
|
||||
pub fn format_placeholder(i: usize) -> String {
|
||||
@@ -77,6 +79,40 @@ pub fn transform_placeholders(stmt: Statement) -> Statement {
|
||||
}
|
||||
}
|
||||
|
||||
/// Give placeholder that cast to certain type `data_type` the same data type as is cast to
|
||||
///
|
||||
/// because it seems datafusion will not give data type to placeholder if it need to be cast to certain type, still unknown if this is a feature or a bug. And if a placeholder expr have no data type, datafusion will fail to extract it using `LogicalPlan::get_parameter_types`
|
||||
pub fn fix_placeholder_types(plan: &mut LogicalPlan) -> Result<()> {
|
||||
let give_placeholder_types = |mut e: datafusion_expr::Expr| {
|
||||
if let datafusion_expr::Expr::Cast(cast) = &mut e {
|
||||
if let datafusion_expr::Expr::Placeholder(ph) = &mut *cast.expr {
|
||||
if ph.data_type.is_none() {
|
||||
ph.data_type = Some(cast.data_type.clone());
|
||||
common_telemetry::debug!(
|
||||
"give placeholder type {:?} to {:?}",
|
||||
cast.data_type,
|
||||
ph
|
||||
);
|
||||
Ok(Transformed::yes(e))
|
||||
} else {
|
||||
Ok(Transformed::no(e))
|
||||
}
|
||||
} else {
|
||||
Ok(Transformed::no(e))
|
||||
}
|
||||
} else {
|
||||
Ok(Transformed::no(e))
|
||||
}
|
||||
};
|
||||
let give_placeholder_types_recursively =
|
||||
|e: datafusion_expr::Expr| e.transform(give_placeholder_types);
|
||||
*plan = std::mem::take(plan)
|
||||
.transform(|p| p.map_expressions(give_placeholder_types_recursively))
|
||||
.context(DataFusionSnafu)?
|
||||
.data;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn visit_placeholders<V>(v: &mut V)
|
||||
where
|
||||
V: VisitMut,
|
||||
@@ -106,6 +142,7 @@ pub fn convert_value(param: &ParamValue, t: &ConcreteDataType) -> Result<ScalarV
|
||||
ConcreteDataType::UInt64(_) => Ok(ScalarValue::UInt64(Some(i as u64))),
|
||||
ConcreteDataType::Float32(_) => Ok(ScalarValue::Float32(Some(i as f32))),
|
||||
ConcreteDataType::Float64(_) => Ok(ScalarValue::Float64(Some(i as f64))),
|
||||
ConcreteDataType::Boolean(_) => Ok(ScalarValue::Boolean(Some(i != 0))),
|
||||
ConcreteDataType::Timestamp(ts_type) => Value::Timestamp(ts_type.create_timestamp(i))
|
||||
.try_to_scalar_value(t)
|
||||
.context(error::ConvertScalarValueSnafu),
|
||||
@@ -127,6 +164,7 @@ pub fn convert_value(param: &ParamValue, t: &ConcreteDataType) -> Result<ScalarV
|
||||
ConcreteDataType::UInt64(_) => Ok(ScalarValue::UInt64(Some(u))),
|
||||
ConcreteDataType::Float32(_) => Ok(ScalarValue::Float32(Some(u as f32))),
|
||||
ConcreteDataType::Float64(_) => Ok(ScalarValue::Float64(Some(u as f64))),
|
||||
ConcreteDataType::Boolean(_) => Ok(ScalarValue::Boolean(Some(u != 0))),
|
||||
ConcreteDataType::Timestamp(ts_type) => {
|
||||
Value::Timestamp(ts_type.create_timestamp(u as i64))
|
||||
.try_to_scalar_value(t)
|
||||
|
||||
@@ -167,6 +167,7 @@ impl From<QueryContext> for api::v1::QueryContext {
|
||||
snapshot_seqs: Some(api::v1::SnapshotSequences {
|
||||
snapshot_seqs: snapshot_seqs.read().unwrap().clone(),
|
||||
}),
|
||||
explain: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -99,6 +99,7 @@ impl From<GrantedRegion> for PbGrantedRegion {
|
||||
PbGrantedRegion {
|
||||
region_id: value.region_id.as_u64(),
|
||||
role: PbRegionRole::from(value.region_role).into(),
|
||||
extensions: HashMap::new(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -434,8 +434,6 @@ pub struct RegionCloseRequest {}
|
||||
/// Alter metadata of a region.
|
||||
#[derive(Debug, PartialEq, Eq, Clone)]
|
||||
pub struct RegionAlterRequest {
|
||||
/// The version of the schema before applying the alteration.
|
||||
pub schema_version: u64,
|
||||
/// Kind of alteration to do.
|
||||
pub kind: AlterKind,
|
||||
}
|
||||
@@ -443,17 +441,6 @@ pub struct RegionAlterRequest {
|
||||
impl RegionAlterRequest {
|
||||
/// Checks whether the request is valid, returns an error if it is invalid.
|
||||
pub fn validate(&self, metadata: &RegionMetadata) -> Result<()> {
|
||||
ensure!(
|
||||
metadata.schema_version == self.schema_version,
|
||||
InvalidRegionRequestSnafu {
|
||||
region_id: metadata.region_id,
|
||||
err: format!(
|
||||
"region schema version {} is not equal to request schema version {}",
|
||||
metadata.schema_version, self.schema_version
|
||||
),
|
||||
}
|
||||
);
|
||||
|
||||
self.kind.validate(metadata)?;
|
||||
|
||||
Ok(())
|
||||
@@ -477,10 +464,7 @@ impl TryFrom<AlterRequest> for RegionAlterRequest {
|
||||
})?;
|
||||
|
||||
let kind = AlterKind::try_from(kind)?;
|
||||
Ok(RegionAlterRequest {
|
||||
schema_version: value.schema_version,
|
||||
kind,
|
||||
})
|
||||
Ok(RegionAlterRequest { kind })
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1229,7 +1213,6 @@ mod tests {
|
||||
assert_eq!(
|
||||
request,
|
||||
RegionAlterRequest {
|
||||
schema_version: 1,
|
||||
kind: AlterKind::AddColumns {
|
||||
columns: vec![AddColumn {
|
||||
column_metadata: ColumnMetadata {
|
||||
@@ -1526,21 +1509,6 @@ mod tests {
|
||||
assert!(kind.need_alter(&metadata));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_validate_schema_version() {
|
||||
let mut metadata = new_metadata();
|
||||
metadata.schema_version = 2;
|
||||
|
||||
RegionAlterRequest {
|
||||
schema_version: 1,
|
||||
kind: AlterKind::DropColumns {
|
||||
names: vec!["field_0".to_string()],
|
||||
},
|
||||
}
|
||||
.validate(&metadata)
|
||||
.unwrap_err();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_validate_add_columns() {
|
||||
let kind = AlterKind::AddColumns {
|
||||
@@ -1571,10 +1539,7 @@ mod tests {
|
||||
},
|
||||
],
|
||||
};
|
||||
let request = RegionAlterRequest {
|
||||
schema_version: 1,
|
||||
kind,
|
||||
};
|
||||
let request = RegionAlterRequest { kind };
|
||||
let mut metadata = new_metadata();
|
||||
metadata.schema_version = 1;
|
||||
request.validate(&metadata).unwrap();
|
||||
@@ -1634,10 +1599,7 @@ mod tests {
|
||||
},
|
||||
},
|
||||
};
|
||||
let request = RegionAlterRequest {
|
||||
schema_version: 1,
|
||||
kind,
|
||||
};
|
||||
let request = RegionAlterRequest { kind };
|
||||
let mut metadata = new_metadata();
|
||||
metadata.schema_version = 1;
|
||||
request.validate(&metadata).unwrap();
|
||||
@@ -1647,10 +1609,7 @@ mod tests {
|
||||
column_name: "tag_0".to_string(),
|
||||
},
|
||||
};
|
||||
let request = RegionAlterRequest {
|
||||
schema_version: 1,
|
||||
kind,
|
||||
};
|
||||
let request = RegionAlterRequest { kind };
|
||||
let mut metadata = new_metadata();
|
||||
metadata.schema_version = 1;
|
||||
request.validate(&metadata).unwrap();
|
||||
|
||||
@@ -223,6 +223,15 @@ async fn execute_alter_table(ctx: FuzzContext, input: FuzzInput) -> Result<()> {
|
||||
assert_eq!(a, b);
|
||||
}
|
||||
});
|
||||
|
||||
// select from table to make sure the table is still ok
|
||||
let sql = format!("SELECT * FROM {}", table_ctx.name);
|
||||
let result = sqlx::query(&sql)
|
||||
.persistent(false)
|
||||
.execute(&ctx.greptime)
|
||||
.await
|
||||
.context(error::ExecuteQuerySnafu { sql: &sql })?;
|
||||
info!("Select from table: {sql}, result: {result:?}");
|
||||
}
|
||||
|
||||
// Cleans up
|
||||
|
||||
@@ -679,6 +679,19 @@ pub async fn test_prom_http_api(store_type: StorageType) {
|
||||
serde_json::from_value::<PrometheusResponse>(json!(["host1"])).unwrap()
|
||||
);
|
||||
|
||||
// single match[] with __name__
|
||||
let res = client
|
||||
.get("/v1/prometheus/api/v1/label/host/values?match[]={__name__%3D%22demo%22}&start=0&end=300")
|
||||
.send()
|
||||
.await;
|
||||
assert_eq!(res.status(), StatusCode::OK);
|
||||
let body = serde_json::from_str::<PrometheusJsonResponse>(&res.text().await).unwrap();
|
||||
assert_eq!(body.status, "success");
|
||||
assert_eq!(
|
||||
body.data,
|
||||
serde_json::from_value::<PrometheusResponse>(json!(["host1"])).unwrap()
|
||||
);
|
||||
|
||||
// single match[]
|
||||
let res = client
|
||||
.get("/v1/prometheus/api/v1/label/idc/values?match[]=demo_metrics_with_nanos&start=0&end=600")
|
||||
|
||||
@@ -192,3 +192,46 @@ DROP TABLE phy;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
CREATE TABLE grpc_latencies (
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
host STRING,
|
||||
method_name STRING,
|
||||
latency DOUBLE,
|
||||
PRIMARY KEY (host, method_name)
|
||||
) with('append_mode'='true');
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
INSERT INTO grpc_latencies (ts, host, method_name, latency) VALUES
|
||||
('2024-07-11 20:00:06', 'host1', 'GetUser', 103.0);
|
||||
|
||||
Affected Rows: 1
|
||||
|
||||
SELECT * FROM grpc_latencies;
|
||||
|
||||
+---------------------+-------+-------------+---------+
|
||||
| ts | host | method_name | latency |
|
||||
+---------------------+-------+-------------+---------+
|
||||
| 2024-07-11T20:00:06 | host1 | GetUser | 103.0 |
|
||||
+---------------------+-------+-------------+---------+
|
||||
|
||||
ALTER TABLE grpc_latencies SET ttl = '10d';
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
ALTER TABLE grpc_latencies ADD COLUMN home INTEGER FIRST;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
SELECT * FROM grpc_latencies;
|
||||
|
||||
+------+---------------------+-------+-------------+---------+
|
||||
| home | ts | host | method_name | latency |
|
||||
+------+---------------------+-------+-------------+---------+
|
||||
| | 2024-07-11T20:00:06 | host1 | GetUser | 103.0 |
|
||||
+------+---------------------+-------+-------------+---------+
|
||||
|
||||
DROP TABLE grpc_latencies;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
|
||||
@@ -95,3 +95,24 @@ DROP TABLE t1;
|
||||
DROP TABLE t2;
|
||||
|
||||
DROP TABLE phy;
|
||||
|
||||
CREATE TABLE grpc_latencies (
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
host STRING,
|
||||
method_name STRING,
|
||||
latency DOUBLE,
|
||||
PRIMARY KEY (host, method_name)
|
||||
) with('append_mode'='true');
|
||||
|
||||
INSERT INTO grpc_latencies (ts, host, method_name, latency) VALUES
|
||||
('2024-07-11 20:00:06', 'host1', 'GetUser', 103.0);
|
||||
|
||||
SELECT * FROM grpc_latencies;
|
||||
|
||||
ALTER TABLE grpc_latencies SET ttl = '10d';
|
||||
|
||||
ALTER TABLE grpc_latencies ADD COLUMN home INTEGER FIRST;
|
||||
|
||||
SELECT * FROM grpc_latencies;
|
||||
|
||||
DROP TABLE grpc_latencies;
|
||||
@@ -229,3 +229,106 @@ DROP TABLE ngx_country;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
-- test nullable pk with no default value
|
||||
CREATE TABLE nullable_pk (
|
||||
pid INT NULL,
|
||||
client STRING,
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY (pid)
|
||||
) WITH (
|
||||
append_mode = 'true'
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
CREATE TABLE out_nullable_pk (
|
||||
pid INT NULL,
|
||||
client STRING,
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY (pid, client)
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
CREATE FLOW calc_nullable_pk SINK TO out_nullable_pk AS
|
||||
SELECT
|
||||
pid,
|
||||
client,
|
||||
ts
|
||||
FROM
|
||||
nullable_pk;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
INSERT INTO
|
||||
nullable_pk
|
||||
VALUES
|
||||
(1, "name1", "2024-10-18 19:00:00"),
|
||||
(2, "name2", "2024-10-18 19:00:00"),
|
||||
(3, "name3", "2024-10-18 19:00:00");
|
||||
|
||||
Affected Rows: 3
|
||||
|
||||
-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED |
|
||||
ADMIN FLUSH_FLOW('calc_nullable_pk');
|
||||
|
||||
+--------------------------------------+
|
||||
| ADMIN FLUSH_FLOW('calc_nullable_pk') |
|
||||
+--------------------------------------+
|
||||
| FLOW_FLUSHED |
|
||||
+--------------------------------------+
|
||||
|
||||
SELECT * FROM out_nullable_pk;
|
||||
|
||||
+-----+--------+---------------------+
|
||||
| pid | client | ts |
|
||||
+-----+--------+---------------------+
|
||||
| 1 | name1 | 2024-10-18T19:00:00 |
|
||||
| 2 | name2 | 2024-10-18T19:00:00 |
|
||||
| 3 | name3 | 2024-10-18T19:00:00 |
|
||||
+-----+--------+---------------------+
|
||||
|
||||
-- pk is nullable
|
||||
INSERT INTO
|
||||
nullable_pk (client, ts)
|
||||
VALUES
|
||||
("name1", "2024-10-18 19:00:00"),
|
||||
("name2", "2024-10-18 19:00:00"),
|
||||
("name3", "2024-10-18 19:00:00");
|
||||
|
||||
Affected Rows: 3
|
||||
|
||||
-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED |
|
||||
ADMIN FLUSH_FLOW('calc_nullable_pk');
|
||||
|
||||
+--------------------------------------+
|
||||
| ADMIN FLUSH_FLOW('calc_nullable_pk') |
|
||||
+--------------------------------------+
|
||||
| FLOW_FLUSHED |
|
||||
+--------------------------------------+
|
||||
|
||||
SELECT * FROM out_nullable_pk;
|
||||
|
||||
+-----+--------+---------------------+
|
||||
| pid | client | ts |
|
||||
+-----+--------+---------------------+
|
||||
| | name1 | 2024-10-18T19:00:00 |
|
||||
| | name2 | 2024-10-18T19:00:00 |
|
||||
| | name3 | 2024-10-18T19:00:00 |
|
||||
| 1 | name1 | 2024-10-18T19:00:00 |
|
||||
| 2 | name2 | 2024-10-18T19:00:00 |
|
||||
| 3 | name3 | 2024-10-18T19:00:00 |
|
||||
+-----+--------+---------------------+
|
||||
|
||||
DROP FLOW calc_nullable_pk;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
DROP TABLE nullable_pk;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
DROP TABLE out_nullable_pk;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
|
||||
@@ -133,3 +133,59 @@ DROP FLOW calc_ngx_country;
|
||||
DROP TABLE ngx_access_log;
|
||||
|
||||
DROP TABLE ngx_country;
|
||||
|
||||
-- test nullable pk with no default value
|
||||
CREATE TABLE nullable_pk (
|
||||
pid INT NULL,
|
||||
client STRING,
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY (pid)
|
||||
) WITH (
|
||||
append_mode = 'true'
|
||||
);
|
||||
|
||||
CREATE TABLE out_nullable_pk (
|
||||
pid INT NULL,
|
||||
client STRING,
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY (pid, client)
|
||||
);
|
||||
|
||||
CREATE FLOW calc_nullable_pk SINK TO out_nullable_pk AS
|
||||
SELECT
|
||||
pid,
|
||||
client,
|
||||
ts
|
||||
FROM
|
||||
nullable_pk;
|
||||
|
||||
INSERT INTO
|
||||
nullable_pk
|
||||
VALUES
|
||||
(1, "name1", "2024-10-18 19:00:00"),
|
||||
(2, "name2", "2024-10-18 19:00:00"),
|
||||
(3, "name3", "2024-10-18 19:00:00");
|
||||
|
||||
-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED |
|
||||
ADMIN FLUSH_FLOW('calc_nullable_pk');
|
||||
|
||||
SELECT * FROM out_nullable_pk;
|
||||
|
||||
-- pk is nullable
|
||||
INSERT INTO
|
||||
nullable_pk (client, ts)
|
||||
VALUES
|
||||
("name1", "2024-10-18 19:00:00"),
|
||||
("name2", "2024-10-18 19:00:00"),
|
||||
("name3", "2024-10-18 19:00:00");
|
||||
|
||||
-- SQLNESS REPLACE (ADMIN\sFLUSH_FLOW\('\w+'\)\s+\|\n\+-+\+\n\|\s+)[0-9]+\s+\| $1 FLOW_FLUSHED |
|
||||
ADMIN FLUSH_FLOW('calc_nullable_pk');
|
||||
|
||||
SELECT * FROM out_nullable_pk;
|
||||
|
||||
DROP FLOW calc_nullable_pk;
|
||||
|
||||
DROP TABLE nullable_pk;
|
||||
|
||||
DROP TABLE out_nullable_pk;
|
||||
|
||||
@@ -13,16 +13,16 @@ affected_rows: 0
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 1;
|
||||
|
||||
+----------+
|
||||
| Int64(1) |
|
||||
+----------+
|
||||
| 1 |
|
||||
+----------+
|
||||
+----+
|
||||
| $1 |
|
||||
+----+
|
||||
| 1 |
|
||||
+----+
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 'a';
|
||||
|
||||
Failed to parse query result, err: MySqlError { ERROR 1815 (HY000): (EngineExecuteQuery): Cast error: Cannot cast string 'a' to value of Int32 type }
|
||||
Failed to execute query, err: MySqlError { ERROR 1210 (HY000): (InvalidArguments): Invalid request parameter: Column expect type: Int32(Int32Type), actual: String(StringType) }
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
@@ -59,14 +59,73 @@ affected_rows: 0
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 1, 'hello';
|
||||
|
||||
+----------+---------------+
|
||||
| Int64(1) | Utf8("hello") |
|
||||
+----------+---------------+
|
||||
| 1 | hello |
|
||||
+----------+---------------+
|
||||
+----+-------+
|
||||
| $1 | $2 |
|
||||
+----+-------+
|
||||
| 1 | hello |
|
||||
+----+-------+
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
-- test if placeholder at limit and offset are parsed correctly
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
PREPARE stmt FROM 'SELECT 1 LIMIT ? OFFSET ?;';
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 1, 2;
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
-- test with data
|
||||
CREATE TABLE IF NOT EXISTS "cake" (
|
||||
`domain` STRING,
|
||||
is_expire BOOLEAN NULL,
|
||||
ts TIMESTAMP(3),
|
||||
TIME INDEX ("ts"),
|
||||
PRIMARY KEY ("domain")
|
||||
) ENGINE=mito
|
||||
WITH(
|
||||
append_mode = 'true',
|
||||
ttl='7days'
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
INSERT INTO cake(domain, is_expire, ts) VALUES('happy', false, '2025-03-18 12:55:51.758000');
|
||||
|
||||
Affected Rows: 1
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
PREPARE stmt FROM 'SELECT `cake`.`domain`, `cake`.`is_expire`, `cake`.`ts` FROM `cake` WHERE `cake`.`domain` = ? LIMIT ? OFFSET ?';
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 'happy', 42, 0;
|
||||
|
||||
+--------+-----------+----------------------------+
|
||||
| domain | is_expire | ts |
|
||||
+--------+-----------+----------------------------+
|
||||
| happy | 0 | 2025-03-18 12:55:51.758000 |
|
||||
+--------+-----------+----------------------------+
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DROP TABLE cake;
|
||||
|
||||
affected_rows: 0
|
||||
|
||||
|
||||
@@ -37,3 +37,40 @@ EXECUTE stmt USING 1, 'hello';
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
|
||||
-- test if placeholder at limit and offset are parsed correctly
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
PREPARE stmt FROM 'SELECT 1 LIMIT ? OFFSET ?;';
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 1, 2;
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
|
||||
-- test with data
|
||||
CREATE TABLE IF NOT EXISTS "cake" (
|
||||
`domain` STRING,
|
||||
is_expire BOOLEAN NULL,
|
||||
ts TIMESTAMP(3),
|
||||
TIME INDEX ("ts"),
|
||||
PRIMARY KEY ("domain")
|
||||
) ENGINE=mito
|
||||
WITH(
|
||||
append_mode = 'true',
|
||||
ttl='7days'
|
||||
);
|
||||
|
||||
INSERT INTO cake(domain, is_expire, ts) VALUES('happy', false, '2025-03-18 12:55:51.758000');
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
PREPARE stmt FROM 'SELECT `cake`.`domain`, `cake`.`is_expire`, `cake`.`ts` FROM `cake` WHERE `cake`.`domain` = ? LIMIT ? OFFSET ?';
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
EXECUTE stmt USING 'happy', 42, 0;
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DEALLOCATE stmt;
|
||||
|
||||
-- SQLNESS PROTOCOL MYSQL
|
||||
DROP TABLE cake;
|
||||
|
||||
@@ -33,6 +33,63 @@ SELECT ts, host, min(val) RANGE (INTERVAL '1' day) FROM host ALIGN (INTERVAL '1'
|
||||
| 1971-01-02T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE ('1 day'::INTERVAL) FROM host ALIGN ('1 day'::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| ts | host | min(host.val) RANGE Utf8("1 day") |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
| 1970-01-01T00:00:00 | host1 | 0 |
|
||||
| 1971-01-02T00:00:00 | host1 | 2 |
|
||||
| 1970-01-01T00:00:00 | host2 | 4 |
|
||||
| 1971-01-02T00:00:00 | host2 | 6 |
|
||||
+---------------------+-------+-----------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE ('1 hour'::INTERVAL) FROM host ALIGN ('1 hour'::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+------------------------------------+
|
||||
| ts | host | min(host.val) RANGE Utf8("1 hour") |
|
||||
+---------------------+-------+------------------------------------+
|
||||
| 1970-01-01T01:00:00 | host1 | 0 |
|
||||
| 1970-01-01T02:00:00 | host1 | 1 |
|
||||
| 1971-01-02T03:00:00 | host1 | 2 |
|
||||
| 1971-01-02T04:00:00 | host1 | 3 |
|
||||
| 1970-01-01T01:00:00 | host2 | 4 |
|
||||
| 1970-01-01T02:00:00 | host2 | 5 |
|
||||
| 1971-01-02T03:00:00 | host2 | 6 |
|
||||
| 1971-01-02T04:00:00 | host2 | 7 |
|
||||
+---------------------+-------+------------------------------------+
|
||||
|
||||
SELECT ts, host, min(val) RANGE ('30 minute'::INTERVAL + '30 minute'::INTERVAL) FROM host ALIGN ('30 minute'::INTERVAL + '30 minute'::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+-----------------------------------------------------------+
|
||||
| ts | host | min(host.val) RANGE Utf8("30 minute") + Utf8("30 minute") |
|
||||
+---------------------+-------+-----------------------------------------------------------+
|
||||
| 1970-01-01T01:00:00 | host1 | 0 |
|
||||
| 1970-01-01T02:00:00 | host1 | 1 |
|
||||
| 1971-01-02T03:00:00 | host1 | 2 |
|
||||
| 1971-01-02T04:00:00 | host1 | 3 |
|
||||
| 1970-01-01T01:00:00 | host2 | 4 |
|
||||
| 1970-01-01T02:00:00 | host2 | 5 |
|
||||
| 1971-01-02T03:00:00 | host2 | 6 |
|
||||
| 1971-01-02T04:00:00 | host2 | 7 |
|
||||
+---------------------+-------+-----------------------------------------------------------+
|
||||
|
||||
--- Test nested cast, even though it is meaningless ----
|
||||
SELECT ts, host, min(val) RANGE ((INTERVAL '1' hour)::INTERVAL) FROM host ALIGN ('1 hour'::INTERVAL::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------------------------------+
|
||||
| ts | host | min(host.val) RANGE IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }") |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------------------------------+
|
||||
| 1970-01-01T01:00:00 | host1 | 0 |
|
||||
| 1970-01-01T02:00:00 | host1 | 1 |
|
||||
| 1971-01-02T03:00:00 | host1 | 2 |
|
||||
| 1971-01-02T04:00:00 | host1 | 3 |
|
||||
| 1970-01-01T01:00:00 | host2 | 4 |
|
||||
| 1970-01-01T02:00:00 | host2 | 5 |
|
||||
| 1971-01-02T03:00:00 | host2 | 6 |
|
||||
| 1971-01-02T04:00:00 | host2 | 7 |
|
||||
+---------------------+-------+---------------------------------------------------------------------------------------------------------------------+
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
@@ -18,4 +18,13 @@ SELECT ts, host, min(val) RANGE (INTERVAL '1 year') FROM host ALIGN (INTERVAL '1
|
||||
|
||||
SELECT ts, host, min(val) RANGE (INTERVAL '1' day) FROM host ALIGN (INTERVAL '1' day) ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE ('1 day'::INTERVAL) FROM host ALIGN ('1 day'::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE ('1 hour'::INTERVAL) FROM host ALIGN ('1 hour'::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
SELECT ts, host, min(val) RANGE ('30 minute'::INTERVAL + '30 minute'::INTERVAL) FROM host ALIGN ('30 minute'::INTERVAL + '30 minute'::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
--- Test nested cast, even though it is meaningless ----
|
||||
SELECT ts, host, min(val) RANGE ((INTERVAL '1' hour)::INTERVAL) FROM host ALIGN ('1 hour'::INTERVAL::INTERVAL) ORDER BY host, ts;
|
||||
|
||||
DROP TABLE host;
|
||||
|
||||
Reference in New Issue
Block a user