feat: manual compact api (#1912)

* merge develop

* chore: merge develop

* fix: some cr commentx

* fix: cr comments
This commit is contained in:
Lei, HUANG
2023-07-11 12:00:39 +08:00
committed by GitHub
parent fc850c9988
commit a7ea3bbc16
22 changed files with 327 additions and 85 deletions

View File

@@ -256,8 +256,8 @@ fn ddl_request_type(request: &DdlRequest) -> &'static str {
Some(Expr::Alter(_)) => "ddl.alter",
Some(Expr::DropTable(_)) => "ddl.drop_table",
Some(Expr::FlushTable(_)) => "ddl.flush_table",
Some(Expr::CompactTable(_)) => "ddl.compact_table",
None => "ddl.empty",
_ => unreachable!("https://github.com/GreptimeTeam/greptimedb/pull/1912"),
}
}

View File

@@ -115,7 +115,7 @@ impl RemoteCatalogManager {
joins.push(self.initiate_schemas(node_id, backend, engine_manager, catalog_name));
}
let _ = futures::future::try_join_all(joins).await?;
futures::future::try_join_all(joins).await?;
Ok(())
}

View File

@@ -17,9 +17,9 @@ use api::v1::ddl_request::Expr as DdlExpr;
use api::v1::greptime_request::Request;
use api::v1::query_request::Query;
use api::v1::{
greptime_response, AffectedRows, AlterExpr, AuthHeader, CreateTableExpr, DdlRequest,
DeleteRequest, DropTableExpr, FlushTableExpr, GreptimeRequest, InsertRequests, PromRangeQuery,
QueryRequest, RequestHeader,
greptime_response, AffectedRows, AlterExpr, AuthHeader, CompactTableExpr, CreateTableExpr,
DdlRequest, DeleteRequest, DropTableExpr, FlushTableExpr, GreptimeRequest, InsertRequests,
PromRangeQuery, QueryRequest, RequestHeader,
};
use arrow_flight::{FlightData, Ticket};
use common_error::prelude::*;
@@ -234,6 +234,14 @@ impl Database {
.await
}
pub async fn compact_table(&self, expr: CompactTableExpr) -> Result<Output> {
let _timer = timer!(metrics::METRIC_GRPC_COMPACT_TABLE);
self.do_get(Request::Ddl(DdlRequest {
expr: Some(DdlExpr::CompactTable(expr)),
}))
.await
}
async fn do_get(&self, request: Request) -> Result<Output> {
// FIXME(paomian): should be added some labels for metrics
let _timer = timer!(metrics::METRIC_GRPC_DO_GET);

View File

@@ -22,4 +22,5 @@ pub const METRIC_GRPC_LOGICAL_PLAN: &str = "grpc.logical_plan";
pub const METRIC_GRPC_ALTER: &str = "grpc.alter";
pub const METRIC_GRPC_DROP_TABLE: &str = "grpc.drop_table";
pub const METRIC_GRPC_FLUSH_TABLE: &str = "grpc.flush_table";
pub const METRIC_GRPC_COMPACT_TABLE: &str = "grpc.compact_table";
pub const METRIC_GRPC_DO_GET: &str = "grpc.do_get";

View File

@@ -346,13 +346,6 @@ pub enum Error {
source: catalog::error::Error,
},
#[snafu(display("Failed to find table {} from catalog, source: {}", table_name, source))]
FindTable {
table_name: String,
location: Location,
source: catalog::error::Error,
},
#[snafu(display("Failed to initialize meta client, source: {}", source))]
MetaClientInit {
location: Location,
@@ -499,7 +492,6 @@ impl ErrorExt for Error {
DecodeLogicalPlan { source, .. } => source.status_code(),
NewCatalog { source, .. } | RegisterSchema { source, .. } => source.status_code(),
FindTable { source, .. } => source.status_code(),
CreateTable { source, .. } => source.status_code(),
DropTable { source, .. } => source.status_code(),
FlushTable { source, .. } => source.status_code(),

View File

@@ -198,9 +198,7 @@ impl Instance {
DdlExpr::CreateDatabase(expr) => self.handle_create_database(expr, query_ctx).await,
DdlExpr::DropTable(expr) => self.handle_drop_table(expr).await,
DdlExpr::FlushTable(expr) => self.handle_flush_table(expr).await,
Expr::CompactTable(_) => {
unreachable!("https://github.com/GreptimeTeam/greptimedb/pull/1912")
}
Expr::CompactTable(expr) => self.handle_compact_table(expr).await,
}
}
}

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use api::v1::{AlterExpr, CreateTableExpr, DropTableExpr, FlushTableExpr};
use api::v1::{AlterExpr, CompactTableExpr, CreateTableExpr, DropTableExpr, FlushTableExpr};
use common_catalog::consts::IMMUTABLE_FILE_ENGINE;
use common_catalog::format_full_table_name;
use common_grpc_expr::{alter_expr_to_request, create_expr_to_request};
@@ -20,7 +20,7 @@ use common_query::Output;
use common_telemetry::info;
use session::context::QueryContext;
use snafu::prelude::*;
use table::requests::{DropTableRequest, FlushTableRequest};
use table::requests::{CompactTableRequest, DropTableRequest, FlushTableRequest};
use crate::error::{
AlterExprToRequestSnafu, BumpTableIdSnafu, CatalogSnafu, CreateExprToRequestSnafu,
@@ -133,6 +133,25 @@ impl Instance {
.execute(SqlRequest::FlushTable(req), QueryContext::arc())
.await
}
pub(crate) async fn handle_compact_table(&self, expr: CompactTableExpr) -> Result<Output> {
let table_name = if expr.table_name.trim().is_empty() {
None
} else {
Some(expr.table_name)
};
let req = CompactTableRequest {
catalog_name: expr.catalog_name,
schema_name: expr.schema_name,
table_name,
region_number: expr.region_number,
wait: None,
};
self.sql_handler()
.execute(SqlRequest::CompactTable(req), QueryContext::arc())
.await
}
}
#[cfg(test)]

View File

@@ -31,6 +31,7 @@ use crate::error::{
use crate::instance::sql::table_idents_to_full_name;
mod alter;
mod compact_table;
mod create;
mod create_external;
mod drop_table;
@@ -44,6 +45,7 @@ pub enum SqlRequest {
Alter(AlterTableRequest),
DropTable(DropTableRequest),
FlushTable(FlushTableRequest),
CompactTable(CompactTableRequest),
}
// Handler to execute SQL except query
@@ -74,6 +76,7 @@ impl SqlHandler {
SqlRequest::Alter(req) => self.alter_table(req).await,
SqlRequest::DropTable(req) => self.drop_table(req).await,
SqlRequest::FlushTable(req) => self.flush_table(req).await,
SqlRequest::CompactTable(req) => self.compact_table(req).await,
};
if let Err(e) = &result {
error!(e; "{query_ctx}");

View File

@@ -0,0 +1,77 @@
// 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 catalog::CatalogManagerRef;
use common_query::Output;
use snafu::{OptionExt, ResultExt};
use table::requests::CompactTableRequest;
use crate::error;
use crate::error::{CatalogSnafu, Result};
use crate::sql::SqlHandler;
impl SqlHandler {
pub(crate) async fn compact_table(&self, req: CompactTableRequest) -> Result<Output> {
if let Some(table) = &req.table_name {
self.compact_table_inner(
&self.catalog_manager,
&req.catalog_name,
&req.schema_name,
table,
req.region_number,
req.wait,
)
.await?;
} else {
let all_table_names = self
.catalog_manager
.table_names(&req.catalog_name, &req.schema_name)
.await
.context(CatalogSnafu)?;
let _ = futures::future::join_all(all_table_names.iter().map(|table| {
self.compact_table_inner(
&self.catalog_manager,
&req.catalog_name,
&req.schema_name,
table,
req.region_number,
req.wait,
)
}))
.await
.into_iter()
.collect::<Result<Vec<_>>>()?;
}
Ok(Output::AffectedRows(0))
}
async fn compact_table_inner(
&self,
catalog_manager: &CatalogManagerRef,
catalog_name: &str,
schema_name: &str,
table_name: &str,
region: Option<u32>,
wait: Option<bool>,
) -> Result<()> {
catalog_manager
.table(catalog_name, schema_name, table_name)
.await
.context(CatalogSnafu)?
.context(error::TableNotFoundSnafu { table_name })?
.compact(region, wait)
.await
.context(error::FlushTableSnafu { table_name })
}
}

View File

@@ -67,7 +67,7 @@ impl SqlHandler {
catalog_manager
.table(catalog_name, schema_name, table_name)
.await
.context(error::FindTableSnafu { table_name })?
.context(CatalogSnafu)?
.context(error::TableNotFoundSnafu { table_name })?
.flush(region, wait)
.await

View File

@@ -22,8 +22,8 @@ use api::helper::ColumnDataTypeWrapper;
use api::v1::ddl_request::{Expr as DdlExpr, Expr};
use api::v1::greptime_request::Request;
use api::v1::{
column_def, AlterExpr, CreateDatabaseExpr, CreateTableExpr, DeleteRequest, DropTableExpr,
FlushTableExpr, InsertRequests, TableId,
column_def, AlterExpr, CompactTableExpr, CreateDatabaseExpr, CreateTableExpr, DeleteRequest,
DropTableExpr, FlushTableExpr, InsertRequests, TableId,
};
use async_trait::async_trait;
use catalog::helper::{SchemaKey, SchemaValue};
@@ -34,6 +34,7 @@ use client::Database;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
use common_catalog::format_full_table_name;
use common_error::prelude::BoxedError;
use common_meta::peer::Peer;
use common_meta::rpc::ddl::{DdlTask, SubmitDdlTaskRequest, SubmitDdlTaskResponse};
use common_meta::rpc::router::{
DeleteRequest as MetaDeleteRequest, Partition as MetaPartition, RouteRequest,
@@ -234,6 +235,66 @@ impl DistInstance {
table_name: TableName,
region_number: Option<RegionNumber>,
) -> Result<Output> {
let candidates = self
.find_flush_or_compaction_candidates(&table_name, region_number)
.await?;
let expr = FlushTableExpr {
catalog_name: table_name.catalog_name.clone(),
schema_name: table_name.schema_name.clone(),
table_name: table_name.table_name.clone(),
region_number,
..Default::default()
};
for candidate in candidates {
debug!("Flushing table {table_name} on Datanode {candidate:?}");
let client = self.datanode_clients.get_client(&candidate).await;
let client = Database::new(&expr.catalog_name, &expr.schema_name, client);
client
.flush_table(expr.clone())
.await
.context(RequestDatanodeSnafu)?;
}
Ok(Output::AffectedRows(0))
}
async fn compact_table(
&self,
table_name: TableName,
region_number: Option<RegionNumber>,
) -> Result<Output> {
let candidates = self
.find_flush_or_compaction_candidates(&table_name, region_number)
.await?;
let expr = CompactTableExpr {
catalog_name: table_name.catalog_name.clone(),
schema_name: table_name.schema_name.clone(),
table_name: table_name.table_name.clone(),
region_number,
};
for candidate in candidates {
debug!("Compacting table {table_name} on Datanode {candidate:?}");
let client = self.datanode_clients.get_client(&candidate).await;
let client = Database::new(&expr.catalog_name, &expr.schema_name, client);
client
.compact_table(expr.clone())
.await
.context(RequestDatanodeSnafu)?;
}
Ok(Output::AffectedRows(0))
}
async fn find_flush_or_compaction_candidates(
&self,
table_name: &TableName,
region_number: Option<RegionNumber>,
) -> Result<Vec<Peer>> {
let _ = self
.catalog_manager
.table(
@@ -255,38 +316,18 @@ impl DistInstance {
.await
.context(RequestMetaSnafu)?;
let expr = FlushTableExpr {
catalog_name: table_name.catalog_name.clone(),
schema_name: table_name.schema_name.clone(),
table_name: table_name.table_name.clone(),
region_number,
..Default::default()
};
for table_route in &route_response.table_routes {
let should_send_rpc = table_route.region_routes.iter().any(|route| {
if let Some(n) = region_number {
n == route.region.id.region_number()
} else {
true
}
});
if !should_send_rpc {
continue;
}
for datanode in table_route.find_leaders() {
debug!("Flushing table {table_name} on Datanode {datanode:?}");
let client = self.datanode_clients.get_client(&datanode).await;
let client = Database::new(&expr.catalog_name, &expr.schema_name, client);
let _ = client
.flush_table(expr.clone())
.await
.context(RequestDatanodeSnafu)?;
}
}
Ok(Output::AffectedRows(0))
let res = route_response
.table_routes
.iter()
.filter(|route| {
route.region_routes.iter().any(|r| {
let Some(n) = region_number else { return true; };
n == r.region.id.region_number()
})
})
.flat_map(|route| route.find_leaders().into_iter())
.collect::<Vec<_>>();
Ok(res)
}
async fn handle_statement(
@@ -597,8 +638,10 @@ impl GrpcQueryHandler for DistInstance {
TableName::new(&expr.catalog_name, &expr.schema_name, &expr.table_name);
self.flush_table(table_name, expr.region_number).await
}
Expr::CompactTable(_) => {
unreachable!("https://github.com/GreptimeTeam/greptimedb/pull/1912")
Expr::CompactTable(expr) => {
let table_name =
TableName::new(&expr.catalog_name, &expr.schema_name, &expr.table_name);
self.compact_table(table_name, expr.region_number).await
}
}
}

View File

@@ -33,8 +33,9 @@ use object_store::ObjectStore;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::manifest::{self, Manifest, ManifestVersion, MetaActionIterator};
use store_api::storage::{
AddColumn, AlterOperation, AlterRequest, ChunkReader, FlushContext, FlushReason, ReadContext,
Region, RegionMeta, RegionNumber, ScanRequest, SchemaRef, Snapshot, WriteContext, WriteRequest,
AddColumn, AlterOperation, AlterRequest, ChunkReader, CompactContext, FlushContext,
FlushReason, ReadContext, Region, RegionMeta, RegionNumber, ScanRequest, SchemaRef, Snapshot,
WriteContext, WriteRequest,
};
use table::error::{
InvalidTableSnafu, RegionSchemaMismatchSnafu, Result as TableResult, TableOperationSnafu,
@@ -332,6 +333,33 @@ impl<R: Region> Table for MitoTable<R> {
Ok(())
}
async fn compact(
&self,
region_number: Option<RegionNumber>,
wait: Option<bool>,
) -> TableResult<()> {
let compact_ctx = wait.map(|wait| CompactContext { wait }).unwrap_or_default();
let regions = self.regions.load();
if let Some(region_number) = region_number {
if let Some(region) = regions.get(&region_number) {
region
.compact(&compact_ctx)
.await
.map_err(BoxedError::new)
.context(table_error::TableOperationSnafu)?;
}
} else {
let _ = futures::future::try_join_all(
regions.values().map(|region| region.compact(&compact_ctx)),
)
.await
.map_err(BoxedError::new)
.context(TableOperationSnafu)?;
}
Ok(())
}
fn region_stats(&self) -> TableResult<Vec<RegionStat>> {
let regions = self.regions.load();

View File

@@ -26,9 +26,10 @@ use datatypes::schema::{ColumnSchema, Schema};
use storage::metadata::{RegionMetaImpl, RegionMetadata};
use storage::write_batch::WriteBatch;
use store_api::storage::{
AlterRequest, Chunk, ChunkReader, CloseOptions, CreateOptions, EngineContext, FlushContext,
GetRequest, GetResponse, OpenOptions, ReadContext, Region, RegionDescriptor, RegionId,
ScanRequest, ScanResponse, SchemaRef, Snapshot, StorageEngine, WriteContext, WriteResponse,
AlterRequest, Chunk, ChunkReader, CloseOptions, CompactContext, CreateOptions, EngineContext,
FlushContext, GetRequest, GetResponse, OpenOptions, ReadContext, Region, RegionDescriptor,
RegionId, ScanRequest, ScanResponse, SchemaRef, Snapshot, StorageEngine, WriteContext,
WriteResponse,
};
pub type Result<T> = std::result::Result<T, MockError>;
@@ -204,6 +205,10 @@ impl Region for MockRegion {
async fn flush(&self, _ctx: &FlushContext) -> Result<()> {
unimplemented!()
}
async fn compact(&self, _ctx: &CompactContext) -> std::result::Result<(), Self::Error> {
unimplemented!()
}
}
impl MockRegionInner {

View File

@@ -64,7 +64,7 @@ use self::influxdb::{influxdb_health, influxdb_ping, influxdb_write_v1, influxdb
use crate::auth::UserProviderRef;
use crate::configurator::ConfiguratorRef;
use crate::error::{AlreadyStartedSnafu, Result, StartHttpSnafu};
use crate::http::admin::flush;
use crate::http::admin::{compact, flush};
use crate::metrics::{
METRIC_HTTP_REQUESTS_ELAPSED, METRIC_HTTP_REQUESTS_TOTAL, METRIC_METHOD_LABEL,
METRIC_PATH_LABEL, METRIC_STATUS_LABEL,
@@ -650,6 +650,7 @@ impl HttpServer {
fn route_admin<S>(&self, grpc_handler: ServerGrpcQueryHandlerRef) -> Router<S> {
Router::new()
.route("/flush", routing::post(flush))
.route("/compact", routing::post(compact))
.with_state(grpc_handler)
}

View File

@@ -16,9 +16,10 @@ use std::collections::HashMap;
use api::v1::ddl_request::Expr;
use api::v1::greptime_request::Request;
use api::v1::{DdlRequest, FlushTableExpr};
use api::v1::{CompactTableExpr, DdlRequest, FlushTableExpr};
use axum::extract::{Query, RawBody, State};
use axum::http::StatusCode;
use common_catalog::consts::DEFAULT_CATALOG_NAME;
use session::context::QueryContext;
use snafu::OptionExt;
@@ -35,7 +36,7 @@ pub async fn flush(
let catalog_name = params
.get("catalog")
.cloned()
.unwrap_or("greptime".to_string());
.unwrap_or(DEFAULT_CATALOG_NAME.to_string());
let schema_name = params
.get("db")
.cloned()
@@ -63,6 +64,46 @@ pub async fn flush(
})),
});
let _ = grpc_handler.do_query(request, QueryContext::arc()).await?;
grpc_handler.do_query(request, QueryContext::arc()).await?;
Ok((StatusCode::NO_CONTENT, ()))
}
#[axum_macros::debug_handler]
pub async fn compact(
State(grpc_handler): State<ServerGrpcQueryHandlerRef>,
Query(params): Query<HashMap<String, String>>,
RawBody(_): RawBody,
) -> Result<(StatusCode, ())> {
let catalog_name = params
.get("catalog")
.cloned()
.unwrap_or(DEFAULT_CATALOG_NAME.to_string());
let schema_name = params
.get("db")
.cloned()
.context(error::InvalidFlushArgumentSnafu {
err_msg: "db is not present",
})?;
// if table name is not present, flush all tables inside schema
let table_name = params.get("table").cloned().unwrap_or_default();
let region_number: Option<u32> = params
.get("region")
.map(|v| v.parse())
.transpose()
.ok()
.flatten();
let request = Request::Ddl(DdlRequest {
expr: Some(Expr::CompactTable(CompactTableExpr {
catalog_name: catalog_name.clone(),
schema_name: schema_name.clone(),
table_name: table_name.clone(),
region_number,
})),
});
grpc_handler.do_query(request, QueryContext::arc()).await?;
Ok((StatusCode::NO_CONTENT, ()))
}

View File

@@ -20,14 +20,14 @@ use common_telemetry::{debug, error, info, timer};
use itertools::Itertools;
use snafu::ResultExt;
use store_api::logstore::LogStore;
use store_api::storage::RegionId;
use store_api::storage::{CompactContext, RegionId};
use crate::compaction::writer::build_sst_reader;
use crate::error;
use crate::error::Result;
use crate::manifest::action::RegionEdit;
use crate::manifest::region::RegionManifest;
use crate::region::{CompactContext, RegionWriterRef, SharedDataRef, WriterCompactRequest};
use crate::region::{RegionWriterRef, SharedDataRef, WriterCompactRequest};
use crate::schema::RegionSchemaRef;
use crate::sst::{
AccessLayerRef, FileHandle, FileId, FileMeta, Level, Source, SstInfo, WriteOptions,

View File

@@ -32,8 +32,8 @@ use store_api::manifest::{
self, Manifest, ManifestLogStorage, ManifestVersion, MetaActionIterator,
};
use store_api::storage::{
AlterRequest, CloseContext, CompactionStrategy, FlushContext, FlushReason, OpenOptions,
ReadContext, Region, RegionId, SequenceNumber, WriteContext, WriteResponse,
AlterRequest, CloseContext, CompactContext, CompactionStrategy, FlushContext, FlushReason,
OpenOptions, ReadContext, Region, RegionId, SequenceNumber, WriteContext, WriteResponse,
};
use crate::compaction::{
@@ -150,6 +150,10 @@ impl<S: LogStore> Region for RegionImpl<S> {
async fn flush(&self, ctx: &FlushContext) -> Result<()> {
self.inner.flush(ctx).await
}
async fn compact(&self, ctx: &CompactContext) -> std::result::Result<(), Self::Error> {
self.inner.compact(ctx).await
}
}
/// Storage related config for region.
@@ -174,18 +178,6 @@ pub struct StoreConfig<S: LogStore> {
pub type RecoveredMetadata = (SequenceNumber, (ManifestVersion, RawRegionMetadata));
pub type RecoveredMetadataMap = BTreeMap<SequenceNumber, (ManifestVersion, RawRegionMetadata)>;
#[derive(Debug)]
pub struct CompactContext {
/// Whether to wait the compaction result.
pub wait: bool,
}
impl Default for CompactContext {
fn default() -> CompactContext {
CompactContext { wait: true }
}
}
impl<S: LogStore> RegionImpl<S> {
/// Create a new region and also persist the region metadata to manifest.
///
@@ -557,7 +549,7 @@ impl<S: LogStore> RegionImpl<S> {
}
/// Compact the region manually.
pub async fn compact(&self, ctx: CompactContext) -> Result<()> {
pub async fn compact(&self, ctx: &CompactContext) -> Result<()> {
self.inner.compact(ctx).await
}
@@ -765,7 +757,7 @@ impl<S: LogStore> RegionInner<S> {
}
/// Compact the region manually.
async fn compact(&self, compact_ctx: CompactContext) -> Result<()> {
async fn compact(&self, compact_ctx: &CompactContext) -> Result<()> {
self.writer
.compact(WriterCompactRequest {
shared_data: self.shared.clone(),
@@ -773,7 +765,7 @@ impl<S: LogStore> RegionInner<S> {
manifest: self.manifest.clone(),
wal: self.wal.clone(),
region_writer: self.writer.clone(),
compact_ctx,
compact_ctx: *compact_ctx,
})
.await
}

View File

@@ -223,7 +223,7 @@ impl CompactionTester {
// Trigger compaction and wait until it is done.
self.base()
.region
.compact(CompactContext::default())
.compact(&CompactContext::default())
.await
.unwrap();
}

View File

@@ -37,7 +37,9 @@ pub use self::engine::{
TwcsOptions,
};
pub use self::metadata::RegionMeta;
pub use self::region::{CloseContext, FlushContext, FlushReason, Region, RegionStat, WriteContext};
pub use self::region::{
CloseContext, CompactContext, FlushContext, FlushReason, Region, RegionStat, WriteContext,
};
pub use self::requests::{
AddColumn, AlterOperation, AlterRequest, GetRequest, ScanRequest, WriteRequest,
};

View File

@@ -86,6 +86,8 @@ pub trait Region: Send + Sync + Clone + std::fmt::Debug + 'static {
/// Flush memtable of the region to disk.
async fn flush(&self, ctx: &FlushContext) -> Result<(), Self::Error>;
async fn compact(&self, ctx: &CompactContext) -> Result<(), Self::Error>;
}
#[derive(Default, Debug)]
@@ -132,6 +134,18 @@ impl Default for FlushContext {
}
}
#[derive(Debug, Copy, Clone)]
pub struct CompactContext {
/// Whether to wait the compaction result.
pub wait: bool,
}
impl Default for CompactContext {
fn default() -> CompactContext {
CompactContext { wait: true }
}
}
/// Reason of flush operation.
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub enum FlushReason {

View File

@@ -285,6 +285,16 @@ pub struct FlushTableRequest {
pub wait: Option<bool>,
}
#[derive(Debug, Clone, Default)]
pub struct CompactTableRequest {
pub catalog_name: String,
pub schema_name: String,
pub table_name: Option<String>,
pub region_number: Option<RegionNumber>,
/// Wait until the compaction is done.
pub wait: Option<bool>,
}
#[macro_export]
macro_rules! meter_insert_request {
($req: expr) => {

View File

@@ -122,6 +122,14 @@ pub trait Table: Send + Sync {
fn statistics(&self) -> Option<TableStatistics> {
None
}
async fn compact(&self, region_number: Option<RegionNumber>, wait: Option<bool>) -> Result<()> {
let _ = (region_number, wait);
UnsupportedSnafu {
operation: "COMPACTION",
}
.fail()?
}
}
pub type TableRef = Arc<dyn Table>;