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:
discord9
2025-12-01 12:10:42 +08:00
committed by GitHub
parent 5d634aeba0
commit ee76d50569
13 changed files with 334 additions and 55 deletions

View File

@@ -59,8 +59,10 @@ use hyper_util::rt::TokioIo;
use meta_client::client::MetaClientBuilder;
use meta_srv::cluster::MetaPeerClientRef;
use meta_srv::discovery;
use meta_srv::gc::GcSchedulerOptions;
use meta_srv::metasrv::{Metasrv, MetasrvOptions, SelectorRef};
use meta_srv::mocks::MockInfo;
use mito2::gc::GcConfig;
use object_store::config::ObjectStoreConfig;
use rand::Rng;
use servers::grpc::GrpcOptions;
@@ -103,6 +105,8 @@ pub struct GreptimeDbClusterBuilder {
datanodes: Option<u32>,
datanode_wal_config: DatanodeWalConfig,
metasrv_wal_config: MetasrvWalConfig,
datanode_gc_config: GcConfig,
metasrv_gc_config: GcSchedulerOptions,
shared_home_dir: Option<Arc<TempDir>>,
meta_selector: Option<SelectorRef>,
}
@@ -134,6 +138,8 @@ impl GreptimeDbClusterBuilder {
datanodes: None,
datanode_wal_config: DatanodeWalConfig::default(),
metasrv_wal_config: MetasrvWalConfig::default(),
datanode_gc_config: GcConfig::default(),
metasrv_gc_config: GcSchedulerOptions::default(),
shared_home_dir: None,
meta_selector: None,
}
@@ -169,6 +175,17 @@ impl GreptimeDbClusterBuilder {
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]
pub fn with_shared_home_dir(mut self, shared_home_dir: Arc<TempDir>) -> Self {
self.shared_home_dir = Some(shared_home_dir);
@@ -205,6 +222,7 @@ impl GreptimeDbClusterBuilder {
server_addr: "127.0.0.1:3002".to_string(),
..Default::default()
},
gc: self.metasrv_gc_config.clone(),
..Default::default()
};
@@ -279,6 +297,7 @@ impl GreptimeDbClusterBuilder {
vec![],
home_dir,
self.datanode_wal_config.clone(),
self.datanode_gc_config.clone(),
)
} else {
let (opts, guard) = create_tmp_dir_and_datanode_opts(
@@ -286,6 +305,7 @@ impl GreptimeDbClusterBuilder {
self.store_providers.clone().unwrap_or_default(),
&format!("{}-dn-{}", self.cluster_name, datanode_id),
self.datanode_wal_config.clone(),
self.datanode_gc_config.clone(),
);
guards.push(guard);

View File

@@ -309,6 +309,7 @@ impl GreptimeDbStandaloneBuilder {
store_types,
&self.instance_name,
self.datanode_wal_config.clone(),
Default::default(),
);
let kv_backend_config = KvBackendConfig::default();

View File

@@ -32,6 +32,7 @@ use common_wal::config::DatanodeWalConfig;
use datanode::config::{DatanodeOptions, StorageConfig};
use frontend::instance::Instance;
use frontend::service_config::{MysqlOptions, PostgresOptions};
use mito2::gc::GcConfig;
use object_store::config::{
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(),
bucket: env::var("GT_S3_BUCKET").unwrap(),
region: Some(env::var("GT_S3_REGION").unwrap()),
endpoint: env::var("GT_S3_ENDPOINT_URL").ok(),
..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(),
credential_path: env::var("GT_GCS_CREDENTIAL_PATH").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()
};
@@ -297,6 +299,7 @@ pub fn create_tmp_dir_and_datanode_opts(
store_provider_types: Vec<StorageType>,
name: &str,
wal_config: DatanodeWalConfig,
gc_config: GcConfig,
) -> (DatanodeOptions, TestGuard) {
let home_tmp_dir = create_temp_dir(&format!("gt_data_{name}"));
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);
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,
@@ -330,7 +339,18 @@ pub(crate) fn create_datanode_opts(
providers: Vec<ObjectStoreConfig>,
home_dir: String,
wal_config: DatanodeWalConfig,
gc_config: GcConfig,
) -> 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 {
node_id: Some(0),
require_lease_before_startup: true,
@@ -343,6 +363,7 @@ pub(crate) fn create_datanode_opts(
.with_bind_addr(PEER_PLACEHOLDER_ADDR)
.with_server_addr(PEER_PLACEHOLDER_ADDR),
wal: wal_config,
region_engine,
..Default::default()
}
}

View File

@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
mod gc;
mod instance_kafka_wal_test;
mod instance_noop_wal_test;
mod instance_test;

View 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");
}

View File

@@ -19,7 +19,7 @@ use common_wal::config::{DatanodeWalConfig, MetasrvWalConfig};
use crate::cluster::GreptimeDbClusterBuilder;
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 {

View File

@@ -24,8 +24,8 @@ use table::table_reference::TableReference;
use crate::cluster::GreptimeDbClusterBuilder;
use crate::tests::test_util::{
MockInstance, MockInstanceBuilder, RebuildableMockInstance, TestContext, dump_kvbackend,
execute_sql, restore_kvbackend, try_execute_sql, wait_procedure,
MockInstanceBuilder, RebuildableMockInstance, TestContext, dump_kvbackend, execute_sql,
restore_kvbackend, try_execute_sql, wait_procedure,
};
const CREATE_MONITOR_TABLE_SQL: &str = r#"

View File

@@ -12,11 +12,13 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashMap;
use std::env;
use std::sync::Arc;
use async_trait::async_trait;
use client::OutputData;
use common_meta::DatanodeId;
use common_meta::kv_backend::KvBackendRef;
use common_meta::range_stream::{DEFAULT_PAGE_SIZE, PaginationStream};
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::{DatanodeKafkaConfig, MetasrvKafkaConfig};
use common_wal::config::{DatanodeWalConfig, MetasrvWalConfig};
use datanode::datanode::Datanode;
use frontend::error::Result;
use frontend::instance::Instance;
use futures::TryStreamExt;
@@ -95,6 +98,13 @@ impl MockInstanceImpl {
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 {
@@ -185,6 +195,14 @@ impl TestContext {
pub(crate) fn metasrv(&self) -> &Arc<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]