mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-23 06:30:05 +00:00
test: gc integration test (#7306)
* test: basic infra for set gc Signed-off-by: discord9 <discord9@163.com> * more stuff Signed-off-by: discord9 <discord9@163.com> * test: basic gc integration test Signed-off-by: discord9 <discord9@163.com> * rm unused Signed-off-by: discord9 <discord9@163.com> * clippy Signed-off-by: discord9 <discord9@163.com> * refactor: remove loader Signed-off-by: discord9 <discord9@163.com> * clippy Signed-off-by: discord9 <discord9@163.com> * fix: allow default endpoint Signed-off-by: discord9 <discord9@163.com> * filter out files Signed-off-by: discord9 <discord9@163.com> * chore: rm minio support Signed-off-by: discord9 <discord9@163.com> --------- Signed-off-by: discord9 <discord9@163.com>
This commit is contained in:
1
Cargo.lock
generated
1
Cargo.lock
generated
@@ -13065,6 +13065,7 @@ dependencies = [
|
|||||||
"loki-proto",
|
"loki-proto",
|
||||||
"meta-client",
|
"meta-client",
|
||||||
"meta-srv",
|
"meta-srv",
|
||||||
|
"mito2",
|
||||||
"moka",
|
"moka",
|
||||||
"mysql_async",
|
"mysql_async",
|
||||||
"object-store",
|
"object-store",
|
||||||
|
|||||||
@@ -246,14 +246,6 @@ pub enum Error {
|
|||||||
#[snafu(implicit)]
|
#[snafu(implicit)]
|
||||||
location: Location,
|
location: Location,
|
||||||
},
|
},
|
||||||
|
|
||||||
#[snafu(display("Loader for {type_name} is not implemented: {reason}"))]
|
|
||||||
ProcedureLoaderNotImplemented {
|
|
||||||
#[snafu(implicit)]
|
|
||||||
location: Location,
|
|
||||||
type_name: String,
|
|
||||||
reason: String,
|
|
||||||
},
|
|
||||||
}
|
}
|
||||||
|
|
||||||
pub type Result<T> = std::result::Result<T, Error>;
|
pub type Result<T> = std::result::Result<T, Error>;
|
||||||
@@ -274,8 +266,7 @@ impl ErrorExt for Error {
|
|||||||
Error::ToJson { .. }
|
Error::ToJson { .. }
|
||||||
| Error::DeleteState { .. }
|
| Error::DeleteState { .. }
|
||||||
| Error::FromJson { .. }
|
| Error::FromJson { .. }
|
||||||
| Error::WaitWatcher { .. }
|
| Error::WaitWatcher { .. } => StatusCode::Internal,
|
||||||
| Error::ProcedureLoaderNotImplemented { .. } => StatusCode::Internal,
|
|
||||||
|
|
||||||
Error::RetryTimesExceeded { .. }
|
Error::RetryTimesExceeded { .. }
|
||||||
| Error::RollbackTimesExceeded { .. }
|
| Error::RollbackTimesExceeded { .. }
|
||||||
|
|||||||
@@ -28,9 +28,10 @@ mod procedure;
|
|||||||
mod scheduler;
|
mod scheduler;
|
||||||
mod tracker;
|
mod tracker;
|
||||||
|
|
||||||
pub(crate) use options::GcSchedulerOptions;
|
pub use options::GcSchedulerOptions;
|
||||||
|
pub use procedure::BatchGcProcedure;
|
||||||
pub(crate) use scheduler::{GcScheduler, GcTickerRef};
|
pub(crate) use scheduler::{GcScheduler, GcTickerRef};
|
||||||
|
|
||||||
pub(crate) type Region2Peers = HashMap<RegionId, (Peer, Vec<Peer>)>;
|
pub type Region2Peers = HashMap<RegionId, (Peer, Vec<Peer>)>;
|
||||||
|
|
||||||
pub(crate) type Peer2Regions = HashMap<Peer, HashSet<RegionId>>;
|
pub(crate) type Peer2Regions = HashMap<Peer, HashSet<RegionId>>;
|
||||||
|
|||||||
@@ -84,44 +84,6 @@ impl DefaultGcSchedulerCtx {
|
|||||||
mailbox: MailboxRef,
|
mailbox: MailboxRef,
|
||||||
server_addr: String,
|
server_addr: String,
|
||||||
) -> Result<Self> {
|
) -> Result<Self> {
|
||||||
// register a noop loader for `GcRegionProcedure` to avoid error when deserializing procedure when rebooting
|
|
||||||
|
|
||||||
procedure_manager
|
|
||||||
.register_loader(
|
|
||||||
GcRegionProcedure::TYPE_NAME,
|
|
||||||
Box::new(move |json| {
|
|
||||||
common_procedure::error::ProcedureLoaderNotImplementedSnafu {
|
|
||||||
type_name: GcRegionProcedure::TYPE_NAME.to_string(),
|
|
||||||
reason:
|
|
||||||
"GC procedure should be retried by scheduler, not reloaded from storage"
|
|
||||||
.to_string(),
|
|
||||||
}
|
|
||||||
.fail()
|
|
||||||
}),
|
|
||||||
)
|
|
||||||
.context(error::RegisterProcedureLoaderSnafu {
|
|
||||||
type_name: GcRegionProcedure::TYPE_NAME,
|
|
||||||
})?;
|
|
||||||
|
|
||||||
// register a noop loader for `BatchGcProcedure` to avoid error when deserializing procedure when rebooting
|
|
||||||
|
|
||||||
procedure_manager
|
|
||||||
.register_loader(
|
|
||||||
BatchGcProcedure::TYPE_NAME,
|
|
||||||
Box::new(move |json| {
|
|
||||||
common_procedure::error::ProcedureLoaderNotImplementedSnafu {
|
|
||||||
type_name: BatchGcProcedure::TYPE_NAME.to_string(),
|
|
||||||
reason:
|
|
||||||
"Batch GC procedure should not be reloaded from storage, as it doesn't need to be retried if interrupted"
|
|
||||||
.to_string(),
|
|
||||||
}
|
|
||||||
.fail()
|
|
||||||
}),
|
|
||||||
)
|
|
||||||
.context(error::RegisterProcedureLoaderSnafu {
|
|
||||||
type_name: BatchGcProcedure::TYPE_NAME,
|
|
||||||
})?;
|
|
||||||
|
|
||||||
Ok(Self {
|
Ok(Self {
|
||||||
table_metadata_manager,
|
table_metadata_manager,
|
||||||
procedure_manager,
|
procedure_manager,
|
||||||
|
|||||||
@@ -54,6 +54,7 @@ log-query = { workspace = true }
|
|||||||
loki-proto.workspace = true
|
loki-proto.workspace = true
|
||||||
meta-client.workspace = true
|
meta-client.workspace = true
|
||||||
meta-srv = { workspace = true, features = ["mock"] }
|
meta-srv = { workspace = true, features = ["mock"] }
|
||||||
|
mito2.workspace = true
|
||||||
moka.workspace = true
|
moka.workspace = true
|
||||||
mysql_async = { version = "0.35", default-features = false, features = [
|
mysql_async = { version = "0.35", default-features = false, features = [
|
||||||
"time",
|
"time",
|
||||||
|
|||||||
@@ -59,8 +59,10 @@ use hyper_util::rt::TokioIo;
|
|||||||
use meta_client::client::MetaClientBuilder;
|
use meta_client::client::MetaClientBuilder;
|
||||||
use meta_srv::cluster::MetaPeerClientRef;
|
use meta_srv::cluster::MetaPeerClientRef;
|
||||||
use meta_srv::discovery;
|
use meta_srv::discovery;
|
||||||
|
use meta_srv::gc::GcSchedulerOptions;
|
||||||
use meta_srv::metasrv::{Metasrv, MetasrvOptions, SelectorRef};
|
use meta_srv::metasrv::{Metasrv, MetasrvOptions, SelectorRef};
|
||||||
use meta_srv::mocks::MockInfo;
|
use meta_srv::mocks::MockInfo;
|
||||||
|
use mito2::gc::GcConfig;
|
||||||
use object_store::config::ObjectStoreConfig;
|
use object_store::config::ObjectStoreConfig;
|
||||||
use rand::Rng;
|
use rand::Rng;
|
||||||
use servers::grpc::GrpcOptions;
|
use servers::grpc::GrpcOptions;
|
||||||
@@ -103,6 +105,8 @@ pub struct GreptimeDbClusterBuilder {
|
|||||||
datanodes: Option<u32>,
|
datanodes: Option<u32>,
|
||||||
datanode_wal_config: DatanodeWalConfig,
|
datanode_wal_config: DatanodeWalConfig,
|
||||||
metasrv_wal_config: MetasrvWalConfig,
|
metasrv_wal_config: MetasrvWalConfig,
|
||||||
|
datanode_gc_config: GcConfig,
|
||||||
|
metasrv_gc_config: GcSchedulerOptions,
|
||||||
shared_home_dir: Option<Arc<TempDir>>,
|
shared_home_dir: Option<Arc<TempDir>>,
|
||||||
meta_selector: Option<SelectorRef>,
|
meta_selector: Option<SelectorRef>,
|
||||||
}
|
}
|
||||||
@@ -134,6 +138,8 @@ impl GreptimeDbClusterBuilder {
|
|||||||
datanodes: None,
|
datanodes: None,
|
||||||
datanode_wal_config: DatanodeWalConfig::default(),
|
datanode_wal_config: DatanodeWalConfig::default(),
|
||||||
metasrv_wal_config: MetasrvWalConfig::default(),
|
metasrv_wal_config: MetasrvWalConfig::default(),
|
||||||
|
datanode_gc_config: GcConfig::default(),
|
||||||
|
metasrv_gc_config: GcSchedulerOptions::default(),
|
||||||
shared_home_dir: None,
|
shared_home_dir: None,
|
||||||
meta_selector: None,
|
meta_selector: None,
|
||||||
}
|
}
|
||||||
@@ -169,6 +175,17 @@ impl GreptimeDbClusterBuilder {
|
|||||||
self
|
self
|
||||||
}
|
}
|
||||||
|
|
||||||
|
#[must_use]
|
||||||
|
pub fn with_datanode_gc_config(mut self, datanode_gc_config: GcConfig) -> Self {
|
||||||
|
self.datanode_gc_config = datanode_gc_config;
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
|
pub fn with_metasrv_gc_config(mut self, metasrv_gc_config: GcSchedulerOptions) -> Self {
|
||||||
|
self.metasrv_gc_config = metasrv_gc_config;
|
||||||
|
self
|
||||||
|
}
|
||||||
|
|
||||||
#[must_use]
|
#[must_use]
|
||||||
pub fn with_shared_home_dir(mut self, shared_home_dir: Arc<TempDir>) -> Self {
|
pub fn with_shared_home_dir(mut self, shared_home_dir: Arc<TempDir>) -> Self {
|
||||||
self.shared_home_dir = Some(shared_home_dir);
|
self.shared_home_dir = Some(shared_home_dir);
|
||||||
@@ -205,6 +222,7 @@ impl GreptimeDbClusterBuilder {
|
|||||||
server_addr: "127.0.0.1:3002".to_string(),
|
server_addr: "127.0.0.1:3002".to_string(),
|
||||||
..Default::default()
|
..Default::default()
|
||||||
},
|
},
|
||||||
|
gc: self.metasrv_gc_config.clone(),
|
||||||
..Default::default()
|
..Default::default()
|
||||||
};
|
};
|
||||||
|
|
||||||
@@ -279,6 +297,7 @@ impl GreptimeDbClusterBuilder {
|
|||||||
vec![],
|
vec![],
|
||||||
home_dir,
|
home_dir,
|
||||||
self.datanode_wal_config.clone(),
|
self.datanode_wal_config.clone(),
|
||||||
|
self.datanode_gc_config.clone(),
|
||||||
)
|
)
|
||||||
} else {
|
} else {
|
||||||
let (opts, guard) = create_tmp_dir_and_datanode_opts(
|
let (opts, guard) = create_tmp_dir_and_datanode_opts(
|
||||||
@@ -286,6 +305,7 @@ impl GreptimeDbClusterBuilder {
|
|||||||
self.store_providers.clone().unwrap_or_default(),
|
self.store_providers.clone().unwrap_or_default(),
|
||||||
&format!("{}-dn-{}", self.cluster_name, datanode_id),
|
&format!("{}-dn-{}", self.cluster_name, datanode_id),
|
||||||
self.datanode_wal_config.clone(),
|
self.datanode_wal_config.clone(),
|
||||||
|
self.datanode_gc_config.clone(),
|
||||||
);
|
);
|
||||||
guards.push(guard);
|
guards.push(guard);
|
||||||
|
|
||||||
|
|||||||
@@ -309,6 +309,7 @@ impl GreptimeDbStandaloneBuilder {
|
|||||||
store_types,
|
store_types,
|
||||||
&self.instance_name,
|
&self.instance_name,
|
||||||
self.datanode_wal_config.clone(),
|
self.datanode_wal_config.clone(),
|
||||||
|
Default::default(),
|
||||||
);
|
);
|
||||||
|
|
||||||
let kv_backend_config = KvBackendConfig::default();
|
let kv_backend_config = KvBackendConfig::default();
|
||||||
|
|||||||
@@ -32,6 +32,7 @@ use common_wal::config::DatanodeWalConfig;
|
|||||||
use datanode::config::{DatanodeOptions, StorageConfig};
|
use datanode::config::{DatanodeOptions, StorageConfig};
|
||||||
use frontend::instance::Instance;
|
use frontend::instance::Instance;
|
||||||
use frontend::service_config::{MysqlOptions, PostgresOptions};
|
use frontend::service_config::{MysqlOptions, PostgresOptions};
|
||||||
|
use mito2::gc::GcConfig;
|
||||||
use object_store::config::{
|
use object_store::config::{
|
||||||
AzblobConfig, FileConfig, GcsConfig, ObjectStoreConfig, OssConfig, S3Config,
|
AzblobConfig, FileConfig, GcsConfig, ObjectStoreConfig, OssConfig, S3Config,
|
||||||
};
|
};
|
||||||
@@ -145,6 +146,7 @@ fn s3_test_config() -> S3Config {
|
|||||||
secret_access_key: env::var("GT_S3_ACCESS_KEY").unwrap().into(),
|
secret_access_key: env::var("GT_S3_ACCESS_KEY").unwrap().into(),
|
||||||
bucket: env::var("GT_S3_BUCKET").unwrap(),
|
bucket: env::var("GT_S3_BUCKET").unwrap(),
|
||||||
region: Some(env::var("GT_S3_REGION").unwrap()),
|
region: Some(env::var("GT_S3_REGION").unwrap()),
|
||||||
|
endpoint: env::var("GT_S3_ENDPOINT_URL").ok(),
|
||||||
..Default::default()
|
..Default::default()
|
||||||
},
|
},
|
||||||
..Default::default()
|
..Default::default()
|
||||||
@@ -163,7 +165,7 @@ pub fn get_test_store_config(store_type: &StorageType) -> (ObjectStoreConfig, Te
|
|||||||
scope: env::var("GT_GCS_SCOPE").unwrap(),
|
scope: env::var("GT_GCS_SCOPE").unwrap(),
|
||||||
credential_path: env::var("GT_GCS_CREDENTIAL_PATH").unwrap().into(),
|
credential_path: env::var("GT_GCS_CREDENTIAL_PATH").unwrap().into(),
|
||||||
credential: env::var("GT_GCS_CREDENTIAL").unwrap().into(),
|
credential: env::var("GT_GCS_CREDENTIAL").unwrap().into(),
|
||||||
endpoint: env::var("GT_GCS_ENDPOINT").unwrap(),
|
endpoint: env::var("GT_GCS_ENDPOINT").unwrap_or_default(),
|
||||||
},
|
},
|
||||||
..Default::default()
|
..Default::default()
|
||||||
};
|
};
|
||||||
@@ -297,6 +299,7 @@ pub fn create_tmp_dir_and_datanode_opts(
|
|||||||
store_provider_types: Vec<StorageType>,
|
store_provider_types: Vec<StorageType>,
|
||||||
name: &str,
|
name: &str,
|
||||||
wal_config: DatanodeWalConfig,
|
wal_config: DatanodeWalConfig,
|
||||||
|
gc_config: GcConfig,
|
||||||
) -> (DatanodeOptions, TestGuard) {
|
) -> (DatanodeOptions, TestGuard) {
|
||||||
let home_tmp_dir = create_temp_dir(&format!("gt_data_{name}"));
|
let home_tmp_dir = create_temp_dir(&format!("gt_data_{name}"));
|
||||||
let home_dir = home_tmp_dir.path().to_str().unwrap().to_string();
|
let home_dir = home_tmp_dir.path().to_str().unwrap().to_string();
|
||||||
@@ -314,7 +317,13 @@ pub fn create_tmp_dir_and_datanode_opts(
|
|||||||
store_providers.push(store);
|
store_providers.push(store);
|
||||||
storage_guards.push(StorageGuard(data_tmp_dir))
|
storage_guards.push(StorageGuard(data_tmp_dir))
|
||||||
}
|
}
|
||||||
let opts = create_datanode_opts(default_store, store_providers, home_dir, wal_config);
|
let opts = create_datanode_opts(
|
||||||
|
default_store,
|
||||||
|
store_providers,
|
||||||
|
home_dir,
|
||||||
|
wal_config,
|
||||||
|
gc_config,
|
||||||
|
);
|
||||||
|
|
||||||
(
|
(
|
||||||
opts,
|
opts,
|
||||||
@@ -330,7 +339,18 @@ pub(crate) fn create_datanode_opts(
|
|||||||
providers: Vec<ObjectStoreConfig>,
|
providers: Vec<ObjectStoreConfig>,
|
||||||
home_dir: String,
|
home_dir: String,
|
||||||
wal_config: DatanodeWalConfig,
|
wal_config: DatanodeWalConfig,
|
||||||
|
gc_config: GcConfig,
|
||||||
) -> DatanodeOptions {
|
) -> DatanodeOptions {
|
||||||
|
let region_engine = DatanodeOptions::default()
|
||||||
|
.region_engine
|
||||||
|
.into_iter()
|
||||||
|
.map(|mut v| {
|
||||||
|
if let datanode::config::RegionEngineConfig::Mito(mito_config) = &mut v {
|
||||||
|
mito_config.gc = gc_config.clone();
|
||||||
|
}
|
||||||
|
v
|
||||||
|
})
|
||||||
|
.collect();
|
||||||
DatanodeOptions {
|
DatanodeOptions {
|
||||||
node_id: Some(0),
|
node_id: Some(0),
|
||||||
require_lease_before_startup: true,
|
require_lease_before_startup: true,
|
||||||
@@ -343,6 +363,7 @@ pub(crate) fn create_datanode_opts(
|
|||||||
.with_bind_addr(PEER_PLACEHOLDER_ADDR)
|
.with_bind_addr(PEER_PLACEHOLDER_ADDR)
|
||||||
.with_server_addr(PEER_PLACEHOLDER_ADDR),
|
.with_server_addr(PEER_PLACEHOLDER_ADDR),
|
||||||
wal: wal_config,
|
wal: wal_config,
|
||||||
|
region_engine,
|
||||||
..Default::default()
|
..Default::default()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -12,6 +12,7 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
|
mod gc;
|
||||||
mod instance_kafka_wal_test;
|
mod instance_kafka_wal_test;
|
||||||
mod instance_noop_wal_test;
|
mod instance_noop_wal_test;
|
||||||
mod instance_test;
|
mod instance_test;
|
||||||
|
|||||||
262
tests-integration/src/tests/gc.rs
Normal file
262
tests-integration/src/tests/gc.rs
Normal file
@@ -0,0 +1,262 @@
|
|||||||
|
// 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, HashSet};
|
||||||
|
use std::time::Duration;
|
||||||
|
|
||||||
|
use common_meta::key::TableMetadataManagerRef;
|
||||||
|
use common_procedure::ProcedureWithId;
|
||||||
|
use common_telemetry::info;
|
||||||
|
use common_test_util::recordbatch::check_output_stream;
|
||||||
|
use futures::TryStreamExt as _;
|
||||||
|
use itertools::Itertools;
|
||||||
|
use meta_srv::gc::{BatchGcProcedure, GcSchedulerOptions, Region2Peers};
|
||||||
|
use mito2::gc::GcConfig;
|
||||||
|
use store_api::storage::RegionId;
|
||||||
|
use table::metadata::TableId;
|
||||||
|
|
||||||
|
use crate::cluster::GreptimeDbClusterBuilder;
|
||||||
|
use crate::test_util::{StorageType, TempDirGuard, get_test_store_config};
|
||||||
|
use crate::tests::test_util::{MockInstanceBuilder, TestContext, execute_sql, wait_procedure};
|
||||||
|
|
||||||
|
/// Helper function to get table route information for GC procedure
|
||||||
|
async fn get_table_route(
|
||||||
|
table_metadata_manager: &TableMetadataManagerRef,
|
||||||
|
table_id: TableId,
|
||||||
|
) -> (Region2Peers, Vec<RegionId>) {
|
||||||
|
// Get physical table route
|
||||||
|
let (_, physical_table_route) = table_metadata_manager
|
||||||
|
.table_route_manager()
|
||||||
|
.get_physical_table_route(table_id)
|
||||||
|
.await
|
||||||
|
.unwrap();
|
||||||
|
|
||||||
|
let mut region_routes = Region2Peers::new();
|
||||||
|
let mut regions = Vec::new();
|
||||||
|
|
||||||
|
// Convert region routes to Region2Peers format
|
||||||
|
for region_route in physical_table_route.region_routes {
|
||||||
|
let region_id = region_route.region.id;
|
||||||
|
let leader_peer = region_route.leader_peer.clone().unwrap();
|
||||||
|
let follower_peers = region_route.follower_peers.clone();
|
||||||
|
|
||||||
|
region_routes.insert(region_id, (leader_peer, follower_peers));
|
||||||
|
regions.push(region_id);
|
||||||
|
}
|
||||||
|
|
||||||
|
(region_routes, regions)
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Helper function to list all SST files
|
||||||
|
async fn list_sst_files(test_context: &TestContext) -> HashSet<String> {
|
||||||
|
let mut sst_files = HashSet::new();
|
||||||
|
|
||||||
|
for datanode in test_context.datanodes().values() {
|
||||||
|
let region_server = datanode.region_server();
|
||||||
|
let mito = region_server.mito_engine().unwrap();
|
||||||
|
let all_files = mito
|
||||||
|
.all_ssts_from_storage()
|
||||||
|
.try_collect::<Vec<_>>()
|
||||||
|
.await
|
||||||
|
.unwrap()
|
||||||
|
.into_iter()
|
||||||
|
.map(|e| e.file_path)
|
||||||
|
.collect_vec();
|
||||||
|
sst_files.extend(all_files);
|
||||||
|
}
|
||||||
|
|
||||||
|
sst_files
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn distributed_with_gc(store_type: &StorageType) -> (TestContext, TempDirGuard) {
|
||||||
|
common_telemetry::init_default_ut_logging();
|
||||||
|
let test_name = uuid::Uuid::new_v4().to_string();
|
||||||
|
let (store_config, guard) = get_test_store_config(store_type);
|
||||||
|
|
||||||
|
let builder = GreptimeDbClusterBuilder::new(&test_name)
|
||||||
|
.await
|
||||||
|
.with_metasrv_gc_config(GcSchedulerOptions {
|
||||||
|
enable: true,
|
||||||
|
..Default::default()
|
||||||
|
})
|
||||||
|
.with_datanode_gc_config(GcConfig {
|
||||||
|
enable: true,
|
||||||
|
// set lingering time to zero for test speedup
|
||||||
|
lingering_time: Some(Duration::ZERO),
|
||||||
|
..Default::default()
|
||||||
|
})
|
||||||
|
.with_store_config(store_config);
|
||||||
|
(
|
||||||
|
TestContext::new(MockInstanceBuilder::Distributed(builder)).await,
|
||||||
|
guard,
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
#[tokio::test]
|
||||||
|
async fn test_gc_basic_different_store() {
|
||||||
|
common_telemetry::init_default_ut_logging();
|
||||||
|
let store_type = StorageType::build_storage_types_based_on_env();
|
||||||
|
for store in store_type {
|
||||||
|
if store == StorageType::File {
|
||||||
|
continue; // no point in test gc in fs storage
|
||||||
|
}
|
||||||
|
info!("Running GC test with storage type: {}", store);
|
||||||
|
test_gc_basic(&store).await;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
async fn test_gc_basic(store_type: &StorageType) {
|
||||||
|
let (test_context, _guard) = distributed_with_gc(store_type).await;
|
||||||
|
let instance = test_context.frontend();
|
||||||
|
let metasrv = test_context.metasrv();
|
||||||
|
|
||||||
|
// Step 1: Create table with append_mode to easily generate multiple files
|
||||||
|
let create_table_sql = r#"
|
||||||
|
CREATE TABLE test_gc_table (
|
||||||
|
ts TIMESTAMP TIME INDEX,
|
||||||
|
val DOUBLE,
|
||||||
|
host STRING
|
||||||
|
) WITH (append_mode = 'true')
|
||||||
|
"#;
|
||||||
|
execute_sql(&instance, create_table_sql).await;
|
||||||
|
|
||||||
|
// Step 2: Generate SST files by inserting data and flushing multiple times
|
||||||
|
for i in 0..4 {
|
||||||
|
let insert_sql = format!(
|
||||||
|
r#"
|
||||||
|
INSERT INTO test_gc_table (ts, val, host) VALUES
|
||||||
|
('2023-01-0{} 10:00:00', {}, 'host{}'),
|
||||||
|
('2023-01-0{} 11:00:00', {}, 'host{}'),
|
||||||
|
('2023-01-0{} 12:00:00', {}, 'host{}')
|
||||||
|
"#,
|
||||||
|
i + 1,
|
||||||
|
10.0 + i as f64,
|
||||||
|
i,
|
||||||
|
i + 1,
|
||||||
|
20.0 + i as f64,
|
||||||
|
i,
|
||||||
|
i + 1,
|
||||||
|
30.0 + i as f64,
|
||||||
|
i
|
||||||
|
);
|
||||||
|
execute_sql(&instance, &insert_sql).await;
|
||||||
|
|
||||||
|
// Flush the table to create SST files
|
||||||
|
let flush_sql = "ADMIN FLUSH_TABLE('test_gc_table')";
|
||||||
|
execute_sql(&instance, flush_sql).await;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Step 3: Get table information
|
||||||
|
let table = instance
|
||||||
|
.catalog_manager()
|
||||||
|
.table("greptime", "public", "test_gc_table", None)
|
||||||
|
.await
|
||||||
|
.unwrap()
|
||||||
|
.unwrap();
|
||||||
|
let table_id = table.table_info().table_id();
|
||||||
|
|
||||||
|
// List SST files before compaction (for verification)
|
||||||
|
let sst_files_before_compaction = list_sst_files(&test_context).await;
|
||||||
|
info!(
|
||||||
|
"SST files before compaction: {:?}",
|
||||||
|
sst_files_before_compaction
|
||||||
|
);
|
||||||
|
assert_eq!(sst_files_before_compaction.len(), 4); // 4 files from 4 flushes
|
||||||
|
|
||||||
|
// Step 4: Trigger compaction to create garbage SST files
|
||||||
|
let compact_sql = "ADMIN COMPACT_TABLE('test_gc_table')";
|
||||||
|
execute_sql(&instance, compact_sql).await;
|
||||||
|
|
||||||
|
// Wait for compaction to complete
|
||||||
|
tokio::time::sleep(Duration::from_secs(2)).await;
|
||||||
|
|
||||||
|
// List SST files after compaction (should have both old and new files)
|
||||||
|
let sst_files_after_compaction = list_sst_files(&test_context).await;
|
||||||
|
info!(
|
||||||
|
"SST files after compaction: {:?}",
|
||||||
|
sst_files_after_compaction
|
||||||
|
);
|
||||||
|
assert_eq!(sst_files_after_compaction.len(), 5); // 4 old + 1 new
|
||||||
|
|
||||||
|
// Step 5: Get table route information for GC procedure
|
||||||
|
let (region_routes, regions) =
|
||||||
|
get_table_route(metasrv.table_metadata_manager(), table_id).await;
|
||||||
|
|
||||||
|
// Step 6: Create and execute BatchGcProcedure
|
||||||
|
let procedure = BatchGcProcedure::new(
|
||||||
|
metasrv.mailbox().clone(),
|
||||||
|
metasrv.options().grpc.server_addr.clone(),
|
||||||
|
regions.clone(),
|
||||||
|
false, // full_file_listing
|
||||||
|
region_routes,
|
||||||
|
HashMap::new(), // related_regions (empty for this simple test)
|
||||||
|
Duration::from_secs(10), // timeout
|
||||||
|
);
|
||||||
|
|
||||||
|
// Submit the procedure to the procedure manager
|
||||||
|
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||||
|
let procedure_id = procedure_with_id.id;
|
||||||
|
|
||||||
|
let _watcher = metasrv
|
||||||
|
.procedure_manager()
|
||||||
|
.submit(procedure_with_id)
|
||||||
|
.await
|
||||||
|
.unwrap();
|
||||||
|
|
||||||
|
// Wait for the procedure to complete
|
||||||
|
wait_procedure(metasrv.procedure_manager(), procedure_id).await;
|
||||||
|
|
||||||
|
// Step 7: Verify GC results
|
||||||
|
let sst_files_after_gc = list_sst_files(&test_context).await;
|
||||||
|
info!("SST files after GC: {:?}", sst_files_after_gc);
|
||||||
|
assert_eq!(sst_files_after_gc.len(), 1); // Only the compacted file should remain after gc
|
||||||
|
|
||||||
|
// Verify that data is still accessible
|
||||||
|
let count_sql = "SELECT COUNT(*) FROM test_gc_table";
|
||||||
|
let count_output = execute_sql(&instance, count_sql).await;
|
||||||
|
let expected = r#"
|
||||||
|
+----------+
|
||||||
|
| count(*) |
|
||||||
|
+----------+
|
||||||
|
| 12 |
|
||||||
|
+----------+"#
|
||||||
|
.trim();
|
||||||
|
check_output_stream(count_output.data, expected).await;
|
||||||
|
|
||||||
|
let select_sql = "SELECT * FROM test_gc_table ORDER BY ts";
|
||||||
|
let select_output = execute_sql(&instance, select_sql).await;
|
||||||
|
let expected = r#"
|
||||||
|
+---------------------+------+-------+
|
||||||
|
| ts | val | host |
|
||||||
|
+---------------------+------+-------+
|
||||||
|
| 2023-01-01T10:00:00 | 10.0 | host0 |
|
||||||
|
| 2023-01-01T11:00:00 | 20.0 | host0 |
|
||||||
|
| 2023-01-01T12:00:00 | 30.0 | host0 |
|
||||||
|
| 2023-01-02T10:00:00 | 11.0 | host1 |
|
||||||
|
| 2023-01-02T11:00:00 | 21.0 | host1 |
|
||||||
|
| 2023-01-02T12:00:00 | 31.0 | host1 |
|
||||||
|
| 2023-01-03T10:00:00 | 12.0 | host2 |
|
||||||
|
| 2023-01-03T11:00:00 | 22.0 | host2 |
|
||||||
|
| 2023-01-03T12:00:00 | 32.0 | host2 |
|
||||||
|
| 2023-01-04T10:00:00 | 13.0 | host3 |
|
||||||
|
| 2023-01-04T11:00:00 | 23.0 | host3 |
|
||||||
|
| 2023-01-04T12:00:00 | 33.0 | host3 |
|
||||||
|
+---------------------+------+-------+"#
|
||||||
|
.trim();
|
||||||
|
check_output_stream(select_output.data, expected).await;
|
||||||
|
|
||||||
|
// TODO: Add more specific assertions once we have proper file system access
|
||||||
|
// For now, the test passes if the procedure executes without errors
|
||||||
|
info!("GC test completed successfully");
|
||||||
|
}
|
||||||
@@ -19,7 +19,7 @@ use common_wal::config::{DatanodeWalConfig, MetasrvWalConfig};
|
|||||||
|
|
||||||
use crate::cluster::GreptimeDbClusterBuilder;
|
use crate::cluster::GreptimeDbClusterBuilder;
|
||||||
use crate::tests::test_util::{
|
use crate::tests::test_util::{
|
||||||
MockInstance, MockInstanceBuilder, RebuildableMockInstance, TestContext, execute_sql,
|
MockInstanceBuilder, RebuildableMockInstance, TestContext, execute_sql,
|
||||||
};
|
};
|
||||||
|
|
||||||
pub(crate) async fn distributed_with_noop_wal() -> TestContext {
|
pub(crate) async fn distributed_with_noop_wal() -> TestContext {
|
||||||
|
|||||||
@@ -24,8 +24,8 @@ use table::table_reference::TableReference;
|
|||||||
|
|
||||||
use crate::cluster::GreptimeDbClusterBuilder;
|
use crate::cluster::GreptimeDbClusterBuilder;
|
||||||
use crate::tests::test_util::{
|
use crate::tests::test_util::{
|
||||||
MockInstance, MockInstanceBuilder, RebuildableMockInstance, TestContext, dump_kvbackend,
|
MockInstanceBuilder, RebuildableMockInstance, TestContext, dump_kvbackend, execute_sql,
|
||||||
execute_sql, restore_kvbackend, try_execute_sql, wait_procedure,
|
restore_kvbackend, try_execute_sql, wait_procedure,
|
||||||
};
|
};
|
||||||
|
|
||||||
const CREATE_MONITOR_TABLE_SQL: &str = r#"
|
const CREATE_MONITOR_TABLE_SQL: &str = r#"
|
||||||
|
|||||||
@@ -12,11 +12,13 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
|
use std::collections::HashMap;
|
||||||
use std::env;
|
use std::env;
|
||||||
use std::sync::Arc;
|
use std::sync::Arc;
|
||||||
|
|
||||||
use async_trait::async_trait;
|
use async_trait::async_trait;
|
||||||
use client::OutputData;
|
use client::OutputData;
|
||||||
|
use common_meta::DatanodeId;
|
||||||
use common_meta::kv_backend::KvBackendRef;
|
use common_meta::kv_backend::KvBackendRef;
|
||||||
use common_meta::range_stream::{DEFAULT_PAGE_SIZE, PaginationStream};
|
use common_meta::range_stream::{DEFAULT_PAGE_SIZE, PaginationStream};
|
||||||
use common_meta::rpc::KeyValue;
|
use common_meta::rpc::KeyValue;
|
||||||
@@ -30,6 +32,7 @@ use common_test_util::find_workspace_path;
|
|||||||
use common_wal::config::kafka::common::{KafkaConnectionConfig, KafkaTopicConfig};
|
use common_wal::config::kafka::common::{KafkaConnectionConfig, KafkaTopicConfig};
|
||||||
use common_wal::config::kafka::{DatanodeKafkaConfig, MetasrvKafkaConfig};
|
use common_wal::config::kafka::{DatanodeKafkaConfig, MetasrvKafkaConfig};
|
||||||
use common_wal::config::{DatanodeWalConfig, MetasrvWalConfig};
|
use common_wal::config::{DatanodeWalConfig, MetasrvWalConfig};
|
||||||
|
use datanode::datanode::Datanode;
|
||||||
use frontend::error::Result;
|
use frontend::error::Result;
|
||||||
use frontend::instance::Instance;
|
use frontend::instance::Instance;
|
||||||
use futures::TryStreamExt;
|
use futures::TryStreamExt;
|
||||||
@@ -95,6 +98,13 @@ impl MockInstanceImpl {
|
|||||||
MockInstanceImpl::Distributed(instance) => &instance.metasrv,
|
MockInstanceImpl::Distributed(instance) => &instance.metasrv,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub(crate) fn datanodes(&self) -> &HashMap<DatanodeId, Datanode> {
|
||||||
|
match self {
|
||||||
|
MockInstanceImpl::Standalone(_) => unreachable!(),
|
||||||
|
MockInstanceImpl::Distributed(instance) => &instance.datanode_instances,
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
impl MockInstance for MockInstanceImpl {
|
impl MockInstance for MockInstanceImpl {
|
||||||
@@ -185,6 +195,14 @@ impl TestContext {
|
|||||||
pub(crate) fn metasrv(&self) -> &Arc<Metasrv> {
|
pub(crate) fn metasrv(&self) -> &Arc<Metasrv> {
|
||||||
self.instance.as_ref().unwrap().metasrv()
|
self.instance.as_ref().unwrap().metasrv()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pub(crate) fn frontend(&self) -> Arc<Instance> {
|
||||||
|
self.instance.as_ref().unwrap().frontend()
|
||||||
|
}
|
||||||
|
|
||||||
|
pub(crate) fn datanodes(&self) -> &HashMap<DatanodeId, Datanode> {
|
||||||
|
self.instance.as_ref().unwrap().datanodes()
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[async_trait::async_trait]
|
#[async_trait::async_trait]
|
||||||
|
|||||||
Reference in New Issue
Block a user