mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-14 01:02:55 +00:00
Compare commits
8 Commits
prune-mani
...
main
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
4b3bd7317b | ||
|
|
279908984d | ||
|
|
a56a00224f | ||
|
|
6487f14f70 | ||
|
|
45b4067721 | ||
|
|
0c373062c2 | ||
|
|
567d3e66e9 | ||
|
|
63284a5081 |
3
Cargo.lock
generated
3
Cargo.lock
generated
@@ -5467,7 +5467,7 @@ dependencies = [
|
||||
[[package]]
|
||||
name = "greptime-proto"
|
||||
version = "0.1.0"
|
||||
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=0e316b86d765e4718d6f0ca77b1ad179f222b822#0e316b86d765e4718d6f0ca77b1ad179f222b822"
|
||||
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=58aeee49267fb1eafa6f9123f9d0c47dd0f62722#58aeee49267fb1eafa6f9123f9d0c47dd0f62722"
|
||||
dependencies = [
|
||||
"prost 0.13.5",
|
||||
"prost-types 0.13.5",
|
||||
@@ -12512,6 +12512,7 @@ dependencies = [
|
||||
"servers",
|
||||
"snafu 0.8.6",
|
||||
"store-api",
|
||||
"table",
|
||||
"tokio",
|
||||
]
|
||||
|
||||
|
||||
@@ -151,7 +151,7 @@ etcd-client = { version = "0.16.1", features = [
|
||||
fst = "0.4.7"
|
||||
futures = "0.3"
|
||||
futures-util = "0.3"
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "0e316b86d765e4718d6f0ca77b1ad179f222b822" }
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "58aeee49267fb1eafa6f9123f9d0c47dd0f62722" }
|
||||
hex = "0.4"
|
||||
http = "1"
|
||||
humantime = "2.1"
|
||||
|
||||
@@ -64,8 +64,8 @@ use plugins::frontend::context::{
|
||||
use plugins::standalone::context::DdlManagerConfigureContext;
|
||||
use servers::tls::{TlsMode, TlsOption, merge_tls_option};
|
||||
use snafu::ResultExt;
|
||||
use standalone::StandaloneInformationExtension;
|
||||
use standalone::options::StandaloneOptions;
|
||||
use standalone::{StandaloneInformationExtension, StandaloneRepartitionProcedureFactory};
|
||||
use tracing_appender::non_blocking::WorkerGuard;
|
||||
|
||||
use crate::error::{OtherSnafu, Result, StartFlownodeSnafu};
|
||||
@@ -509,8 +509,13 @@ impl StartCommand {
|
||||
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
|
||||
};
|
||||
|
||||
let ddl_manager = DdlManager::try_new(ddl_context, procedure_manager.clone(), true)
|
||||
.context(error::InitDdlManagerSnafu)?;
|
||||
let ddl_manager = DdlManager::try_new(
|
||||
ddl_context,
|
||||
procedure_manager.clone(),
|
||||
Arc::new(StandaloneRepartitionProcedureFactory),
|
||||
true,
|
||||
)
|
||||
.context(error::InitDdlManagerSnafu)?;
|
||||
|
||||
let ddl_manager = if let Some(configurator) =
|
||||
plugins.get::<DdlManagerConfiguratorRef<DdlManagerConfigureContext>>()
|
||||
|
||||
@@ -34,7 +34,7 @@ use table::requests::{
|
||||
};
|
||||
|
||||
use crate::error::{
|
||||
ColumnNotFoundSnafu, InvalidColumnDefSnafu, InvalidIndexOptionSnafu,
|
||||
self, ColumnNotFoundSnafu, InvalidColumnDefSnafu, InvalidIndexOptionSnafu,
|
||||
InvalidSetFulltextOptionRequestSnafu, InvalidSetSkippingIndexOptionRequestSnafu,
|
||||
InvalidSetTableOptionRequestSnafu, InvalidUnsetTableOptionRequestSnafu,
|
||||
MissingAlterIndexOptionSnafu, MissingFieldSnafu, MissingTableMetaSnafu,
|
||||
@@ -251,6 +251,10 @@ pub fn alter_expr_to_request(
|
||||
.collect::<Result<Vec<_>>>()?;
|
||||
AlterKind::SetDefaults { defaults }
|
||||
}
|
||||
Kind::Repartition(_) => error::UnexpectedSnafu {
|
||||
err_msg: "Repartition operation should be handled through DdlManager and not converted to AlterTableRequest",
|
||||
}
|
||||
.fail()?,
|
||||
};
|
||||
|
||||
let request = AlterTableRequest {
|
||||
|
||||
@@ -161,6 +161,13 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Unexpected: {err_msg}"))]
|
||||
Unexpected {
|
||||
err_msg: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -188,6 +195,7 @@ impl ErrorExt for Error {
|
||||
Error::ColumnNotFound { .. } => StatusCode::TableColumnNotFound,
|
||||
Error::SqlCommon { source, .. } => source.status_code(),
|
||||
Error::MissingTableMeta { .. } => StatusCode::Unexpected,
|
||||
Error::Unexpected { .. } => StatusCode::Unexpected,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -22,7 +22,7 @@ use snafu::OptionExt;
|
||||
use table::metadata::RawTableInfo;
|
||||
|
||||
use crate::ddl::alter_table::AlterTableProcedure;
|
||||
use crate::error::{InvalidProtoMsgSnafu, Result};
|
||||
use crate::error::{self, InvalidProtoMsgSnafu, Result};
|
||||
|
||||
impl AlterTableProcedure {
|
||||
/// Makes alter kind proto that all regions can reuse.
|
||||
@@ -112,6 +112,10 @@ fn create_proto_alter_kind(
|
||||
Kind::UnsetIndexes(v) => Ok(Some(alter_request::Kind::UnsetIndexes(v.clone()))),
|
||||
Kind::DropDefaults(v) => Ok(Some(alter_request::Kind::DropDefaults(v.clone()))),
|
||||
Kind::SetDefaults(v) => Ok(Some(alter_request::Kind::SetDefaults(v.clone()))),
|
||||
Kind::Repartition(_) => error::UnexpectedSnafu {
|
||||
err_msg: "Repartition operation should be handled through DdlManager and not converted to AlterTableRequest",
|
||||
}
|
||||
.fail()?,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -161,7 +161,18 @@ impl TableMetadataAllocator {
|
||||
})
|
||||
}
|
||||
|
||||
/// Returns the table id allocator.
|
||||
pub fn table_id_allocator(&self) -> ResourceIdAllocatorRef {
|
||||
self.table_id_allocator.clone()
|
||||
}
|
||||
|
||||
/// Returns the wal options allocator.
|
||||
pub fn wal_options_allocator(&self) -> WalOptionsAllocatorRef {
|
||||
self.wal_options_allocator.clone()
|
||||
}
|
||||
|
||||
/// Returns the region routes allocator.
|
||||
pub fn region_routes_allocator(&self) -> RegionRoutesAllocatorRef {
|
||||
self.region_routes_allocator.clone()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -14,15 +14,19 @@
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use api::v1::Repartition;
|
||||
use api::v1::alter_table_expr::Kind;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_procedure::{
|
||||
BoxedProcedureLoader, Output, ProcedureId, ProcedureManagerRef, ProcedureWithId, watcher,
|
||||
BoxedProcedure, BoxedProcedureLoader, Output, ProcedureId, ProcedureManagerRef,
|
||||
ProcedureWithId, watcher,
|
||||
};
|
||||
use common_telemetry::tracing_context::{FutureExt, TracingContext};
|
||||
use common_telemetry::{debug, info, tracing};
|
||||
use derive_builder::Builder;
|
||||
use snafu::{OptionExt, ResultExt, ensure};
|
||||
use store_api::storage::TableId;
|
||||
use table::table_name::TableName;
|
||||
|
||||
use crate::ddl::alter_database::AlterDatabaseProcedure;
|
||||
use crate::ddl::alter_logical_tables::AlterLogicalTablesProcedure;
|
||||
@@ -40,7 +44,8 @@ use crate::ddl::drop_view::DropViewProcedure;
|
||||
use crate::ddl::truncate_table::TruncateTableProcedure;
|
||||
use crate::ddl::{DdlContext, utils};
|
||||
use crate::error::{
|
||||
EmptyDdlTasksSnafu, ProcedureOutputSnafu, RegisterProcedureLoaderSnafu, Result,
|
||||
CreateRepartitionProcedureSnafu, EmptyDdlTasksSnafu, ProcedureOutputSnafu,
|
||||
RegisterProcedureLoaderSnafu, RegisterRepartitionProcedureLoaderSnafu, Result,
|
||||
SubmitProcedureSnafu, TableInfoNotFoundSnafu, TableNotFoundSnafu, TableRouteNotFoundSnafu,
|
||||
UnexpectedLogicalRouteTableSnafu, WaitProcedureSnafu,
|
||||
};
|
||||
@@ -90,6 +95,7 @@ pub type BoxedProcedureLoaderFactory = dyn Fn(DdlContext) -> BoxedProcedureLoade
|
||||
pub struct DdlManager {
|
||||
ddl_context: DdlContext,
|
||||
procedure_manager: ProcedureManagerRef,
|
||||
repartition_procedure_factory: RepartitionProcedureFactoryRef,
|
||||
#[cfg(feature = "enterprise")]
|
||||
trigger_ddl_manager: Option<TriggerDdlManagerRef>,
|
||||
}
|
||||
@@ -143,16 +149,37 @@ macro_rules! procedure_loader {
|
||||
};
|
||||
}
|
||||
|
||||
pub type RepartitionProcedureFactoryRef = Arc<dyn RepartitionProcedureFactory>;
|
||||
|
||||
pub trait RepartitionProcedureFactory: Send + Sync {
|
||||
fn create(
|
||||
&self,
|
||||
ddl_ctx: &DdlContext,
|
||||
table_name: TableName,
|
||||
table_id: TableId,
|
||||
from_exprs: Vec<String>,
|
||||
to_exprs: Vec<String>,
|
||||
) -> std::result::Result<BoxedProcedure, BoxedError>;
|
||||
|
||||
fn register_loaders(
|
||||
&self,
|
||||
ddl_ctx: &DdlContext,
|
||||
procedure_manager: &ProcedureManagerRef,
|
||||
) -> std::result::Result<(), BoxedError>;
|
||||
}
|
||||
|
||||
impl DdlManager {
|
||||
/// Returns a new [DdlManager] with all Ddl [BoxedProcedureLoader](common_procedure::procedure::BoxedProcedureLoader)s registered.
|
||||
pub fn try_new(
|
||||
ddl_context: DdlContext,
|
||||
procedure_manager: ProcedureManagerRef,
|
||||
repartition_procedure_factory: RepartitionProcedureFactoryRef,
|
||||
register_loaders: bool,
|
||||
) -> Result<Self> {
|
||||
let manager = Self {
|
||||
ddl_context,
|
||||
procedure_manager,
|
||||
repartition_procedure_factory,
|
||||
#[cfg(feature = "enterprise")]
|
||||
trigger_ddl_manager: None,
|
||||
};
|
||||
@@ -204,9 +231,63 @@ impl DdlManager {
|
||||
.context(RegisterProcedureLoaderSnafu { type_name })?;
|
||||
}
|
||||
|
||||
self.repartition_procedure_factory
|
||||
.register_loaders(&self.ddl_context, &self.procedure_manager)
|
||||
.context(RegisterRepartitionProcedureLoaderSnafu)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Submits a repartition procedure for the specified table.
|
||||
///
|
||||
/// This creates a repartition procedure using the provided `table_id`,
|
||||
/// `table_name`, and `Repartition` configuration, and then either executes it
|
||||
/// to completion or just submits it for asynchronous execution.
|
||||
///
|
||||
/// The `Repartition` argument contains the original (`from_partition_exprs`)
|
||||
/// and target (`into_partition_exprs`) partition expressions that define how
|
||||
/// the table should be repartitioned.
|
||||
///
|
||||
/// The `wait` flag controls whether this method waits for the repartition
|
||||
/// procedure to finish:
|
||||
/// - If `wait` is `true`, the procedure is executed and this method awaits
|
||||
/// its completion, returning both the generated `ProcedureId` and the
|
||||
/// final `Output` of the procedure.
|
||||
/// - If `wait` is `false`, the procedure is only submitted to the procedure
|
||||
/// manager for asynchronous execution, and this method returns the
|
||||
/// `ProcedureId` along with `None` as the output.
|
||||
async fn submit_repartition_task(
|
||||
&self,
|
||||
table_id: TableId,
|
||||
table_name: TableName,
|
||||
Repartition {
|
||||
from_partition_exprs,
|
||||
into_partition_exprs,
|
||||
wait,
|
||||
}: Repartition,
|
||||
) -> Result<(ProcedureId, Option<Output>)> {
|
||||
let context = self.create_context();
|
||||
|
||||
let procedure = self
|
||||
.repartition_procedure_factory
|
||||
.create(
|
||||
&context,
|
||||
table_name,
|
||||
table_id,
|
||||
from_partition_exprs,
|
||||
into_partition_exprs,
|
||||
)
|
||||
.context(CreateRepartitionProcedureSnafu)?;
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
if wait {
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
} else {
|
||||
self.submit_procedure(procedure_with_id)
|
||||
.await
|
||||
.map(|p| (p, None))
|
||||
}
|
||||
}
|
||||
|
||||
/// Submits and executes an alter table task.
|
||||
#[tracing::instrument(skip_all)]
|
||||
pub async fn submit_alter_table_task(
|
||||
@@ -214,13 +295,28 @@ impl DdlManager {
|
||||
table_id: TableId,
|
||||
alter_table_task: AlterTableTask,
|
||||
) -> Result<(ProcedureId, Option<Output>)> {
|
||||
let context = self.create_context();
|
||||
// make alter_table_task mutable so we can call .take() on its field
|
||||
let mut alter_table_task = alter_table_task;
|
||||
if let Some(Kind::Repartition(_)) = alter_table_task.alter_table.kind.as_ref()
|
||||
&& let Kind::Repartition(repartition) =
|
||||
alter_table_task.alter_table.kind.take().unwrap()
|
||||
{
|
||||
let table_name = TableName::new(
|
||||
alter_table_task.alter_table.catalog_name,
|
||||
alter_table_task.alter_table.schema_name,
|
||||
alter_table_task.alter_table.table_name,
|
||||
);
|
||||
return self
|
||||
.submit_repartition_task(table_id, table_name, repartition)
|
||||
.await;
|
||||
}
|
||||
|
||||
let context = self.create_context();
|
||||
let procedure = AlterTableProcedure::new(table_id, alter_table_task, context)?;
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a create table task.
|
||||
@@ -235,7 +331,7 @@ impl DdlManager {
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a `[CreateViewTask]`.
|
||||
@@ -250,7 +346,7 @@ impl DdlManager {
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a create multiple logical table tasks.
|
||||
@@ -267,7 +363,7 @@ impl DdlManager {
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes alter multiple table tasks.
|
||||
@@ -284,7 +380,7 @@ impl DdlManager {
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a drop table task.
|
||||
@@ -299,7 +395,7 @@ impl DdlManager {
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a create database task.
|
||||
@@ -318,7 +414,7 @@ impl DdlManager {
|
||||
CreateDatabaseProcedure::new(catalog, schema, create_if_not_exists, options, context);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a drop table task.
|
||||
@@ -335,7 +431,7 @@ impl DdlManager {
|
||||
let procedure = DropDatabaseProcedure::new(catalog, schema, drop_if_exists, context);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
pub async fn submit_alter_database(
|
||||
@@ -346,7 +442,7 @@ impl DdlManager {
|
||||
let procedure = AlterDatabaseProcedure::new(alter_database_task, context)?;
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a create flow task.
|
||||
@@ -360,7 +456,7 @@ impl DdlManager {
|
||||
let procedure = CreateFlowProcedure::new(create_flow, query_context, context);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a drop flow task.
|
||||
@@ -373,7 +469,7 @@ impl DdlManager {
|
||||
let procedure = DropFlowProcedure::new(drop_flow, context);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a drop view task.
|
||||
@@ -386,7 +482,7 @@ impl DdlManager {
|
||||
let procedure = DropViewProcedure::new(drop_view, context);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a truncate table task.
|
||||
@@ -407,7 +503,7 @@ impl DdlManager {
|
||||
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
/// Submits and executes a comment on task.
|
||||
@@ -420,10 +516,11 @@ impl DdlManager {
|
||||
let procedure = CommentOnProcedure::new(comment_on_task, context);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
|
||||
self.submit_procedure(procedure_with_id).await
|
||||
self.execute_procedure_and_wait(procedure_with_id).await
|
||||
}
|
||||
|
||||
async fn submit_procedure(
|
||||
/// Executes a procedure and waits for the result.
|
||||
async fn execute_procedure_and_wait(
|
||||
&self,
|
||||
procedure_with_id: ProcedureWithId,
|
||||
) -> Result<(ProcedureId, Option<Output>)> {
|
||||
@@ -442,6 +539,18 @@ impl DdlManager {
|
||||
Ok((procedure_id, output))
|
||||
}
|
||||
|
||||
/// Submits a procedure and returns the procedure id.
|
||||
async fn submit_procedure(&self, procedure_with_id: ProcedureWithId) -> Result<ProcedureId> {
|
||||
let procedure_id = procedure_with_id.id;
|
||||
let _ = self
|
||||
.procedure_manager
|
||||
.submit(procedure_with_id)
|
||||
.await
|
||||
.context(SubmitProcedureSnafu)?;
|
||||
|
||||
Ok(procedure_id)
|
||||
}
|
||||
|
||||
pub async fn submit_ddl_task(
|
||||
&self,
|
||||
ctx: &ExecutorContext,
|
||||
@@ -947,8 +1056,12 @@ async fn handle_comment_on_task(
|
||||
mod tests {
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_procedure::local::LocalManager;
|
||||
use common_procedure::test_util::InMemoryPoisonStore;
|
||||
use common_procedure::{BoxedProcedure, ProcedureManagerRef};
|
||||
use store_api::storage::TableId;
|
||||
use table::table_name::TableName;
|
||||
|
||||
use super::DdlManager;
|
||||
use crate::cache_invalidator::DummyCacheInvalidator;
|
||||
@@ -959,6 +1072,7 @@ mod tests {
|
||||
use crate::ddl::table_meta::TableMetadataAllocator;
|
||||
use crate::ddl::truncate_table::TruncateTableProcedure;
|
||||
use crate::ddl::{DdlContext, NoopRegionFailureDetectorControl};
|
||||
use crate::ddl_manager::RepartitionProcedureFactory;
|
||||
use crate::key::TableMetadataManager;
|
||||
use crate::key::flow::FlowMetadataManager;
|
||||
use crate::kv_backend::memory::MemoryKvBackend;
|
||||
@@ -987,6 +1101,30 @@ mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
struct DummyRepartitionProcedureFactory;
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl RepartitionProcedureFactory for DummyRepartitionProcedureFactory {
|
||||
fn create(
|
||||
&self,
|
||||
_ddl_ctx: &DdlContext,
|
||||
_table_name: TableName,
|
||||
_table_id: TableId,
|
||||
_from_exprs: Vec<String>,
|
||||
_to_exprs: Vec<String>,
|
||||
) -> std::result::Result<BoxedProcedure, BoxedError> {
|
||||
unimplemented!()
|
||||
}
|
||||
|
||||
fn register_loaders(
|
||||
&self,
|
||||
_ddl_ctx: &DdlContext,
|
||||
_procedure_manager: &ProcedureManagerRef,
|
||||
) -> std::result::Result<(), BoxedError> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_try_new() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::new());
|
||||
@@ -1023,6 +1161,7 @@ mod tests {
|
||||
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
|
||||
},
|
||||
procedure_manager.clone(),
|
||||
Arc::new(DummyRepartitionProcedureFactory),
|
||||
true,
|
||||
);
|
||||
|
||||
|
||||
@@ -104,6 +104,20 @@ pub enum Error {
|
||||
source: common_procedure::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to register repartition procedure loader"))]
|
||||
RegisterRepartitionProcedureLoader {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: BoxedError,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to create repartition procedure"))]
|
||||
CreateRepartitionProcedure {
|
||||
source: BoxedError,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to submit procedure"))]
|
||||
SubmitProcedure {
|
||||
#[snafu(implicit)]
|
||||
@@ -1170,6 +1184,8 @@ impl ErrorExt for Error {
|
||||
PutPoison { source, .. } => source.status_code(),
|
||||
ConvertColumnDef { source, .. } => source.status_code(),
|
||||
ProcedureStateReceiver { source, .. } => source.status_code(),
|
||||
RegisterRepartitionProcedureLoader { source, .. } => source.status_code(),
|
||||
CreateRepartitionProcedure { source, .. } => source.status_code(),
|
||||
|
||||
ParseProcedureId { .. }
|
||||
| InvalidNumTopics { .. }
|
||||
|
||||
@@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::fmt::{Display, Formatter};
|
||||
use std::time::Duration;
|
||||
|
||||
@@ -432,11 +432,11 @@ where
|
||||
pub struct GetFileRefs {
|
||||
/// List of region IDs to get file references from active FileHandles (in-memory).
|
||||
pub query_regions: Vec<RegionId>,
|
||||
/// Mapping from the source region ID (where to read the manifest) to
|
||||
/// the target region IDs (whose file references to look for).
|
||||
/// Key: The region ID of the manifest.
|
||||
/// Value: The list of region IDs to find references for in that manifest.
|
||||
pub related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
/// Mapping from the src region IDs (whose file references to look for) to
|
||||
/// the dst region IDs (where to read the manifests).
|
||||
/// Key: The source region IDs (where files originally came from).
|
||||
/// Value: The set of destination region IDs (whose manifests need to be read).
|
||||
pub related_regions: HashMap<RegionId, HashSet<RegionId>>,
|
||||
}
|
||||
|
||||
impl Display for GetFileRefs {
|
||||
|
||||
@@ -1280,6 +1280,11 @@ impl TableMetadataManager {
|
||||
region_distribution(current_table_route_value.region_routes()?);
|
||||
let new_region_distribution = region_distribution(&new_region_routes);
|
||||
|
||||
let update_topic_region_txn = self.topic_region_manager.build_update_txn(
|
||||
table_id,
|
||||
®ion_info.region_wal_options,
|
||||
new_region_wal_options,
|
||||
)?;
|
||||
let update_datanode_table_txn = self.datanode_table_manager().build_update_txn(
|
||||
table_id,
|
||||
region_info,
|
||||
@@ -1291,13 +1296,16 @@ impl TableMetadataManager {
|
||||
|
||||
// Updates the table_route.
|
||||
let new_table_route_value = current_table_route_value.update(new_region_routes)?;
|
||||
|
||||
let (update_table_route_txn, on_update_table_route_failure) = self
|
||||
.table_route_manager()
|
||||
.table_route_storage()
|
||||
.build_update_txn(table_id, current_table_route_value, &new_table_route_value)?;
|
||||
|
||||
let txn = Txn::merge_all(vec![update_datanode_table_txn, update_table_route_txn]);
|
||||
let txn = Txn::merge_all(vec![
|
||||
update_datanode_table_txn,
|
||||
update_table_route_txn,
|
||||
update_topic_region_txn,
|
||||
]);
|
||||
|
||||
let mut r = self.kv_backend.txn(txn).await?;
|
||||
|
||||
@@ -1482,6 +1490,7 @@ mod tests {
|
||||
use crate::key::table_info::TableInfoValue;
|
||||
use crate::key::table_name::TableNameKey;
|
||||
use crate::key::table_route::TableRouteValue;
|
||||
use crate::key::topic_region::TopicRegionKey;
|
||||
use crate::key::{
|
||||
DeserializedValueWithBytes, RegionDistribution, RegionRoleSet, TOPIC_REGION_PREFIX,
|
||||
TableMetadataManager, ViewInfoValue,
|
||||
@@ -2262,6 +2271,218 @@ mod tests {
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_update_table_route_with_topic_region_mapping() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
let table_metadata_manager = TableMetadataManager::new(mem_kv.clone());
|
||||
let region_route = new_test_region_route();
|
||||
let region_routes = vec![region_route.clone()];
|
||||
let table_info: RawTableInfo = new_test_table_info().into();
|
||||
let table_id = table_info.ident.table_id;
|
||||
let engine = table_info.meta.engine.as_str();
|
||||
let region_storage_path =
|
||||
region_storage_path(&table_info.catalog_name, &table_info.schema_name);
|
||||
|
||||
// Create initial metadata with Kafka WAL options
|
||||
let old_region_wal_options: HashMap<RegionNumber, String> = vec![
|
||||
(
|
||||
1,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(),
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
(
|
||||
2,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_2".to_string(),
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.collect();
|
||||
|
||||
create_physical_table_metadata(
|
||||
&table_metadata_manager,
|
||||
table_info.clone(),
|
||||
region_routes.clone(),
|
||||
old_region_wal_options.clone(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let current_table_route_value = DeserializedValueWithBytes::from_inner(
|
||||
TableRouteValue::physical(region_routes.clone()),
|
||||
);
|
||||
|
||||
// Verify initial topic region mappings exist
|
||||
let region_id_1 = RegionId::new(table_id, 1);
|
||||
let region_id_2 = RegionId::new(table_id, 2);
|
||||
let topic_1_key = TopicRegionKey::new(region_id_1, "topic_1");
|
||||
let topic_2_key = TopicRegionKey::new(region_id_2, "topic_2");
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_1_key.clone())
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some()
|
||||
);
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_2_key.clone())
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some()
|
||||
);
|
||||
|
||||
// Test 1: Add new region with new topic
|
||||
let new_region_routes = vec![
|
||||
new_region_route(1, 1),
|
||||
new_region_route(2, 2),
|
||||
new_region_route(3, 3), // New region
|
||||
];
|
||||
let new_region_wal_options: HashMap<RegionNumber, String> = vec![
|
||||
(
|
||||
1,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(), // Unchanged
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
(
|
||||
2,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_2".to_string(), // Unchanged
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
(
|
||||
3,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_3".to_string(), // New topic
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.collect();
|
||||
let current_table_route_value_updated = DeserializedValueWithBytes::from_inner(
|
||||
current_table_route_value
|
||||
.inner
|
||||
.update(new_region_routes.clone())
|
||||
.unwrap(),
|
||||
);
|
||||
table_metadata_manager
|
||||
.update_table_route(
|
||||
table_id,
|
||||
RegionInfo {
|
||||
engine: engine.to_string(),
|
||||
region_storage_path: region_storage_path.clone(),
|
||||
region_options: HashMap::new(),
|
||||
region_wal_options: old_region_wal_options.clone(),
|
||||
},
|
||||
¤t_table_route_value,
|
||||
new_region_routes.clone(),
|
||||
&HashMap::new(),
|
||||
&new_region_wal_options,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
// Verify new topic region mapping was created
|
||||
let region_id_3 = RegionId::new(table_id, 3);
|
||||
let topic_3_key = TopicRegionKey::new(region_id_3, "topic_3");
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_3_key)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some()
|
||||
);
|
||||
// Test 2: Remove a region and change topic for another
|
||||
let newer_region_routes = vec![
|
||||
new_region_route(1, 1),
|
||||
// Region 2 removed
|
||||
// Region 3 now has different topic
|
||||
];
|
||||
let newer_region_wal_options: HashMap<RegionNumber, String> = vec![
|
||||
(
|
||||
1,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(), // Unchanged
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
(
|
||||
3,
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_3_new".to_string(), // Changed topic
|
||||
}))
|
||||
.unwrap(),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.collect();
|
||||
table_metadata_manager
|
||||
.update_table_route(
|
||||
table_id,
|
||||
RegionInfo {
|
||||
engine: engine.to_string(),
|
||||
region_storage_path: region_storage_path.clone(),
|
||||
region_options: HashMap::new(),
|
||||
region_wal_options: new_region_wal_options.clone(),
|
||||
},
|
||||
¤t_table_route_value_updated,
|
||||
newer_region_routes.clone(),
|
||||
&HashMap::new(),
|
||||
&newer_region_wal_options,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
// Verify region 2 mapping was deleted
|
||||
let topic_2_key_new = TopicRegionKey::new(region_id_2, "topic_2");
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_2_key_new)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_none()
|
||||
);
|
||||
// Verify region 3 old topic mapping was deleted
|
||||
let topic_3_key_old = TopicRegionKey::new(region_id_3, "topic_3");
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_3_key_old)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_none()
|
||||
);
|
||||
// Verify region 3 new topic mapping was created
|
||||
let topic_3_key_new = TopicRegionKey::new(region_id_3, "topic_3_new");
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_3_key_new)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some()
|
||||
);
|
||||
// Verify region 1 mapping still exists (unchanged)
|
||||
assert!(
|
||||
table_metadata_manager
|
||||
.topic_region_manager
|
||||
.get(topic_1_key)
|
||||
.await
|
||||
.unwrap()
|
||||
.is_some()
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_destroy_table_metadata() {
|
||||
let mem_kv = Arc::new(MemoryKvBackend::default());
|
||||
|
||||
@@ -243,7 +243,7 @@ impl TopicRegionManager {
|
||||
let topic_region_mapping = self.get_topic_region_mapping(table_id, ®ion_wal_options);
|
||||
let topic_region_keys = topic_region_mapping
|
||||
.iter()
|
||||
.map(|(topic, region_id)| TopicRegionKey::new(*topic, region_id))
|
||||
.map(|(region_id, topic)| TopicRegionKey::new(*region_id, topic))
|
||||
.collect::<Vec<_>>();
|
||||
let operations = topic_region_keys
|
||||
.into_iter()
|
||||
@@ -252,6 +252,55 @@ impl TopicRegionManager {
|
||||
Ok(Txn::new().and_then(operations))
|
||||
}
|
||||
|
||||
/// Build a update topic region mapping transaction.
|
||||
pub fn build_update_txn(
|
||||
&self,
|
||||
table_id: TableId,
|
||||
old_region_wal_options: &HashMap<RegionNumber, String>,
|
||||
new_region_wal_options: &HashMap<RegionNumber, String>,
|
||||
) -> Result<Txn> {
|
||||
let old_wal_options_parsed = parse_region_wal_options(old_region_wal_options)?;
|
||||
let new_wal_options_parsed = parse_region_wal_options(new_region_wal_options)?;
|
||||
let old_mapping = self.get_topic_region_mapping(table_id, &old_wal_options_parsed);
|
||||
let new_mapping = self.get_topic_region_mapping(table_id, &new_wal_options_parsed);
|
||||
|
||||
// Convert to HashMap for easier lookup: RegionId -> Topic
|
||||
let old_map: HashMap<RegionId, &str> = old_mapping.into_iter().collect();
|
||||
let new_map: HashMap<RegionId, &str> = new_mapping.into_iter().collect();
|
||||
let mut ops = Vec::new();
|
||||
|
||||
// Check for deletes (in old but not in new, or topic changed)
|
||||
for (region_id, old_topic) in &old_map {
|
||||
match new_map.get(region_id) {
|
||||
Some(new_topic) if *new_topic == *old_topic => {
|
||||
// Same topic, do nothing (preserve checkpoint)
|
||||
}
|
||||
_ => {
|
||||
// Removed or topic changed -> Delete old
|
||||
let key = TopicRegionKey::new(*region_id, old_topic);
|
||||
ops.push(TxnOp::Delete(key.to_bytes()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Check for adds (in new but not in old, or topic changed)
|
||||
for (region_id, new_topic) in &new_map {
|
||||
match old_map.get(region_id) {
|
||||
Some(old_topic) if *old_topic == *new_topic => {
|
||||
// Same topic, already handled (do nothing)
|
||||
}
|
||||
_ => {
|
||||
// New or topic changed -> Put new
|
||||
let key = TopicRegionKey::new(*region_id, new_topic);
|
||||
// Initialize with empty value (default TopicRegionValue)
|
||||
ops.push(TxnOp::Put(key.to_bytes(), vec![]));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Txn::new().and_then(ops))
|
||||
}
|
||||
|
||||
/// Returns the map of [`RegionId`] to their corresponding topic [`TopicRegionValue`].
|
||||
pub async fn regions(&self, topic: &str) -> Result<HashMap<RegionId, TopicRegionValue>> {
|
||||
let prefix = TopicRegionKey::range_topic_key(topic);
|
||||
@@ -431,4 +480,420 @@ mod tests {
|
||||
RegionId::from_u64(4410931412992)
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_create_txn() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(),
|
||||
}),
|
||||
),
|
||||
(2, WalOptions::RaftEngine), // Should be ignored
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
|
||||
let txn = manager
|
||||
.build_create_txn(table_id, ®ion_wal_options)
|
||||
.unwrap();
|
||||
|
||||
// Verify the transaction contains correct operations
|
||||
// Should create mappings for region 0 and 1, but not region 2 (RaftEngine)
|
||||
let ops = txn.req().success.clone();
|
||||
assert_eq!(ops.len(), 2);
|
||||
|
||||
let keys: Vec<_> = ops
|
||||
.iter()
|
||||
.filter_map(|op| {
|
||||
if let TxnOp::Put(key, _) = op {
|
||||
TopicRegionKey::from_bytes(key).ok()
|
||||
} else {
|
||||
None
|
||||
}
|
||||
})
|
||||
.collect();
|
||||
|
||||
assert_eq!(keys.len(), 2);
|
||||
let region_ids: Vec<_> = keys.iter().map(|k| k.region_id).collect();
|
||||
assert!(region_ids.contains(&RegionId::new(table_id, 0)));
|
||||
assert!(region_ids.contains(&RegionId::new(table_id, 1)));
|
||||
assert!(!region_ids.contains(&RegionId::new(table_id, 2)));
|
||||
|
||||
// Verify topics are correct
|
||||
for key in keys {
|
||||
match key.region_id.region_number() {
|
||||
0 => assert_eq!(key.topic, "topic_0"),
|
||||
1 => assert_eq!(key.topic, "topic_1"),
|
||||
_ => panic!("Unexpected region number"),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_update_txn_add_new_region() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let old_region_wal_options = vec![(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let new_region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(),
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let txn = manager
|
||||
.build_update_txn(table_id, &old_region_wal_options, &new_region_wal_options)
|
||||
.unwrap();
|
||||
let ops = txn.req().success.clone();
|
||||
// Should only have Put for new region 1 (region 0 unchanged)
|
||||
assert_eq!(ops.len(), 1);
|
||||
if let TxnOp::Put(key, _) = &ops[0] {
|
||||
let topic_key = TopicRegionKey::from_bytes(key).unwrap();
|
||||
assert_eq!(topic_key.region_id, RegionId::new(table_id, 1));
|
||||
assert_eq!(topic_key.topic, "topic_1");
|
||||
} else {
|
||||
panic!("Expected Put operation");
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_update_txn_remove_region() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let old_region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(),
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let new_region_wal_options = vec![(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let txn = manager
|
||||
.build_update_txn(table_id, &old_region_wal_options, &new_region_wal_options)
|
||||
.unwrap();
|
||||
let ops = txn.req().success.clone();
|
||||
// Should only have Delete for removed region 1 (region 0 unchanged)
|
||||
assert_eq!(ops.len(), 1);
|
||||
match &ops[0] {
|
||||
TxnOp::Delete(key) => {
|
||||
let topic_key = TopicRegionKey::from_bytes(key).unwrap();
|
||||
assert_eq!(topic_key.region_id, RegionId::new(table_id, 1));
|
||||
assert_eq!(topic_key.topic, "topic_1");
|
||||
}
|
||||
TxnOp::Put(_, _) | TxnOp::Get(_) => {
|
||||
panic!("Expected Delete operation");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_update_txn_change_topic() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let old_region_wal_options = vec![(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let new_region_wal_options = vec![(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0_new".to_string(),
|
||||
}),
|
||||
)]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let txn = manager
|
||||
.build_update_txn(table_id, &old_region_wal_options, &new_region_wal_options)
|
||||
.unwrap();
|
||||
let ops = txn.req().success.clone();
|
||||
// Should have Delete for old topic and Put for new topic
|
||||
assert_eq!(ops.len(), 2);
|
||||
|
||||
let mut delete_found = false;
|
||||
let mut put_found = false;
|
||||
for op in ops {
|
||||
match op {
|
||||
TxnOp::Delete(key) => {
|
||||
let topic_key = TopicRegionKey::from_bytes(&key).unwrap();
|
||||
assert_eq!(topic_key.region_id, RegionId::new(table_id, 0));
|
||||
assert_eq!(topic_key.topic, "topic_0");
|
||||
delete_found = true;
|
||||
}
|
||||
TxnOp::Put(key, _) => {
|
||||
let topic_key = TopicRegionKey::from_bytes(&key).unwrap();
|
||||
assert_eq!(topic_key.region_id, RegionId::new(table_id, 0));
|
||||
assert_eq!(topic_key.topic, "topic_0_new");
|
||||
put_found = true;
|
||||
}
|
||||
TxnOp::Get(_) => {
|
||||
// Get operations shouldn't appear in this context
|
||||
panic!("Unexpected Get operation in update transaction");
|
||||
}
|
||||
}
|
||||
}
|
||||
assert!(delete_found, "Expected Delete operation for old topic");
|
||||
assert!(put_found, "Expected Put operation for new topic");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_update_txn_no_change() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(),
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let txn = manager
|
||||
.build_update_txn(table_id, ®ion_wal_options, ®ion_wal_options)
|
||||
.unwrap();
|
||||
// Should have no operations when nothing changes (preserves checkpoint)
|
||||
let ops = txn.req().success.clone();
|
||||
assert_eq!(ops.len(), 0);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_update_txn_mixed_scenarios() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let old_region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
2,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_2".to_string(),
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let new_region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(), // Unchanged
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1_new".to_string(), // Topic changed
|
||||
}),
|
||||
),
|
||||
// Region 2 removed
|
||||
(
|
||||
3,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_3".to_string(), // New region
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let txn = manager
|
||||
.build_update_txn(table_id, &old_region_wal_options, &new_region_wal_options)
|
||||
.unwrap();
|
||||
|
||||
let ops = txn.req().success.clone();
|
||||
// Should have:
|
||||
// - Delete for region 2 (removed)
|
||||
// - Delete for region 1 old topic (topic changed)
|
||||
// - Put for region 1 new topic (topic changed)
|
||||
// - Put for region 3 (new)
|
||||
// Region 0 unchanged, so no operation
|
||||
assert_eq!(ops.len(), 4);
|
||||
|
||||
let mut delete_ops = 0;
|
||||
let mut put_ops = 0;
|
||||
let mut delete_region_2 = false;
|
||||
let mut delete_region_1_old = false;
|
||||
let mut put_region_1_new = false;
|
||||
let mut put_region_3 = false;
|
||||
|
||||
for op in ops {
|
||||
match op {
|
||||
TxnOp::Delete(key) => {
|
||||
delete_ops += 1;
|
||||
let topic_key = TopicRegionKey::from_bytes(&key).unwrap();
|
||||
match topic_key.region_id.region_number() {
|
||||
1 => {
|
||||
assert_eq!(topic_key.topic, "topic_1");
|
||||
delete_region_1_old = true;
|
||||
}
|
||||
2 => {
|
||||
assert_eq!(topic_key.topic, "topic_2");
|
||||
delete_region_2 = true;
|
||||
}
|
||||
_ => panic!("Unexpected delete operation for region"),
|
||||
}
|
||||
}
|
||||
TxnOp::Put(key, _) => {
|
||||
put_ops += 1;
|
||||
let topic_key: TopicRegionKey<'_> = TopicRegionKey::from_bytes(&key).unwrap();
|
||||
match topic_key.region_id.region_number() {
|
||||
1 => {
|
||||
assert_eq!(topic_key.topic, "topic_1_new");
|
||||
put_region_1_new = true;
|
||||
}
|
||||
3 => {
|
||||
assert_eq!(topic_key.topic, "topic_3");
|
||||
put_region_3 = true;
|
||||
}
|
||||
_ => panic!("Unexpected put operation for region"),
|
||||
}
|
||||
}
|
||||
TxnOp::Get(_) => {
|
||||
panic!("Unexpected Get operation in update transaction");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
assert_eq!(delete_ops, 2);
|
||||
assert_eq!(put_ops, 2);
|
||||
assert!(delete_region_2, "Expected delete for removed region 2");
|
||||
assert!(
|
||||
delete_region_1_old,
|
||||
"Expected delete for region 1 old topic"
|
||||
);
|
||||
assert!(put_region_1_new, "Expected put for region 1 new topic");
|
||||
assert!(put_region_3, "Expected put for new region 3");
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_build_update_txn_with_raft_engine() {
|
||||
let kv_backend = Arc::new(MemoryKvBackend::default());
|
||||
let manager = TopicRegionManager::new(kv_backend.clone());
|
||||
let table_id = 1;
|
||||
let old_region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(1, WalOptions::RaftEngine), // Should be ignored
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let new_region_wal_options = vec![
|
||||
(
|
||||
0,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_0".to_string(),
|
||||
}),
|
||||
),
|
||||
(
|
||||
1,
|
||||
WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: "topic_1".to_string(), // Changed from RaftEngine to Kafka
|
||||
}),
|
||||
),
|
||||
]
|
||||
.into_iter()
|
||||
.map(|(num, opts)| (num, serde_json::to_string(&opts).unwrap()))
|
||||
.collect::<HashMap<_, _>>();
|
||||
let txn = manager
|
||||
.build_update_txn(table_id, &old_region_wal_options, &new_region_wal_options)
|
||||
.unwrap();
|
||||
let ops = txn.req().success.clone();
|
||||
// Should only have Put for region 1 (new Kafka topic)
|
||||
// Region 0 unchanged, so no operation
|
||||
// Region 1 was RaftEngine before (not tracked), so only Put needed
|
||||
assert_eq!(ops.len(), 1);
|
||||
match &ops[0] {
|
||||
TxnOp::Put(key, _) => {
|
||||
let topic_key = TopicRegionKey::from_bytes(key).unwrap();
|
||||
assert_eq!(topic_key.region_id, RegionId::new(table_id, 1));
|
||||
assert_eq!(topic_key.topic, "topic_1");
|
||||
}
|
||||
TxnOp::Delete(_) | TxnOp::Get(_) => {
|
||||
panic!("Expected Put operation for new Kafka region");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -37,6 +37,7 @@ use datafusion::datasource::DefaultTableSource;
|
||||
use futures::Stream;
|
||||
use futures::stream::StreamExt;
|
||||
use query::parser::PromQuery;
|
||||
use servers::http::prom_store::PHYSICAL_TABLE_PARAM;
|
||||
use servers::interceptor::{GrpcQueryInterceptor, GrpcQueryInterceptorRef};
|
||||
use servers::query_handler::grpc::GrpcQueryHandler;
|
||||
use servers::query_handler::sql::SqlQueryHandler;
|
||||
@@ -73,10 +74,20 @@ impl GrpcQueryHandler for Instance {
|
||||
|
||||
let output = match request {
|
||||
Request::Inserts(requests) => self.handle_inserts(requests, ctx.clone()).await?,
|
||||
Request::RowInserts(requests) => {
|
||||
self.handle_row_inserts(requests, ctx.clone(), false, false)
|
||||
Request::RowInserts(requests) => match ctx.extension(PHYSICAL_TABLE_PARAM) {
|
||||
Some(physical_table) => {
|
||||
self.handle_metric_row_inserts(
|
||||
requests,
|
||||
ctx.clone(),
|
||||
physical_table.to_string(),
|
||||
)
|
||||
.await?
|
||||
}
|
||||
}
|
||||
None => {
|
||||
self.handle_row_inserts(requests, ctx.clone(), false, false)
|
||||
.await?
|
||||
}
|
||||
},
|
||||
Request::Deletes(requests) => self.handle_deletes(requests, ctx.clone()).await?,
|
||||
Request::RowDeletes(requests) => self.handle_row_deletes(requests, ctx.clone()).await?,
|
||||
Request::Query(query_request) => {
|
||||
|
||||
@@ -1038,7 +1038,7 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to serialize partition expression: {}", source))]
|
||||
#[snafu(display("Failed to serialize partition expression"))]
|
||||
SerializePartitionExpr {
|
||||
#[snafu(source)]
|
||||
source: partition::error::Error,
|
||||
@@ -1046,6 +1046,20 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to deserialize partition expression"))]
|
||||
DeserializePartitionExpr {
|
||||
#[snafu(source)]
|
||||
source: partition::error::Error,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Empty partition expression"))]
|
||||
EmptyPartitionExpr {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display(
|
||||
"Partition expression mismatch, region id: {}, expected: {}, actual: {}",
|
||||
region_id,
|
||||
@@ -1163,7 +1177,8 @@ impl ErrorExt for Error {
|
||||
| Error::MailboxChannelClosed { .. }
|
||||
| Error::IsNotLeader { .. } => StatusCode::IllegalState,
|
||||
Error::RetryLaterWithSource { source, .. } => source.status_code(),
|
||||
Error::SerializePartitionExpr { source, .. } => source.status_code(),
|
||||
Error::SerializePartitionExpr { source, .. }
|
||||
| Error::DeserializePartitionExpr { source, .. } => source.status_code(),
|
||||
|
||||
Error::Unsupported { .. } => StatusCode::Unsupported,
|
||||
|
||||
@@ -1189,7 +1204,8 @@ impl ErrorExt for Error {
|
||||
| Error::RepartitionSourceRegionMissing { .. }
|
||||
| Error::RepartitionTargetRegionMissing { .. }
|
||||
| Error::PartitionExprMismatch { .. }
|
||||
| Error::RepartitionSourceExprMismatch { .. } => StatusCode::InvalidArguments,
|
||||
| Error::RepartitionSourceExprMismatch { .. }
|
||||
| Error::EmptyPartitionExpr { .. } => StatusCode::InvalidArguments,
|
||||
Error::LeaseKeyFromUtf8 { .. }
|
||||
| Error::LeaseValueFromUtf8 { .. }
|
||||
| Error::InvalidRegionKeyFromUtf8 { .. }
|
||||
|
||||
@@ -12,9 +12,6 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
// TODO(discord9): remove this once gc scheduler is fully merged
|
||||
#![allow(unused)]
|
||||
|
||||
use std::collections::{HashMap, HashSet};
|
||||
|
||||
use common_meta::peer::Peer;
|
||||
@@ -29,6 +26,7 @@ mod options;
|
||||
mod procedure;
|
||||
mod scheduler;
|
||||
mod tracker;
|
||||
mod util;
|
||||
|
||||
pub use options::GcSchedulerOptions;
|
||||
pub use procedure::BatchGcProcedure;
|
||||
|
||||
@@ -12,29 +12,22 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::collections::HashMap;
|
||||
use std::time::Duration;
|
||||
|
||||
use api::v1::meta::MailboxMessage;
|
||||
use common_meta::datanode::RegionStat;
|
||||
use common_meta::instruction::{
|
||||
GcRegions, GetFileRefs, GetFileRefsReply, Instruction, InstructionReply,
|
||||
};
|
||||
use common_meta::key::TableMetadataManagerRef;
|
||||
use common_meta::key::table_route::PhysicalTableRouteValue;
|
||||
use common_meta::peer::Peer;
|
||||
use common_procedure::{ProcedureManagerRef, ProcedureWithId, watcher};
|
||||
use common_telemetry::{debug, error, warn};
|
||||
use common_telemetry::debug;
|
||||
use snafu::{OptionExt as _, ResultExt as _};
|
||||
use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId};
|
||||
use store_api::storage::{GcReport, RegionId};
|
||||
use table::metadata::TableId;
|
||||
|
||||
use crate::cluster::MetaPeerClientRef;
|
||||
use crate::error::{self, Result, TableMetadataManagerSnafu, UnexpectedSnafu};
|
||||
use crate::gc::Region2Peers;
|
||||
use crate::gc::procedure::{BatchGcProcedure, GcRegionProcedure};
|
||||
use crate::handler::HeartbeatMailbox;
|
||||
use crate::service::mailbox::{Channel, MailboxRef};
|
||||
use crate::error::{self, Result, TableMetadataManagerSnafu};
|
||||
use crate::gc::procedure::BatchGcProcedure;
|
||||
use crate::service::mailbox::MailboxRef;
|
||||
|
||||
#[async_trait::async_trait]
|
||||
pub(crate) trait SchedulerCtx: Send + Sync {
|
||||
@@ -45,19 +38,9 @@ pub(crate) trait SchedulerCtx: Send + Sync {
|
||||
table_id: TableId,
|
||||
) -> Result<(TableId, PhysicalTableRouteValue)>;
|
||||
|
||||
async fn get_file_references(
|
||||
&self,
|
||||
query_regions: &[RegionId],
|
||||
related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
region_routes: &Region2Peers,
|
||||
timeout: Duration,
|
||||
) -> Result<FileRefsManifest>;
|
||||
|
||||
async fn gc_regions(
|
||||
&self,
|
||||
peer: Peer,
|
||||
region_ids: &[RegionId],
|
||||
file_refs_manifest: &FileRefsManifest,
|
||||
full_file_listing: bool,
|
||||
timeout: Duration,
|
||||
) -> Result<GcReport>;
|
||||
@@ -100,7 +83,7 @@ impl SchedulerCtx for DefaultGcSchedulerCtx {
|
||||
let dn_stats = self.meta_peer_client.get_all_dn_stat_kvs().await?;
|
||||
let mut table_to_region_stats: HashMap<TableId, Vec<RegionStat>> = HashMap::new();
|
||||
for (_dn_id, stats) in dn_stats {
|
||||
let mut stats = stats.stats;
|
||||
let stats = stats.stats;
|
||||
|
||||
let Some(latest_stat) = stats.iter().max_by_key(|s| s.timestamp_millis).cloned() else {
|
||||
continue;
|
||||
@@ -129,142 +112,34 @@ impl SchedulerCtx for DefaultGcSchedulerCtx {
|
||||
|
||||
async fn gc_regions(
|
||||
&self,
|
||||
peer: Peer,
|
||||
region_ids: &[RegionId],
|
||||
file_refs_manifest: &FileRefsManifest,
|
||||
full_file_listing: bool,
|
||||
timeout: Duration,
|
||||
) -> Result<GcReport> {
|
||||
self.gc_regions_inner(
|
||||
peer,
|
||||
region_ids,
|
||||
file_refs_manifest,
|
||||
full_file_listing,
|
||||
timeout,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
async fn get_file_references(
|
||||
&self,
|
||||
query_regions: &[RegionId],
|
||||
related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
region_routes: &Region2Peers,
|
||||
timeout: Duration,
|
||||
) -> Result<FileRefsManifest> {
|
||||
debug!(
|
||||
"Getting file references for {} regions",
|
||||
query_regions.len()
|
||||
);
|
||||
|
||||
// Group regions by datanode to minimize RPC calls
|
||||
let mut datanode2query_regions: HashMap<Peer, Vec<RegionId>> = HashMap::new();
|
||||
|
||||
for region_id in query_regions {
|
||||
if let Some((leader, followers)) = region_routes.get(region_id) {
|
||||
datanode2query_regions
|
||||
.entry(leader.clone())
|
||||
.or_default()
|
||||
.push(*region_id);
|
||||
// also need to send for follower regions for file refs in case query is running on follower
|
||||
for follower in followers {
|
||||
datanode2query_regions
|
||||
.entry(follower.clone())
|
||||
.or_default()
|
||||
.push(*region_id);
|
||||
}
|
||||
} else {
|
||||
return error::UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"region_routes: {region_routes:?} does not contain region_id: {region_id}",
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
}
|
||||
let mut datanode2related_regions: HashMap<Peer, HashMap<RegionId, Vec<RegionId>>> =
|
||||
HashMap::new();
|
||||
for (related_region, queries) in related_regions {
|
||||
if let Some((leader, followers)) = region_routes.get(&related_region) {
|
||||
datanode2related_regions
|
||||
.entry(leader.clone())
|
||||
.or_default()
|
||||
.insert(related_region, queries.clone());
|
||||
} // since read from manifest, no need to send to followers
|
||||
}
|
||||
|
||||
// Send GetFileRefs instructions to each datanode
|
||||
let mut all_file_refs: HashMap<RegionId, HashSet<_>> = HashMap::new();
|
||||
let mut all_manifest_versions = HashMap::new();
|
||||
|
||||
for (peer, regions) in datanode2query_regions {
|
||||
let related_regions = datanode2related_regions.remove(&peer).unwrap_or_default();
|
||||
match self
|
||||
.send_get_file_refs_instruction(&peer, ®ions, related_regions, timeout)
|
||||
.await
|
||||
{
|
||||
Ok(manifest) => {
|
||||
// TODO(discord9): if other regions provide file refs for one region on other datanode, and no version,
|
||||
// is it correct to merge manifest_version directly?
|
||||
// FIXME: follower region how to merge version???
|
||||
|
||||
for (region_id, file_refs) in manifest.file_refs {
|
||||
all_file_refs
|
||||
.entry(region_id)
|
||||
.or_default()
|
||||
.extend(file_refs);
|
||||
}
|
||||
// region manifest version should be the smallest one among all peers, so outdated region can be detected
|
||||
for (region_id, version) in manifest.manifest_version {
|
||||
let entry = all_manifest_versions.entry(region_id).or_insert(version);
|
||||
*entry = (*entry).min(version);
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
warn!(
|
||||
"Failed to get file refs from datanode {}: {}. Skipping regions on this datanode.",
|
||||
peer, e
|
||||
);
|
||||
// Continue processing other datanodes instead of failing the entire operation
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(FileRefsManifest {
|
||||
file_refs: all_file_refs,
|
||||
manifest_version: all_manifest_versions,
|
||||
})
|
||||
self.gc_regions_inner(region_ids, full_file_listing, timeout)
|
||||
.await
|
||||
}
|
||||
}
|
||||
|
||||
impl DefaultGcSchedulerCtx {
|
||||
async fn gc_regions_inner(
|
||||
&self,
|
||||
peer: Peer,
|
||||
region_ids: &[RegionId],
|
||||
file_refs_manifest: &FileRefsManifest,
|
||||
full_file_listing: bool,
|
||||
timeout: Duration,
|
||||
) -> Result<GcReport> {
|
||||
debug!(
|
||||
"Sending GC instruction to datanode {} for {} regions (full_file_listing: {})",
|
||||
peer,
|
||||
"Sending GC instruction for {} regions (full_file_listing: {})",
|
||||
region_ids.len(),
|
||||
full_file_listing
|
||||
);
|
||||
|
||||
let gc_regions = GcRegions {
|
||||
regions: region_ids.to_vec(),
|
||||
file_refs_manifest: file_refs_manifest.clone(),
|
||||
full_file_listing,
|
||||
};
|
||||
let procedure = GcRegionProcedure::new(
|
||||
let procedure = BatchGcProcedure::new(
|
||||
self.mailbox.clone(),
|
||||
self.table_metadata_manager.clone(),
|
||||
self.server_addr.clone(),
|
||||
peer,
|
||||
gc_regions,
|
||||
format!("GC for {} regions", region_ids.len()),
|
||||
region_ids.to_vec(),
|
||||
full_file_listing,
|
||||
timeout,
|
||||
);
|
||||
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
|
||||
@@ -285,96 +160,8 @@ impl DefaultGcSchedulerCtx {
|
||||
),
|
||||
})?;
|
||||
|
||||
let gc_report = GcRegionProcedure::cast_result(res)?;
|
||||
let gc_report = BatchGcProcedure::cast_result(res)?;
|
||||
|
||||
Ok(gc_report)
|
||||
}
|
||||
|
||||
/// TODO(discord9): add support to read manifest of related regions for file refs too
|
||||
/// (now it's only reading active FileHandles)
|
||||
async fn send_get_file_refs_instruction(
|
||||
&self,
|
||||
peer: &Peer,
|
||||
query_regions: &[RegionId],
|
||||
related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
timeout: Duration,
|
||||
) -> Result<FileRefsManifest> {
|
||||
debug!(
|
||||
"Sending GetFileRefs instruction to datanode {} for {} regions",
|
||||
peer,
|
||||
query_regions.len()
|
||||
);
|
||||
|
||||
let instruction = Instruction::GetFileRefs(GetFileRefs {
|
||||
query_regions: query_regions.to_vec(),
|
||||
related_regions,
|
||||
});
|
||||
|
||||
let reply = self
|
||||
.send_instruction(peer, instruction, "Get file references", timeout)
|
||||
.await?;
|
||||
|
||||
let InstructionReply::GetFileRefs(GetFileRefsReply {
|
||||
file_refs_manifest,
|
||||
success,
|
||||
error,
|
||||
}) = reply
|
||||
else {
|
||||
return error::UnexpectedInstructionReplySnafu {
|
||||
mailbox_message: format!("{:?}", reply),
|
||||
reason: "Unexpected reply of the GetFileRefs instruction",
|
||||
}
|
||||
.fail();
|
||||
};
|
||||
|
||||
if !success {
|
||||
return error::UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"Failed to get file references from datanode {}: {:?}",
|
||||
peer, error
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
Ok(file_refs_manifest)
|
||||
}
|
||||
|
||||
async fn send_instruction(
|
||||
&self,
|
||||
peer: &Peer,
|
||||
instruction: Instruction,
|
||||
description: &str,
|
||||
timeout: Duration,
|
||||
) -> Result<InstructionReply> {
|
||||
let msg = MailboxMessage::json_message(
|
||||
&format!("{}: {}", description, instruction),
|
||||
&format!("Metasrv@{}", self.server_addr),
|
||||
&format!("Datanode-{}@{}", peer.id, peer.addr),
|
||||
common_time::util::current_time_millis(),
|
||||
&instruction,
|
||||
)
|
||||
.with_context(|_| error::SerializeToJsonSnafu {
|
||||
input: instruction.to_string(),
|
||||
})?;
|
||||
|
||||
let mailbox_rx = self
|
||||
.mailbox
|
||||
.send(&Channel::Datanode(peer.id), msg, timeout)
|
||||
.await?;
|
||||
|
||||
match mailbox_rx.await {
|
||||
Ok(reply_msg) => {
|
||||
let reply = HeartbeatMailbox::json_reply(&reply_msg)?;
|
||||
Ok(reply)
|
||||
}
|
||||
Err(e) => {
|
||||
error!(
|
||||
"Failed to receive reply from datanode {} for {}: {}",
|
||||
peer, description, e
|
||||
);
|
||||
Err(e)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -15,24 +15,17 @@
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::time::Instant;
|
||||
|
||||
use common_meta::key::table_route::PhysicalTableRouteValue;
|
||||
use common_meta::peer::Peer;
|
||||
use common_telemetry::{debug, error, info, warn};
|
||||
use futures::StreamExt;
|
||||
use itertools::Itertools;
|
||||
use store_api::storage::{FileRefsManifest, GcReport, RegionId};
|
||||
use store_api::storage::{GcReport, RegionId};
|
||||
use table::metadata::TableId;
|
||||
use tokio::time::sleep;
|
||||
|
||||
use crate::error::Result;
|
||||
use crate::gc::candidate::GcCandidate;
|
||||
use crate::gc::scheduler::{GcJobReport, GcScheduler};
|
||||
use crate::gc::tracker::RegionGcInfo;
|
||||
use crate::region;
|
||||
|
||||
pub(crate) type Region2Peers = HashMap<RegionId, (Peer, Vec<Peer>)>;
|
||||
|
||||
pub(crate) type Peer2Regions = HashMap<Peer, HashSet<RegionId>>;
|
||||
|
||||
impl GcScheduler {
|
||||
/// Iterate through all region stats, find region that might need gc, and send gc instruction to
|
||||
@@ -61,6 +54,8 @@ impl GcScheduler {
|
||||
.aggregate_candidates_by_datanode(per_table_candidates)
|
||||
.await?;
|
||||
|
||||
// TODO(discord9): add deleted regions from repartition mapping
|
||||
|
||||
if datanode_to_candidates.is_empty() {
|
||||
info!("No valid datanode candidates found, skipping GC cycle");
|
||||
return Ok(Default::default());
|
||||
@@ -83,17 +78,6 @@ impl GcScheduler {
|
||||
Ok(report)
|
||||
}
|
||||
|
||||
/// Find related regions that might share files with the candidate regions.
|
||||
/// Currently returns the same regions since repartition is not implemented yet.
|
||||
/// TODO(discord9): When repartition is implemented, this should also find src/dst regions
|
||||
/// that might share files with the candidate regions.
|
||||
pub(crate) async fn find_related_regions(
|
||||
&self,
|
||||
candidate_region_ids: &[RegionId],
|
||||
) -> Result<HashMap<RegionId, Vec<RegionId>>> {
|
||||
Ok(candidate_region_ids.iter().map(|&r| (r, vec![r])).collect())
|
||||
}
|
||||
|
||||
/// Aggregate GC candidates by their corresponding datanode peer.
|
||||
pub(crate) async fn aggregate_candidates_by_datanode(
|
||||
&self,
|
||||
@@ -210,28 +194,11 @@ impl GcScheduler {
|
||||
|
||||
let all_region_ids: Vec<RegionId> = candidates.iter().map(|(_, c)| c.region_id).collect();
|
||||
|
||||
let all_related_regions = self.find_related_regions(&all_region_ids).await?;
|
||||
|
||||
let (region_to_peer, _) = self
|
||||
.discover_datanodes_for_regions(&all_related_regions.keys().cloned().collect_vec())
|
||||
.await?;
|
||||
|
||||
// Step 1: Get file references for all regions on this datanode
|
||||
let file_refs_manifest = self
|
||||
.ctx
|
||||
.get_file_references(
|
||||
&all_region_ids,
|
||||
all_related_regions,
|
||||
®ion_to_peer,
|
||||
self.config.mailbox_timeout,
|
||||
)
|
||||
.await?;
|
||||
|
||||
// Step 2: Create a single GcRegionProcedure for all regions on this datanode
|
||||
// Step 2: Run GC for all regions on this datanode in a single batch
|
||||
let (gc_report, fully_listed_regions) = {
|
||||
// Partition regions into full listing and fast listing in a single pass
|
||||
|
||||
let mut batch_full_listing_decisions =
|
||||
let batch_full_listing_decisions =
|
||||
self.batch_should_use_full_listing(&all_region_ids).await;
|
||||
|
||||
let need_full_list_regions = batch_full_listing_decisions
|
||||
@@ -242,7 +209,7 @@ impl GcScheduler {
|
||||
},
|
||||
)
|
||||
.collect_vec();
|
||||
let mut fast_list_regions = batch_full_listing_decisions
|
||||
let fast_list_regions = batch_full_listing_decisions
|
||||
.iter()
|
||||
.filter_map(
|
||||
|(®ion_id, &need_full)| {
|
||||
@@ -257,13 +224,7 @@ impl GcScheduler {
|
||||
if !fast_list_regions.is_empty() {
|
||||
match self
|
||||
.ctx
|
||||
.gc_regions(
|
||||
peer.clone(),
|
||||
&fast_list_regions,
|
||||
&file_refs_manifest,
|
||||
false,
|
||||
self.config.mailbox_timeout,
|
||||
)
|
||||
.gc_regions(&fast_list_regions, false, self.config.mailbox_timeout)
|
||||
.await
|
||||
{
|
||||
Ok(report) => combined_report.merge(report),
|
||||
@@ -284,13 +245,7 @@ impl GcScheduler {
|
||||
if !need_full_list_regions.is_empty() {
|
||||
match self
|
||||
.ctx
|
||||
.gc_regions(
|
||||
peer.clone(),
|
||||
&need_full_list_regions,
|
||||
&file_refs_manifest,
|
||||
true,
|
||||
self.config.mailbox_timeout,
|
||||
)
|
||||
.gc_regions(&need_full_list_regions, true, self.config.mailbox_timeout)
|
||||
.await
|
||||
{
|
||||
Ok(report) => combined_report.merge(report),
|
||||
@@ -330,98 +285,6 @@ impl GcScheduler {
|
||||
Ok(gc_report)
|
||||
}
|
||||
|
||||
/// Discover datanodes for the given regions(and it's related regions) by fetching table routes in batches.
|
||||
/// Returns mappings from region to peer(leader, Vec<followers>) and peer to regions.
|
||||
async fn discover_datanodes_for_regions(
|
||||
&self,
|
||||
regions: &[RegionId],
|
||||
) -> Result<(Region2Peers, Peer2Regions)> {
|
||||
let all_related_regions = self
|
||||
.find_related_regions(regions)
|
||||
.await?
|
||||
.into_iter()
|
||||
.flat_map(|(k, mut v)| {
|
||||
v.push(k);
|
||||
v
|
||||
})
|
||||
.collect_vec();
|
||||
let mut region_to_peer = HashMap::new();
|
||||
let mut peer_to_regions = HashMap::new();
|
||||
|
||||
// Group regions by table ID for batch processing
|
||||
let mut table_to_regions: HashMap<TableId, Vec<RegionId>> = HashMap::new();
|
||||
for region_id in all_related_regions {
|
||||
let table_id = region_id.table_id();
|
||||
table_to_regions
|
||||
.entry(table_id)
|
||||
.or_default()
|
||||
.push(region_id);
|
||||
}
|
||||
|
||||
// Process each table's regions together for efficiency
|
||||
for (table_id, table_regions) in table_to_regions {
|
||||
match self.ctx.get_table_route(table_id).await {
|
||||
Ok((_phy_table_id, table_route)) => {
|
||||
self.get_table_regions_peer(
|
||||
&table_route,
|
||||
&table_regions,
|
||||
&mut region_to_peer,
|
||||
&mut peer_to_regions,
|
||||
);
|
||||
}
|
||||
Err(e) => {
|
||||
// Continue with other tables instead of failing completely
|
||||
// TODO(discord9): consider failing here instead
|
||||
warn!(
|
||||
"Failed to get table route for table {}: {}, skipping its regions",
|
||||
table_id, e
|
||||
);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok((region_to_peer, peer_to_regions))
|
||||
}
|
||||
|
||||
/// Process regions for a single table to find their current leader peers.
|
||||
fn get_table_regions_peer(
|
||||
&self,
|
||||
table_route: &PhysicalTableRouteValue,
|
||||
table_regions: &[RegionId],
|
||||
region_to_peer: &mut Region2Peers,
|
||||
peer_to_regions: &mut Peer2Regions,
|
||||
) {
|
||||
for ®ion_id in table_regions {
|
||||
let mut found = false;
|
||||
|
||||
// Find the region in the table route
|
||||
for region_route in &table_route.region_routes {
|
||||
if region_route.region.id == region_id
|
||||
&& let Some(leader_peer) = ®ion_route.leader_peer
|
||||
{
|
||||
region_to_peer.insert(
|
||||
region_id,
|
||||
(leader_peer.clone(), region_route.follower_peers.clone()),
|
||||
);
|
||||
peer_to_regions
|
||||
.entry(leader_peer.clone())
|
||||
.or_default()
|
||||
.insert(region_id);
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if !found {
|
||||
warn!(
|
||||
"Failed to find region {} in table route or no leader peer found",
|
||||
region_id,
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn batch_should_use_full_listing(
|
||||
&self,
|
||||
region_ids: &[RegionId],
|
||||
|
||||
@@ -36,10 +36,9 @@ use store_api::storage::{FileRefsManifest, GcReport, RegionId};
|
||||
use table::metadata::TableId;
|
||||
use tokio::sync::mpsc::Sender;
|
||||
|
||||
use crate::error::{Result, UnexpectedSnafu};
|
||||
use crate::error::Result;
|
||||
use crate::gc::candidate::GcCandidate;
|
||||
use crate::gc::ctx::SchedulerCtx;
|
||||
use crate::gc::handler::Region2Peers;
|
||||
use crate::gc::options::GcSchedulerOptions;
|
||||
use crate::gc::scheduler::{Event, GcScheduler};
|
||||
|
||||
@@ -67,12 +66,10 @@ pub struct MockSchedulerCtx {
|
||||
pub gc_reports: Arc<Mutex<HashMap<RegionId, GcReport>>>,
|
||||
pub candidates: Arc<Mutex<Option<HashMap<TableId, Vec<GcCandidate>>>>>,
|
||||
pub get_table_to_region_stats_calls: Arc<Mutex<usize>>,
|
||||
pub get_file_references_calls: Arc<Mutex<usize>>,
|
||||
pub gc_regions_calls: Arc<Mutex<usize>>,
|
||||
// Error injection fields for testing
|
||||
pub get_table_to_region_stats_error: Arc<Mutex<Option<crate::error::Error>>>,
|
||||
pub get_table_route_error: Arc<Mutex<Option<crate::error::Error>>>,
|
||||
pub get_file_references_error: Arc<Mutex<Option<crate::error::Error>>>,
|
||||
pub gc_regions_error: Arc<Mutex<Option<crate::error::Error>>>,
|
||||
// Retry testing fields
|
||||
pub gc_regions_retry_count: Arc<Mutex<HashMap<RegionId, usize>>>,
|
||||
@@ -119,57 +116,12 @@ impl MockSchedulerCtx {
|
||||
*self.get_table_route_error.lock().unwrap() = Some(error);
|
||||
}
|
||||
|
||||
/// Set an error to be returned by `get_file_references`
|
||||
#[allow(dead_code)]
|
||||
pub fn with_get_file_references_error(self, error: crate::error::Error) -> Self {
|
||||
*self.get_file_references_error.lock().unwrap() = Some(error);
|
||||
self
|
||||
}
|
||||
|
||||
/// Set an error to be returned by `gc_regions`
|
||||
pub fn with_gc_regions_error(self, error: crate::error::Error) -> Self {
|
||||
*self.gc_regions_error.lock().unwrap() = Some(error);
|
||||
self
|
||||
}
|
||||
|
||||
/// Set a sequence of errors to be returned by `gc_regions` for retry testing
|
||||
pub fn set_gc_regions_error_sequence(&self, errors: Vec<crate::error::Error>) {
|
||||
*self.gc_regions_error_sequence.lock().unwrap() = errors;
|
||||
}
|
||||
|
||||
/// Set success after a specific number of retries for a region
|
||||
pub fn set_gc_regions_success_after_retries(&self, region_id: RegionId, retries: usize) {
|
||||
self.gc_regions_success_after_retries
|
||||
.lock()
|
||||
.unwrap()
|
||||
.insert(region_id, retries);
|
||||
}
|
||||
|
||||
/// Get the retry count for a specific region
|
||||
pub fn get_retry_count(&self, region_id: RegionId) -> usize {
|
||||
self.gc_regions_retry_count
|
||||
.lock()
|
||||
.unwrap()
|
||||
.get(®ion_id)
|
||||
.copied()
|
||||
.unwrap_or(0)
|
||||
}
|
||||
|
||||
/// Reset all retry tracking
|
||||
pub fn reset_retry_tracking(&self) {
|
||||
*self.gc_regions_retry_count.lock().unwrap() = HashMap::new();
|
||||
*self.gc_regions_error_sequence.lock().unwrap() = Vec::new();
|
||||
*self.gc_regions_success_after_retries.lock().unwrap() = HashMap::new();
|
||||
}
|
||||
|
||||
/// Set an error to be returned for a specific region
|
||||
pub fn set_gc_regions_error_for_region(&self, region_id: RegionId, error: crate::error::Error) {
|
||||
self.gc_regions_per_region_errors
|
||||
.lock()
|
||||
.unwrap()
|
||||
.insert(region_id, error);
|
||||
}
|
||||
|
||||
/// Clear per-region errors
|
||||
#[allow(unused)]
|
||||
pub fn clear_gc_regions_per_region_errors(&self) {
|
||||
@@ -213,39 +165,9 @@ impl SchedulerCtx for MockSchedulerCtx {
|
||||
.unwrap_or_else(|| (table_id, PhysicalTableRouteValue::default())))
|
||||
}
|
||||
|
||||
async fn get_file_references(
|
||||
&self,
|
||||
query_regions: &[RegionId],
|
||||
_related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
region_to_peer: &Region2Peers,
|
||||
_timeout: Duration,
|
||||
) -> Result<FileRefsManifest> {
|
||||
*self.get_file_references_calls.lock().unwrap() += 1;
|
||||
|
||||
// Check if we should return an injected error
|
||||
if let Some(error) = self.get_file_references_error.lock().unwrap().take() {
|
||||
return Err(error);
|
||||
}
|
||||
if query_regions
|
||||
.iter()
|
||||
.any(|region_id| !region_to_peer.contains_key(region_id))
|
||||
{
|
||||
UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"region_to_peer{region_to_peer:?} does not contain all region_ids requested: {:?}",
|
||||
query_regions
|
||||
),
|
||||
}.fail()?;
|
||||
}
|
||||
|
||||
Ok(self.file_refs.lock().unwrap().clone().unwrap_or_default())
|
||||
}
|
||||
|
||||
async fn gc_regions(
|
||||
&self,
|
||||
_peer: Peer,
|
||||
region_ids: &[RegionId],
|
||||
_file_refs_manifest: &FileRefsManifest,
|
||||
_full_file_listing: bool,
|
||||
_timeout: Duration,
|
||||
) -> Result<GcReport> {
|
||||
|
||||
@@ -152,7 +152,6 @@ async fn test_handle_tick() {
|
||||
);
|
||||
|
||||
assert_eq!(*ctx.get_table_to_region_stats_calls.lock().unwrap(), 1);
|
||||
assert_eq!(*ctx.get_file_references_calls.lock().unwrap(), 1);
|
||||
assert_eq!(*ctx.gc_regions_calls.lock().unwrap(), 1);
|
||||
|
||||
let tracker = scheduler.region_gc_tracker.lock().await;
|
||||
|
||||
@@ -64,6 +64,7 @@ async fn test_gc_regions_failure_handling() {
|
||||
region_id,
|
||||
HashSet::from([FileRef::new(region_id, FileId::random(), None)]),
|
||||
)]),
|
||||
cross_region_refs: HashMap::new(),
|
||||
};
|
||||
|
||||
let ctx = Arc::new(
|
||||
@@ -121,10 +122,6 @@ async fn test_gc_regions_failure_handling() {
|
||||
1,
|
||||
"Expected 1 call to get_table_to_region_stats"
|
||||
);
|
||||
assert!(
|
||||
*ctx.get_file_references_calls.lock().unwrap() >= 1,
|
||||
"Expected at least 1 call to get_file_references"
|
||||
);
|
||||
assert!(
|
||||
*ctx.gc_regions_calls.lock().unwrap() >= 1,
|
||||
"Expected at least 1 call to gc_regions"
|
||||
@@ -206,13 +203,6 @@ async fn test_get_file_references_failure() {
|
||||
datanode_report.deleted_files[®ion_id].is_empty(),
|
||||
"Should have empty deleted files due to file refs failure"
|
||||
);
|
||||
|
||||
// Should still attempt to get file references (may be called multiple times due to retry logic)
|
||||
assert!(
|
||||
*ctx.get_file_references_calls.lock().unwrap() >= 1,
|
||||
"Expected at least 1 call to get_file_references, got {}",
|
||||
*ctx.get_file_references_calls.lock().unwrap()
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
@@ -255,42 +245,22 @@ async fn test_get_table_route_failure() {
|
||||
last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())),
|
||||
};
|
||||
|
||||
// Get candidates first
|
||||
let stats = &ctx
|
||||
.table_to_region_stats
|
||||
.lock()
|
||||
.unwrap()
|
||||
.clone()
|
||||
.unwrap_or_default();
|
||||
let candidates = scheduler.select_gc_candidates(stats).await.unwrap();
|
||||
// Test the full workflow to trigger table route failure during aggregation
|
||||
// The table route failure should cause the entire GC cycle to fail
|
||||
let result = scheduler.handle_tick().await;
|
||||
|
||||
// Convert table-based candidates to datanode-based candidates
|
||||
let datanode_to_candidates = HashMap::from([(
|
||||
Peer::new(1, ""),
|
||||
candidates
|
||||
.into_iter()
|
||||
.flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c)))
|
||||
.collect(),
|
||||
)]);
|
||||
|
||||
// This should handle table route failure gracefully
|
||||
let report = scheduler
|
||||
.parallel_process_datanodes(datanode_to_candidates)
|
||||
.await;
|
||||
|
||||
// Should process the datanode but handle route error gracefully
|
||||
assert_eq!(
|
||||
report.per_datanode_reports.len(),
|
||||
0,
|
||||
"Expected 0 datanode report"
|
||||
);
|
||||
assert_eq!(
|
||||
report.failed_datanodes.len(),
|
||||
1,
|
||||
"Expected 1 failed datanodes (route error handled gracefully)"
|
||||
);
|
||||
// The table route failure should be propagated as an error
|
||||
assert!(
|
||||
report.failed_datanodes.contains_key(&1),
|
||||
"Failed datanodes should contain the datanode with route error"
|
||||
result.is_err(),
|
||||
"Expected table route failure to propagate as error"
|
||||
);
|
||||
|
||||
// Verify the error message contains our simulated failure
|
||||
let error = result.unwrap_err();
|
||||
let error_msg = format!("{}", error);
|
||||
assert!(
|
||||
error_msg.contains("Simulated table route failure for testing"),
|
||||
"Error message should contain our simulated failure: {}",
|
||||
error_msg
|
||||
);
|
||||
}
|
||||
|
||||
@@ -123,11 +123,6 @@ async fn test_full_gc_workflow() {
|
||||
1,
|
||||
"Expected 1 call to get_table_to_region_stats"
|
||||
);
|
||||
assert_eq!(
|
||||
*ctx.get_file_references_calls.lock().unwrap(),
|
||||
1,
|
||||
"Expected 1 call to get_file_references"
|
||||
);
|
||||
assert_eq!(
|
||||
*ctx.gc_regions_calls.lock().unwrap(),
|
||||
1,
|
||||
|
||||
@@ -19,6 +19,8 @@ use std::time::Duration;
|
||||
|
||||
use api::v1::meta::MailboxMessage;
|
||||
use common_meta::instruction::{self, GcRegions, GetFileRefs, GetFileRefsReply, InstructionReply};
|
||||
use common_meta::key::TableMetadataManagerRef;
|
||||
use common_meta::key::table_route::PhysicalTableRouteValue;
|
||||
use common_meta::lock_key::RegionLock;
|
||||
use common_meta::peer::Peer;
|
||||
use common_procedure::error::ToJsonSnafu;
|
||||
@@ -26,14 +28,16 @@ use common_procedure::{
|
||||
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
|
||||
Result as ProcedureResult, Status,
|
||||
};
|
||||
use common_telemetry::{debug, error, info, warn};
|
||||
use common_telemetry::{error, info, warn};
|
||||
use itertools::Itertools as _;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::ResultExt as _;
|
||||
use store_api::storage::{FileRefsManifest, GcReport, RegionId};
|
||||
use table::metadata::TableId;
|
||||
|
||||
use crate::error::{self, Result, SerializeToJsonSnafu};
|
||||
use crate::gc::Region2Peers;
|
||||
use crate::error::{self, KvBackendSnafu, Result, SerializeToJsonSnafu, TableMetadataManagerSnafu};
|
||||
use crate::gc::util::table_route_to_region;
|
||||
use crate::gc::{Peer2Regions, Region2Peers};
|
||||
use crate::handler::HeartbeatMailbox;
|
||||
use crate::service::mailbox::{Channel, MailboxRef};
|
||||
|
||||
@@ -146,56 +150,73 @@ async fn send_gc_regions(
|
||||
}
|
||||
}
|
||||
|
||||
/// TODO(discord9): another procedure which do both get file refs and gc regions.
|
||||
pub struct GcRegionProcedure {
|
||||
/// Procedure to perform get file refs then batch GC for multiple regions,
|
||||
/// it holds locks for all regions during the whole procedure.
|
||||
pub struct BatchGcProcedure {
|
||||
mailbox: MailboxRef,
|
||||
data: GcRegionData,
|
||||
table_metadata_manager: TableMetadataManagerRef,
|
||||
data: BatchGcData,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize)]
|
||||
pub struct GcRegionData {
|
||||
pub struct BatchGcData {
|
||||
state: State,
|
||||
/// Meta server address
|
||||
server_addr: String,
|
||||
peer: Peer,
|
||||
gc_regions: GcRegions,
|
||||
description: String,
|
||||
/// The regions to be GC-ed
|
||||
regions: Vec<RegionId>,
|
||||
full_file_listing: bool,
|
||||
region_routes: Region2Peers,
|
||||
/// Related regions (e.g., for shared files after repartition).
|
||||
/// The source regions (where those files originally came from) are used as the key, and the destination regions (where files are currently stored) are used as the value.
|
||||
related_regions: HashMap<RegionId, HashSet<RegionId>>,
|
||||
/// Acquired file references (Populated in Acquiring state)
|
||||
file_refs: FileRefsManifest,
|
||||
/// mailbox timeout duration
|
||||
timeout: Duration,
|
||||
gc_report: Option<GcReport>,
|
||||
}
|
||||
|
||||
impl GcRegionProcedure {
|
||||
pub const TYPE_NAME: &'static str = "metasrv-procedure::GcRegionProcedure";
|
||||
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
|
||||
pub enum State {
|
||||
/// Initial state
|
||||
Start,
|
||||
/// Fetching file references from datanodes
|
||||
Acquiring,
|
||||
/// Sending GC instruction to the target datanode
|
||||
Gcing,
|
||||
/// Updating region repartition info in kvbackend after GC based on the GC result
|
||||
UpdateRepartition,
|
||||
}
|
||||
|
||||
impl BatchGcProcedure {
|
||||
pub const TYPE_NAME: &'static str = "metasrv-procedure::BatchGcProcedure";
|
||||
|
||||
pub fn new(
|
||||
mailbox: MailboxRef,
|
||||
table_metadata_manager: TableMetadataManagerRef,
|
||||
server_addr: String,
|
||||
peer: Peer,
|
||||
gc_regions: GcRegions,
|
||||
description: String,
|
||||
regions: Vec<RegionId>,
|
||||
full_file_listing: bool,
|
||||
timeout: Duration,
|
||||
) -> Self {
|
||||
Self {
|
||||
mailbox,
|
||||
data: GcRegionData {
|
||||
peer,
|
||||
table_metadata_manager,
|
||||
data: BatchGcData {
|
||||
state: State::Start,
|
||||
server_addr,
|
||||
gc_regions,
|
||||
description,
|
||||
regions,
|
||||
full_file_listing,
|
||||
timeout,
|
||||
region_routes: HashMap::new(),
|
||||
related_regions: HashMap::new(),
|
||||
file_refs: FileRefsManifest::default(),
|
||||
gc_report: None,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
async fn send_gc_instr(&self) -> Result<GcReport> {
|
||||
send_gc_regions(
|
||||
&self.mailbox,
|
||||
&self.data.peer,
|
||||
self.data.gc_regions.clone(),
|
||||
&self.data.server_addr,
|
||||
self.data.timeout,
|
||||
&self.data.description,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub fn cast_result(res: Arc<dyn Any>) -> Result<GcReport> {
|
||||
res.downcast_ref::<GcReport>().cloned().ok_or_else(|| {
|
||||
error::UnexpectedSnafu {
|
||||
@@ -207,111 +228,129 @@ impl GcRegionProcedure {
|
||||
.build()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl Procedure for GcRegionProcedure {
|
||||
fn type_name(&self) -> &str {
|
||||
Self::TYPE_NAME
|
||||
}
|
||||
|
||||
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
|
||||
// Send GC instruction to the datanode. This procedure only handle lock&send, results or other kind of
|
||||
// errors will be reported back via the oneshot channel.
|
||||
let reply = self
|
||||
.send_gc_instr()
|
||||
async fn get_table_route(
|
||||
&self,
|
||||
table_id: TableId,
|
||||
) -> Result<(TableId, PhysicalTableRouteValue)> {
|
||||
self.table_metadata_manager
|
||||
.table_route_manager()
|
||||
.get_physical_table_route(table_id)
|
||||
.await
|
||||
.map_err(ProcedureError::external)?;
|
||||
|
||||
Ok(Status::done_with_output(reply))
|
||||
.context(TableMetadataManagerSnafu)
|
||||
}
|
||||
|
||||
fn dump(&self) -> ProcedureResult<String> {
|
||||
serde_json::to_string(&self.data).context(ToJsonSnafu)
|
||||
}
|
||||
|
||||
/// Read lock all regions involved in this GC procedure.
|
||||
/// So i.e. region migration won't happen during GC and cause race conditions.
|
||||
///
|
||||
/// only read lock the regions not catatlog/schema because it can run concurrently with other procedures(i.e. drop database/table)
|
||||
/// TODO:(discord9): integration test to verify this
|
||||
fn lock_key(&self) -> LockKey {
|
||||
let lock_key: Vec<_> = self
|
||||
.data
|
||||
.gc_regions
|
||||
.regions
|
||||
.iter()
|
||||
.sorted() // sort to have a deterministic lock order
|
||||
.map(|id| RegionLock::Read(*id).into())
|
||||
.collect();
|
||||
|
||||
LockKey::new(lock_key)
|
||||
}
|
||||
}
|
||||
|
||||
/// Procedure to perform get file refs then batch GC for multiple regions, should only be used by admin function
|
||||
/// for triggering manual gc, as it holds locks for too long and for all regions during the procedure.
|
||||
pub struct BatchGcProcedure {
|
||||
mailbox: MailboxRef,
|
||||
data: BatchGcData,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize)]
|
||||
pub struct BatchGcData {
|
||||
state: State,
|
||||
server_addr: String,
|
||||
/// The regions to be GC-ed
|
||||
regions: Vec<RegionId>,
|
||||
full_file_listing: bool,
|
||||
region_routes: Region2Peers,
|
||||
/// Related regions (e.g., for shared files). Map: RegionId -> List of related RegionIds.
|
||||
related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
/// Acquired file references (Populated in Acquiring state)
|
||||
file_refs: FileRefsManifest,
|
||||
/// mailbox timeout duration
|
||||
timeout: Duration,
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
|
||||
pub enum State {
|
||||
/// Initial state
|
||||
Start,
|
||||
/// Fetching file references from datanodes
|
||||
Acquiring,
|
||||
/// Sending GC instruction to the target datanode
|
||||
Gcing,
|
||||
}
|
||||
|
||||
impl BatchGcProcedure {
|
||||
pub const TYPE_NAME: &'static str = "metasrv-procedure::BatchGcProcedure";
|
||||
|
||||
pub fn new(
|
||||
mailbox: MailboxRef,
|
||||
server_addr: String,
|
||||
regions: Vec<RegionId>,
|
||||
full_file_listing: bool,
|
||||
region_routes: Region2Peers,
|
||||
related_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
timeout: Duration,
|
||||
) -> Self {
|
||||
Self {
|
||||
mailbox,
|
||||
data: BatchGcData {
|
||||
state: State::Start,
|
||||
server_addr,
|
||||
regions,
|
||||
full_file_listing,
|
||||
region_routes,
|
||||
related_regions,
|
||||
file_refs: FileRefsManifest::default(),
|
||||
timeout,
|
||||
},
|
||||
/// Return related regions for the given regions.
|
||||
/// The returned map uses the source regions (where those files originally came from) as the key,
|
||||
/// and the destination regions (where files are currently stored) as the value.
|
||||
/// If a region is not found in the repartition manager, the returned map still have this region as key,
|
||||
/// just empty value
|
||||
async fn find_related_regions(
|
||||
&self,
|
||||
regions: &[RegionId],
|
||||
) -> Result<HashMap<RegionId, HashSet<RegionId>>> {
|
||||
let repart_mgr = self.table_metadata_manager.table_repart_manager();
|
||||
let mut related_regions: HashMap<RegionId, HashSet<RegionId>> = HashMap::new();
|
||||
for src_region in regions {
|
||||
// TODO(discord9): batch get
|
||||
if let Some(dst_regions) = repart_mgr
|
||||
.get_dst_regions(*src_region)
|
||||
.await
|
||||
.context(KvBackendSnafu)?
|
||||
{
|
||||
related_regions.insert(*src_region, dst_regions.into_iter().collect());
|
||||
} else {
|
||||
related_regions.insert(*src_region, Default::default());
|
||||
}
|
||||
}
|
||||
Ok(related_regions)
|
||||
}
|
||||
|
||||
/// Clean up region repartition info in kvbackend after GC
|
||||
/// according to cross reference in `FileRefsManifest`.
|
||||
async fn cleanup_region_repartition(&self) -> Result<()> {
|
||||
for (src_region, dst_regions) in self.data.file_refs.cross_region_refs.iter() {
|
||||
// TODO(discord9): batch update
|
||||
self.table_metadata_manager
|
||||
.table_repart_manager()
|
||||
.update_mappings(*src_region, &dst_regions.iter().cloned().collect_vec())
|
||||
.await
|
||||
.context(KvBackendSnafu)?;
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Discover region routes for the given regions.
|
||||
async fn discover_route_for_regions(
|
||||
&self,
|
||||
regions: &[RegionId],
|
||||
) -> Result<(Region2Peers, Peer2Regions)> {
|
||||
let mut region_to_peer = HashMap::new();
|
||||
let mut peer_to_regions = HashMap::new();
|
||||
|
||||
// Group regions by table ID for batch processing
|
||||
let mut table_to_regions: HashMap<TableId, Vec<RegionId>> = HashMap::new();
|
||||
for region_id in regions {
|
||||
let table_id = region_id.table_id();
|
||||
table_to_regions
|
||||
.entry(table_id)
|
||||
.or_default()
|
||||
.push(*region_id);
|
||||
}
|
||||
|
||||
// Process each table's regions together for efficiency
|
||||
for (table_id, table_regions) in table_to_regions {
|
||||
match self.get_table_route(table_id).await {
|
||||
Ok((_phy_table_id, table_route)) => {
|
||||
table_route_to_region(
|
||||
&table_route,
|
||||
&table_regions,
|
||||
&mut region_to_peer,
|
||||
&mut peer_to_regions,
|
||||
);
|
||||
}
|
||||
Err(e) => {
|
||||
// Continue with other tables instead of failing completely
|
||||
// TODO(discord9): consider failing here instead
|
||||
warn!(
|
||||
"Failed to get table route for table {}: {}, skipping its regions",
|
||||
table_id, e
|
||||
);
|
||||
continue;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok((region_to_peer, peer_to_regions))
|
||||
}
|
||||
|
||||
/// Set region routes and related regions for GC procedure
|
||||
async fn set_routes_and_related_regions(&mut self) -> Result<()> {
|
||||
let related_regions = self.find_related_regions(&self.data.regions).await?;
|
||||
|
||||
self.data.related_regions = related_regions.clone();
|
||||
|
||||
// Discover routes for all regions involved in GC, including both the
|
||||
// primary GC regions and their related regions.
|
||||
let mut regions_set: HashSet<RegionId> = self.data.regions.iter().cloned().collect();
|
||||
|
||||
regions_set.extend(related_regions.keys().cloned());
|
||||
regions_set.extend(related_regions.values().flat_map(|v| v.iter()).cloned());
|
||||
|
||||
let regions_to_discover = regions_set.into_iter().collect_vec();
|
||||
|
||||
let (region_to_peer, _) = self
|
||||
.discover_route_for_regions(®ions_to_discover)
|
||||
.await?;
|
||||
|
||||
self.data.region_routes = region_to_peer;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Get file references from all datanodes that host the regions
|
||||
async fn get_file_references(&self) -> Result<FileRefsManifest> {
|
||||
use std::collections::{HashMap, HashSet};
|
||||
async fn get_file_references(&mut self) -> Result<FileRefsManifest> {
|
||||
self.set_routes_and_related_regions().await?;
|
||||
|
||||
let query_regions = &self.data.regions;
|
||||
let related_regions = &self.data.related_regions;
|
||||
@@ -344,20 +383,25 @@ impl BatchGcProcedure {
|
||||
}
|
||||
}
|
||||
|
||||
let mut datanode2related_regions: HashMap<Peer, HashMap<RegionId, Vec<RegionId>>> =
|
||||
let mut datanode2related_regions: HashMap<Peer, HashMap<RegionId, HashSet<RegionId>>> =
|
||||
HashMap::new();
|
||||
for (related_region, queries) in related_regions {
|
||||
if let Some((leader, _followers)) = region_routes.get(related_region) {
|
||||
datanode2related_regions
|
||||
.entry(leader.clone())
|
||||
.or_default()
|
||||
.insert(*related_region, queries.clone());
|
||||
} // since read from manifest, no need to send to followers
|
||||
for (src_region, dst_regions) in related_regions {
|
||||
for dst_region in dst_regions {
|
||||
if let Some((leader, _followers)) = region_routes.get(dst_region) {
|
||||
datanode2related_regions
|
||||
.entry(leader.clone())
|
||||
.or_default()
|
||||
.entry(*src_region)
|
||||
.or_default()
|
||||
.insert(*dst_region);
|
||||
} // since read from manifest, no need to send to followers
|
||||
}
|
||||
}
|
||||
|
||||
// Send GetFileRefs instructions to each datanode
|
||||
let mut all_file_refs: HashMap<RegionId, HashSet<_>> = HashMap::new();
|
||||
let mut all_manifest_versions = HashMap::new();
|
||||
let mut all_cross_region_refs = HashMap::new();
|
||||
|
||||
for (peer, regions) in datanode2query_regions {
|
||||
let related_regions_for_peer =
|
||||
@@ -400,17 +444,25 @@ impl BatchGcProcedure {
|
||||
let entry = all_manifest_versions.entry(region_id).or_insert(version);
|
||||
*entry = (*entry).min(version);
|
||||
}
|
||||
|
||||
for (region_id, related_region_ids) in reply.file_refs_manifest.cross_region_refs {
|
||||
let entry = all_cross_region_refs
|
||||
.entry(region_id)
|
||||
.or_insert_with(HashSet::new);
|
||||
entry.extend(related_region_ids);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(FileRefsManifest {
|
||||
file_refs: all_file_refs,
|
||||
manifest_version: all_manifest_versions,
|
||||
cross_region_refs: all_cross_region_refs,
|
||||
})
|
||||
}
|
||||
|
||||
/// Send GC instruction to all datanodes that host the regions,
|
||||
/// returns regions that need retry.
|
||||
async fn send_gc_instructions(&self) -> Result<Vec<RegionId>> {
|
||||
async fn send_gc_instructions(&self) -> Result<GcReport> {
|
||||
let regions = &self.data.regions;
|
||||
let region_routes = &self.data.region_routes;
|
||||
let file_refs = &self.data.file_refs;
|
||||
@@ -418,6 +470,7 @@ impl BatchGcProcedure {
|
||||
|
||||
// Group regions by datanode
|
||||
let mut datanode2regions: HashMap<Peer, Vec<RegionId>> = HashMap::new();
|
||||
let mut all_report = GcReport::default();
|
||||
|
||||
for region_id in regions {
|
||||
if let Some((leader, _followers)) = region_routes.get(region_id) {
|
||||
@@ -469,10 +522,15 @@ impl BatchGcProcedure {
|
||||
peer, success, need_retry
|
||||
);
|
||||
}
|
||||
all_need_retry.extend(report.need_retry_regions);
|
||||
all_need_retry.extend(report.need_retry_regions.clone());
|
||||
all_report.merge(report);
|
||||
}
|
||||
|
||||
Ok(all_need_retry.into_iter().collect())
|
||||
if !all_need_retry.is_empty() {
|
||||
warn!("Regions need retry after batch GC: {:?}", all_need_retry);
|
||||
}
|
||||
|
||||
Ok(all_report)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -507,12 +565,10 @@ impl Procedure for BatchGcProcedure {
|
||||
// Send GC instructions to all datanodes
|
||||
// TODO(discord9): handle need-retry regions
|
||||
match self.send_gc_instructions().await {
|
||||
Ok(_) => {
|
||||
info!(
|
||||
"Batch GC completed successfully for regions {:?}",
|
||||
self.data.regions
|
||||
);
|
||||
Ok(Status::done())
|
||||
Ok(report) => {
|
||||
self.data.state = State::UpdateRepartition;
|
||||
self.data.gc_report = Some(report);
|
||||
Ok(Status::executing(false))
|
||||
}
|
||||
Err(e) => {
|
||||
error!("Failed to send GC instructions: {}", e);
|
||||
@@ -520,6 +576,29 @@ impl Procedure for BatchGcProcedure {
|
||||
}
|
||||
}
|
||||
}
|
||||
State::UpdateRepartition => match self.cleanup_region_repartition().await {
|
||||
Ok(()) => {
|
||||
info!(
|
||||
"Cleanup region repartition info completed successfully for regions {:?}",
|
||||
self.data.regions
|
||||
);
|
||||
info!(
|
||||
"Batch GC completed successfully for regions {:?}",
|
||||
self.data.regions
|
||||
);
|
||||
let Some(report) = self.data.gc_report.take() else {
|
||||
return common_procedure::error::UnexpectedSnafu {
|
||||
err_msg: "GC report should be present after GC completion".to_string(),
|
||||
}
|
||||
.fail();
|
||||
};
|
||||
Ok(Status::done_with_output(report))
|
||||
}
|
||||
Err(e) => {
|
||||
error!("Failed to cleanup region repartition info: {}", e);
|
||||
Err(ProcedureError::external(e))
|
||||
}
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
@@ -38,29 +38,6 @@ pub struct GcJobReport {
|
||||
pub per_datanode_reports: HashMap<DatanodeId, GcReport>,
|
||||
pub failed_datanodes: HashMap<DatanodeId, Vec<Error>>,
|
||||
}
|
||||
impl GcJobReport {
|
||||
pub fn merge(&mut self, mut other: GcJobReport) {
|
||||
// merge per_datanode_reports&failed_datanodes
|
||||
for (dn_id, report) in other.per_datanode_reports {
|
||||
let self_report = self.per_datanode_reports.entry(dn_id).or_default();
|
||||
self_report.merge(report);
|
||||
}
|
||||
let all_failed_dn_ids = self
|
||||
.failed_datanodes
|
||||
.keys()
|
||||
.cloned()
|
||||
.chain(other.failed_datanodes.keys().cloned())
|
||||
.collect::<HashSet<_>>();
|
||||
for dn_id in all_failed_dn_ids {
|
||||
let entry = self.failed_datanodes.entry(dn_id).or_default();
|
||||
if let Some(other_errors) = other.failed_datanodes.remove(&dn_id) {
|
||||
entry.extend(other_errors);
|
||||
}
|
||||
}
|
||||
self.failed_datanodes
|
||||
.retain(|dn_id, _| !self.per_datanode_reports.contains_key(dn_id));
|
||||
}
|
||||
}
|
||||
|
||||
/// [`Event`] represents various types of events that can be processed by the gc ticker.
|
||||
///
|
||||
|
||||
@@ -30,15 +30,6 @@ pub(crate) struct RegionGcInfo {
|
||||
pub(crate) last_full_listing_time: Option<Instant>,
|
||||
}
|
||||
|
||||
impl RegionGcInfo {
|
||||
pub(crate) fn new(last_gc_time: Instant) -> Self {
|
||||
Self {
|
||||
last_gc_time,
|
||||
last_full_listing_time: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Tracks the last GC time for regions to implement cooldown.
|
||||
pub(crate) type RegionGcTracker = HashMap<RegionId, RegionGcInfo>;
|
||||
|
||||
@@ -46,7 +37,7 @@ impl GcScheduler {
|
||||
/// Clean up stale entries from the region GC tracker if enough time has passed.
|
||||
/// This removes entries for regions that no longer exist in the current table routes.
|
||||
pub(crate) async fn cleanup_tracker_if_needed(&self) -> Result<()> {
|
||||
let mut last_cleanup = *self.last_tracker_cleanup.lock().await;
|
||||
let last_cleanup = *self.last_tracker_cleanup.lock().await;
|
||||
let now = Instant::now();
|
||||
|
||||
// Check if enough time has passed since last cleanup
|
||||
@@ -85,25 +76,6 @@ impl GcScheduler {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
/// Determine if full file listing should be used for a region based on the last full listing time.
|
||||
pub(crate) async fn should_use_full_listing(&self, region_id: RegionId) -> bool {
|
||||
let gc_tracker = self.region_gc_tracker.lock().await;
|
||||
let now = Instant::now();
|
||||
|
||||
if let Some(gc_info) = gc_tracker.get(®ion_id) {
|
||||
if let Some(last_full_listing) = gc_info.last_full_listing_time {
|
||||
let elapsed = now.saturating_duration_since(last_full_listing);
|
||||
elapsed >= self.config.full_file_listing_interval
|
||||
} else {
|
||||
// Never did full listing for this region, do it now
|
||||
true
|
||||
}
|
||||
} else {
|
||||
// First time GC for this region, do full listing
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) async fn update_full_listing_time(
|
||||
&self,
|
||||
region_id: RegionId,
|
||||
|
||||
55
src/meta-srv/src/gc/util.rs
Normal file
55
src/meta-srv/src/gc/util.rs
Normal file
@@ -0,0 +1,55 @@
|
||||
// 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_meta::key::table_route::PhysicalTableRouteValue;
|
||||
use common_telemetry::warn;
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use crate::gc::{Peer2Regions, Region2Peers};
|
||||
|
||||
pub fn table_route_to_region(
|
||||
table_route: &PhysicalTableRouteValue,
|
||||
table_regions: &[RegionId],
|
||||
region_to_peer: &mut Region2Peers,
|
||||
peer_to_regions: &mut Peer2Regions,
|
||||
) {
|
||||
for ®ion_id in table_regions {
|
||||
let mut found = false;
|
||||
|
||||
// Find the region in the table route
|
||||
for region_route in &table_route.region_routes {
|
||||
if region_route.region.id == region_id
|
||||
&& let Some(leader_peer) = ®ion_route.leader_peer
|
||||
{
|
||||
region_to_peer.insert(
|
||||
region_id,
|
||||
(leader_peer.clone(), region_route.follower_peers.clone()),
|
||||
);
|
||||
peer_to_regions
|
||||
.entry(leader_peer.clone())
|
||||
.or_default()
|
||||
.insert(region_id);
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if !found {
|
||||
warn!(
|
||||
"Failed to find region {} in table route or no leader peer found",
|
||||
region_id,
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -70,6 +70,7 @@ use crate::metasrv::{
|
||||
use crate::peer::MetasrvPeerAllocator;
|
||||
use crate::procedure::region_migration::DefaultContextFactory;
|
||||
use crate::procedure::region_migration::manager::RegionMigrationManager;
|
||||
use crate::procedure::repartition::DefaultRepartitionProcedureFactory;
|
||||
use crate::procedure::wal_prune::Context as WalPruneContext;
|
||||
use crate::procedure::wal_prune::manager::{WalPruneManager, WalPruneTicker};
|
||||
use crate::region::flush_trigger::RegionFlushTrigger;
|
||||
@@ -400,8 +401,17 @@ impl MetasrvBuilder {
|
||||
region_failure_detector_controller,
|
||||
};
|
||||
let procedure_manager_c = procedure_manager.clone();
|
||||
let ddl_manager = DdlManager::try_new(ddl_context, procedure_manager_c, true)
|
||||
.context(error::InitDdlManagerSnafu)?;
|
||||
let repartition_procedure_factory = Arc::new(DefaultRepartitionProcedureFactory::new(
|
||||
mailbox.clone(),
|
||||
options.grpc.server_addr.clone(),
|
||||
));
|
||||
let ddl_manager = DdlManager::try_new(
|
||||
ddl_context,
|
||||
procedure_manager_c,
|
||||
repartition_procedure_factory,
|
||||
true,
|
||||
)
|
||||
.context(error::InitDdlManagerSnafu)?;
|
||||
|
||||
let ddl_manager = if let Some(configurator) = plugins
|
||||
.as_ref()
|
||||
|
||||
@@ -166,6 +166,8 @@ pub struct SubmitRegionMigrationTaskResult {
|
||||
pub peer_conflict: Vec<RegionId>,
|
||||
/// Regions whose table is not found.
|
||||
pub table_not_found: Vec<RegionId>,
|
||||
/// Regions whose table exists but region route is not found (e.g., removed after repartition).
|
||||
pub region_not_found: Vec<RegionId>,
|
||||
/// Regions still pending migration.
|
||||
pub migrating: Vec<RegionId>,
|
||||
/// Regions that have been submitted for migration.
|
||||
@@ -396,6 +398,7 @@ impl RegionMigrationManager {
|
||||
leader_changed,
|
||||
peer_conflict,
|
||||
mut table_not_found,
|
||||
region_not_found,
|
||||
pending,
|
||||
} = analyze_region_migration_task(&task, &self.context_factory.table_metadata_manager)
|
||||
.await?;
|
||||
@@ -405,6 +408,7 @@ impl RegionMigrationManager {
|
||||
leader_changed,
|
||||
peer_conflict,
|
||||
table_not_found,
|
||||
region_not_found,
|
||||
migrating: migrating_region_ids,
|
||||
submitted: vec![],
|
||||
procedure_id: None,
|
||||
@@ -445,6 +449,7 @@ impl RegionMigrationManager {
|
||||
leader_changed,
|
||||
peer_conflict,
|
||||
table_not_found,
|
||||
region_not_found,
|
||||
migrating: migrating_region_ids,
|
||||
submitted: vec![],
|
||||
procedure_id: None,
|
||||
@@ -460,6 +465,7 @@ impl RegionMigrationManager {
|
||||
leader_changed,
|
||||
peer_conflict,
|
||||
table_not_found,
|
||||
region_not_found,
|
||||
migrating: migrating_region_ids,
|
||||
submitted: submitting_region_ids,
|
||||
procedure_id: Some(procedure_id),
|
||||
|
||||
@@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::fmt::Display;
|
||||
use std::time::Duration;
|
||||
|
||||
@@ -82,13 +82,13 @@ impl RegionMigrationTaskBatch {
|
||||
/// Returns the table regions map.
|
||||
///
|
||||
/// The key is the table id, the value is the region ids of the table.
|
||||
pub(crate) fn table_regions(&self) -> HashMap<TableId, Vec<RegionId>> {
|
||||
pub(crate) fn table_regions(&self) -> HashMap<TableId, HashSet<RegionId>> {
|
||||
let mut table_regions = HashMap::new();
|
||||
for region_id in &self.region_ids {
|
||||
table_regions
|
||||
.entry(region_id.table_id())
|
||||
.or_insert_with(Vec::new)
|
||||
.push(*region_id);
|
||||
.or_insert_with(HashSet::new)
|
||||
.insert(*region_id);
|
||||
}
|
||||
table_regions
|
||||
}
|
||||
@@ -105,6 +105,8 @@ pub(crate) struct RegionMigrationAnalysis {
|
||||
pub(crate) peer_conflict: Vec<RegionId>,
|
||||
/// Regions whose table is not found.
|
||||
pub(crate) table_not_found: Vec<RegionId>,
|
||||
/// Regions whose table exists but region route is not found (e.g., removed after repartition).
|
||||
pub(crate) region_not_found: Vec<RegionId>,
|
||||
/// Regions still pending migration.
|
||||
pub(crate) pending: Vec<RegionId>,
|
||||
}
|
||||
@@ -209,6 +211,12 @@ pub async fn analyze_region_migration_task(
|
||||
err_msg: format!("TableRoute({table_id:?}) is a non-physical TableRouteValue."),
|
||||
}
|
||||
})?;
|
||||
|
||||
let existing_region_ids = region_routes
|
||||
.iter()
|
||||
.map(|r| r.region.id)
|
||||
.collect::<HashSet<_>>();
|
||||
|
||||
for region_route in region_routes
|
||||
.iter()
|
||||
.filter(|r| region_ids.contains(&r.region.id))
|
||||
@@ -220,6 +228,12 @@ pub async fn analyze_region_migration_task(
|
||||
task.to_peer.id,
|
||||
)?;
|
||||
}
|
||||
|
||||
for region_id in region_ids {
|
||||
if !existing_region_ids.contains(region_id) {
|
||||
result.region_not_found.push(*region_id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(result)
|
||||
@@ -460,6 +474,8 @@ mod tests {
|
||||
RegionId::new(1024, 2),
|
||||
RegionId::new(1024, 3),
|
||||
RegionId::new(1024, 4),
|
||||
// Region of existing table but route removed (e.g., after repartition).
|
||||
RegionId::new(1024, 5),
|
||||
RegionId::new(1025, 1),
|
||||
],
|
||||
from_peer: Peer::empty(1),
|
||||
@@ -477,6 +493,7 @@ mod tests {
|
||||
migrated: vec![RegionId::new(1024, 1)],
|
||||
leader_changed: vec![RegionId::new(1024, 2)],
|
||||
peer_conflict: vec![RegionId::new(1024, 3)],
|
||||
region_not_found: vec![RegionId::new(1024, 5)],
|
||||
table_not_found: vec![RegionId::new(1025, 1)],
|
||||
}
|
||||
);
|
||||
|
||||
@@ -23,28 +23,43 @@ pub mod repartition_start;
|
||||
pub mod utils;
|
||||
|
||||
use std::any::Any;
|
||||
use std::collections::HashMap;
|
||||
use std::fmt::Debug;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::cache_invalidator::CacheInvalidatorRef;
|
||||
use common_meta::ddl::DdlContext;
|
||||
use common_meta::ddl::allocator::region_routes::RegionRoutesAllocatorRef;
|
||||
use common_meta::ddl::allocator::wal_options::WalOptionsAllocatorRef;
|
||||
use common_meta::ddl_manager::RepartitionProcedureFactory;
|
||||
use common_meta::instruction::CacheIdent;
|
||||
use common_meta::key::datanode_table::RegionInfo;
|
||||
use common_meta::key::table_info::TableInfoValue;
|
||||
use common_meta::key::table_route::TableRouteValue;
|
||||
use common_meta::key::{DeserializedValueWithBytes, TableMetadataManagerRef};
|
||||
use common_meta::lock_key::{CatalogLock, SchemaLock, TableLock, TableNameLock};
|
||||
use common_meta::node_manager::NodeManagerRef;
|
||||
use common_meta::region_keeper::MemoryRegionKeeperRef;
|
||||
use common_meta::region_registry::LeaderRegionRegistryRef;
|
||||
use common_meta::rpc::router::RegionRoute;
|
||||
use common_procedure::{Context as ProcedureContext, Status};
|
||||
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
|
||||
use common_procedure::{
|
||||
BoxedProcedure, Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
|
||||
ProcedureManagerRef, Result as ProcedureResult, Status, StringKey, UserMetadata,
|
||||
};
|
||||
use common_telemetry::error;
|
||||
use partition::expr::PartitionExpr;
|
||||
use serde::{Deserialize, Serialize};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::storage::TableId;
|
||||
use store_api::storage::{RegionNumber, TableId};
|
||||
use table::table_name::TableName;
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::procedure::repartition::group::{
|
||||
Context as RepartitionGroupContext, RepartitionGroupProcedure,
|
||||
};
|
||||
use crate::procedure::repartition::plan::RepartitionPlanEntry;
|
||||
use crate::procedure::repartition::repartition_start::RepartitionStart;
|
||||
use crate::procedure::repartition::utils::get_datanode_table_value;
|
||||
use crate::service::mailbox::MailboxRef;
|
||||
|
||||
@@ -60,6 +75,35 @@ pub struct PersistentContext {
|
||||
pub plans: Vec<RepartitionPlanEntry>,
|
||||
}
|
||||
|
||||
impl PersistentContext {
|
||||
pub fn new(
|
||||
TableName {
|
||||
catalog_name,
|
||||
schema_name,
|
||||
table_name,
|
||||
}: TableName,
|
||||
table_id: TableId,
|
||||
) -> Self {
|
||||
Self {
|
||||
catalog_name,
|
||||
schema_name,
|
||||
table_name,
|
||||
table_id,
|
||||
plans: vec![],
|
||||
}
|
||||
}
|
||||
|
||||
pub fn lock_key(&self) -> Vec<StringKey> {
|
||||
vec![
|
||||
CatalogLock::Read(&self.catalog_name).into(),
|
||||
SchemaLock::read(&self.catalog_name, &self.schema_name).into(),
|
||||
TableLock::Write(self.table_id).into(),
|
||||
TableNameLock::new(&self.catalog_name, &self.schema_name, &self.table_name).into(),
|
||||
]
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Clone)]
|
||||
pub struct Context {
|
||||
pub persistent_ctx: PersistentContext,
|
||||
pub table_metadata_manager: TableMetadataManagerRef,
|
||||
@@ -74,6 +118,26 @@ pub struct Context {
|
||||
}
|
||||
|
||||
impl Context {
|
||||
pub fn new(
|
||||
ddl_ctx: &DdlContext,
|
||||
mailbox: MailboxRef,
|
||||
server_addr: String,
|
||||
persistent_ctx: PersistentContext,
|
||||
) -> Self {
|
||||
Self {
|
||||
persistent_ctx,
|
||||
table_metadata_manager: ddl_ctx.table_metadata_manager.clone(),
|
||||
memory_region_keeper: ddl_ctx.memory_region_keeper.clone(),
|
||||
node_manager: ddl_ctx.node_manager.clone(),
|
||||
leader_region_registry: ddl_ctx.leader_region_registry.clone(),
|
||||
mailbox,
|
||||
server_addr,
|
||||
cache_invalidator: ddl_ctx.cache_invalidator.clone(),
|
||||
region_routes_allocator: ddl_ctx.table_metadata_allocator.region_routes_allocator(),
|
||||
wal_options_allocator: ddl_ctx.table_metadata_allocator.wal_options_allocator(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Retrieves the table route value for the given table id.
|
||||
///
|
||||
/// Retry:
|
||||
@@ -135,6 +199,7 @@ impl Context {
|
||||
&self,
|
||||
current_table_route_value: &DeserializedValueWithBytes<TableRouteValue>,
|
||||
new_region_routes: Vec<RegionRoute>,
|
||||
new_region_wal_options: HashMap<RegionNumber, String>,
|
||||
) -> Result<()> {
|
||||
let table_id = self.persistent_ctx.table_id;
|
||||
if new_region_routes.is_empty() {
|
||||
@@ -158,6 +223,16 @@ impl Context {
|
||||
region_wal_options,
|
||||
..
|
||||
} = &datanode_table_value.region_info;
|
||||
|
||||
// Merge and validate the new region wal options.
|
||||
let validated_region_wal_options =
|
||||
crate::procedure::repartition::utils::merge_and_validate_region_wal_options(
|
||||
region_wal_options,
|
||||
new_region_wal_options,
|
||||
&new_region_routes,
|
||||
table_id,
|
||||
)?;
|
||||
|
||||
self.table_metadata_manager
|
||||
.update_table_route(
|
||||
table_id,
|
||||
@@ -165,7 +240,7 @@ impl Context {
|
||||
current_table_route_value,
|
||||
new_region_routes,
|
||||
region_options,
|
||||
region_wal_options,
|
||||
&validated_region_wal_options,
|
||||
)
|
||||
.await
|
||||
.context(error::TableMetadataManagerSnafu)
|
||||
@@ -207,3 +282,202 @@ pub(crate) trait State: Sync + Send + Debug {
|
||||
|
||||
fn as_any(&self) -> &dyn Any;
|
||||
}
|
||||
|
||||
pub struct RepartitionProcedure {
|
||||
state: Box<dyn State>,
|
||||
context: Context,
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize)]
|
||||
struct RepartitionData<'a> {
|
||||
state: &'a dyn State,
|
||||
persistent_ctx: &'a PersistentContext,
|
||||
}
|
||||
|
||||
#[derive(Debug, Deserialize)]
|
||||
struct RepartitionDataOwned {
|
||||
state: Box<dyn State>,
|
||||
persistent_ctx: PersistentContext,
|
||||
}
|
||||
|
||||
impl RepartitionProcedure {
|
||||
const TYPE_NAME: &'static str = "metasrv-procedure::Repartition";
|
||||
|
||||
pub fn new(
|
||||
from_exprs: Vec<PartitionExpr>,
|
||||
to_exprs: Vec<PartitionExpr>,
|
||||
context: Context,
|
||||
) -> Self {
|
||||
let state = Box::new(RepartitionStart::new(from_exprs, to_exprs));
|
||||
|
||||
Self { state, context }
|
||||
}
|
||||
|
||||
pub fn from_json<F>(json: &str, ctx_factory: F) -> ProcedureResult<Self>
|
||||
where
|
||||
F: FnOnce(PersistentContext) -> Context,
|
||||
{
|
||||
let RepartitionDataOwned {
|
||||
state,
|
||||
persistent_ctx,
|
||||
} = serde_json::from_str(json).context(FromJsonSnafu)?;
|
||||
let context = ctx_factory(persistent_ctx);
|
||||
|
||||
Ok(Self { state, context })
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl Procedure for RepartitionProcedure {
|
||||
fn type_name(&self) -> &str {
|
||||
Self::TYPE_NAME
|
||||
}
|
||||
|
||||
async fn execute(&mut self, _ctx: &ProcedureContext) -> ProcedureResult<Status> {
|
||||
let state = &mut self.state;
|
||||
match state.next(&mut self.context, _ctx).await {
|
||||
Ok((next, status)) => {
|
||||
*state = next;
|
||||
Ok(status)
|
||||
}
|
||||
Err(e) => {
|
||||
if e.is_retryable() {
|
||||
Err(ProcedureError::retry_later(e))
|
||||
} else {
|
||||
error!(
|
||||
e;
|
||||
"Repartition procedure failed, table id: {}",
|
||||
self.context.persistent_ctx.table_id,
|
||||
);
|
||||
Err(ProcedureError::external(e))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn rollback_supported(&self) -> bool {
|
||||
// TODO(weny): support rollback.
|
||||
false
|
||||
}
|
||||
|
||||
fn dump(&self) -> ProcedureResult<String> {
|
||||
let data = RepartitionData {
|
||||
state: self.state.as_ref(),
|
||||
persistent_ctx: &self.context.persistent_ctx,
|
||||
};
|
||||
serde_json::to_string(&data).context(ToJsonSnafu)
|
||||
}
|
||||
|
||||
fn lock_key(&self) -> LockKey {
|
||||
LockKey::new(self.context.persistent_ctx.lock_key())
|
||||
}
|
||||
|
||||
fn user_metadata(&self) -> Option<UserMetadata> {
|
||||
// TODO(weny): support user metadata.
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
pub struct DefaultRepartitionProcedureFactory {
|
||||
mailbox: MailboxRef,
|
||||
server_addr: String,
|
||||
}
|
||||
|
||||
impl DefaultRepartitionProcedureFactory {
|
||||
pub fn new(mailbox: MailboxRef, server_addr: String) -> Self {
|
||||
Self {
|
||||
mailbox,
|
||||
server_addr,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl RepartitionProcedureFactory for DefaultRepartitionProcedureFactory {
|
||||
fn create(
|
||||
&self,
|
||||
ddl_ctx: &DdlContext,
|
||||
table_name: TableName,
|
||||
table_id: TableId,
|
||||
from_exprs: Vec<String>,
|
||||
to_exprs: Vec<String>,
|
||||
) -> std::result::Result<BoxedProcedure, BoxedError> {
|
||||
let persistent_ctx = PersistentContext::new(table_name, table_id);
|
||||
let from_exprs = from_exprs
|
||||
.iter()
|
||||
.map(|e| {
|
||||
PartitionExpr::from_json_str(e)
|
||||
.context(error::DeserializePartitionExprSnafu)?
|
||||
.context(error::EmptyPartitionExprSnafu)
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.map_err(BoxedError::new)?;
|
||||
let to_exprs = to_exprs
|
||||
.iter()
|
||||
.map(|e| {
|
||||
PartitionExpr::from_json_str(e)
|
||||
.context(error::DeserializePartitionExprSnafu)?
|
||||
.context(error::EmptyPartitionExprSnafu)
|
||||
})
|
||||
.collect::<Result<Vec<_>>>()
|
||||
.map_err(BoxedError::new)?;
|
||||
|
||||
let procedure = RepartitionProcedure::new(
|
||||
from_exprs,
|
||||
to_exprs,
|
||||
Context::new(
|
||||
ddl_ctx,
|
||||
self.mailbox.clone(),
|
||||
self.server_addr.clone(),
|
||||
persistent_ctx,
|
||||
),
|
||||
);
|
||||
|
||||
Ok(Box::new(procedure))
|
||||
}
|
||||
|
||||
fn register_loaders(
|
||||
&self,
|
||||
ddl_ctx: &DdlContext,
|
||||
procedure_manager: &ProcedureManagerRef,
|
||||
) -> std::result::Result<(), BoxedError> {
|
||||
// Registers the repartition procedure loader.
|
||||
let mailbox = self.mailbox.clone();
|
||||
let server_addr = self.server_addr.clone();
|
||||
let moved_ddl_ctx = ddl_ctx.clone();
|
||||
procedure_manager
|
||||
.register_loader(
|
||||
RepartitionProcedure::TYPE_NAME,
|
||||
Box::new(move |json| {
|
||||
let mailbox = mailbox.clone();
|
||||
let server_addr = server_addr.clone();
|
||||
let ddl_ctx = moved_ddl_ctx.clone();
|
||||
let factory = move |persistent_ctx| {
|
||||
Context::new(&ddl_ctx, mailbox, server_addr, persistent_ctx)
|
||||
};
|
||||
RepartitionProcedure::from_json(json, factory).map(|p| Box::new(p) as _)
|
||||
}),
|
||||
)
|
||||
.map_err(BoxedError::new)?;
|
||||
|
||||
// Registers the repartition group procedure loader.
|
||||
let mailbox = self.mailbox.clone();
|
||||
let server_addr = self.server_addr.clone();
|
||||
let moved_ddl_ctx = ddl_ctx.clone();
|
||||
procedure_manager
|
||||
.register_loader(
|
||||
RepartitionGroupProcedure::TYPE_NAME,
|
||||
Box::new(move |json| {
|
||||
let mailbox = mailbox.clone();
|
||||
let server_addr = server_addr.clone();
|
||||
let ddl_ctx = moved_ddl_ctx.clone();
|
||||
let factory = move |persistent_ctx| {
|
||||
RepartitionGroupContext::new(&ddl_ctx, mailbox, server_addr, persistent_ctx)
|
||||
};
|
||||
RepartitionGroupProcedure::from_json(json, factory).map(|p| Box::new(p) as _)
|
||||
}),
|
||||
)
|
||||
.map_err(BoxedError::new)?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -119,7 +119,7 @@ impl State for AllocateRegion {
|
||||
let _guard = procedure_ctx.provider.acquire_lock(&table_lock).await;
|
||||
let new_region_routes =
|
||||
Self::generate_region_routes(region_routes, &new_allocated_region_routes);
|
||||
ctx.update_table_route(&table_route_value, new_region_routes)
|
||||
ctx.update_table_route(&table_route_value, new_region_routes, wal_options)
|
||||
.await?;
|
||||
ctx.invalidate_table_cache().await?;
|
||||
|
||||
|
||||
@@ -98,7 +98,7 @@ impl State for DeallocateRegion {
|
||||
let region_routes = table_route_value.region_routes().unwrap();
|
||||
let new_region_routes =
|
||||
Self::generate_region_routes(region_routes, &pending_deallocate_region_ids);
|
||||
ctx.update_table_route(&table_route_value, new_region_routes)
|
||||
ctx.update_table_route(&table_route_value, new_region_routes, HashMap::new())
|
||||
.await?;
|
||||
ctx.invalidate_table_cache().await?;
|
||||
|
||||
|
||||
@@ -27,6 +27,7 @@ use std::time::Duration;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::cache_invalidator::CacheInvalidatorRef;
|
||||
use common_meta::ddl::DdlContext;
|
||||
use common_meta::instruction::CacheIdent;
|
||||
use common_meta::key::datanode_table::{DatanodeTableValue, RegionInfo};
|
||||
use common_meta::key::table_route::TableRouteValue;
|
||||
@@ -34,7 +35,7 @@ use common_meta::key::{DeserializedValueWithBytes, TableMetadataManagerRef};
|
||||
use common_meta::lock_key::{CatalogLock, RegionLock, SchemaLock};
|
||||
use common_meta::peer::Peer;
|
||||
use common_meta::rpc::router::RegionRoute;
|
||||
use common_procedure::error::ToJsonSnafu;
|
||||
use common_procedure::error::{FromJsonSnafu, ToJsonSnafu};
|
||||
use common_procedure::{
|
||||
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
|
||||
Result as ProcedureResult, Status, StringKey, UserMetadata,
|
||||
@@ -60,8 +61,20 @@ pub struct RepartitionGroupProcedure {
|
||||
context: Context,
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize)]
|
||||
struct RepartitionGroupData<'a> {
|
||||
persistent_ctx: &'a PersistentContext,
|
||||
state: &'a dyn State,
|
||||
}
|
||||
|
||||
#[derive(Debug, Deserialize)]
|
||||
struct RepartitionGroupDataOwned {
|
||||
persistent_ctx: PersistentContext,
|
||||
state: Box<dyn State>,
|
||||
}
|
||||
|
||||
impl RepartitionGroupProcedure {
|
||||
const TYPE_NAME: &'static str = "metasrv-procedure::RepartitionGroup";
|
||||
pub(crate) const TYPE_NAME: &'static str = "metasrv-procedure::RepartitionGroup";
|
||||
|
||||
pub fn new(persistent_context: PersistentContext, context: &repartition::Context) -> Self {
|
||||
let state = Box::new(RepartitionStart);
|
||||
@@ -77,12 +90,19 @@ impl RepartitionGroupProcedure {
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize)]
|
||||
pub struct RepartitionGroupData<'a> {
|
||||
persistent_ctx: &'a PersistentContext,
|
||||
state: &'a dyn State,
|
||||
pub fn from_json<F>(json: &str, ctx_factory: F) -> ProcedureResult<Self>
|
||||
where
|
||||
F: FnOnce(PersistentContext) -> Context,
|
||||
{
|
||||
let RepartitionGroupDataOwned {
|
||||
state,
|
||||
persistent_ctx,
|
||||
} = serde_json::from_str(json).context(FromJsonSnafu)?;
|
||||
let context = ctx_factory(persistent_ctx);
|
||||
|
||||
Ok(Self { state, context })
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
@@ -149,6 +169,23 @@ pub struct Context {
|
||||
pub server_addr: String,
|
||||
}
|
||||
|
||||
impl Context {
|
||||
pub fn new(
|
||||
ddl_ctx: &DdlContext,
|
||||
mailbox: MailboxRef,
|
||||
server_addr: String,
|
||||
persistent_ctx: PersistentContext,
|
||||
) -> Self {
|
||||
Self {
|
||||
persistent_ctx,
|
||||
cache_invalidator: ddl_ctx.cache_invalidator.clone(),
|
||||
table_metadata_manager: ddl_ctx.table_metadata_manager.clone(),
|
||||
mailbox,
|
||||
server_addr,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// The result of the group preparation phase, containing validated region routes.
|
||||
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)]
|
||||
pub struct GroupPrepareResult {
|
||||
|
||||
@@ -31,7 +31,7 @@ use store_api::storage::RegionId;
|
||||
|
||||
use crate::error::{self, Error, Result};
|
||||
use crate::handler::HeartbeatMailbox;
|
||||
use crate::procedure::repartition::group::update_metadata::UpdateMetadata;
|
||||
use crate::procedure::repartition::group::repartition_end::RepartitionEnd;
|
||||
use crate::procedure::repartition::group::utils::{
|
||||
HandleMultipleResult, group_region_routes_by_peer, handle_multiple_results,
|
||||
};
|
||||
@@ -52,10 +52,7 @@ impl State for ApplyStagingManifest {
|
||||
) -> Result<(Box<dyn State>, Status)> {
|
||||
self.apply_staging_manifests(ctx).await?;
|
||||
|
||||
Ok((
|
||||
Box::new(UpdateMetadata::ApplyStaging),
|
||||
Status::executing(true),
|
||||
))
|
||||
Ok((Box::new(RepartitionEnd), Status::executing(true)))
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
@@ -63,13 +60,18 @@ impl State for ApplyStagingManifest {
|
||||
}
|
||||
}
|
||||
|
||||
struct ApplyStagingManifestInstructions {
|
||||
instructions: HashMap<Peer, Vec<common_meta::instruction::ApplyStagingManifest>>,
|
||||
central_region_instruction: Option<(Peer, common_meta::instruction::ApplyStagingManifest)>,
|
||||
}
|
||||
|
||||
impl ApplyStagingManifest {
|
||||
fn build_apply_staging_manifest_instructions(
|
||||
staging_manifest_paths: &HashMap<RegionId, String>,
|
||||
target_routes: &[RegionRoute],
|
||||
targets: &[RegionDescriptor],
|
||||
central_region_id: RegionId,
|
||||
) -> Result<HashMap<Peer, Vec<common_meta::instruction::ApplyStagingManifest>>> {
|
||||
) -> Result<ApplyStagingManifestInstructions> {
|
||||
let target_partition_expr_by_region = targets
|
||||
.iter()
|
||||
.map(|target| {
|
||||
@@ -86,7 +88,25 @@ impl ApplyStagingManifest {
|
||||
let target_region_routes_by_peer = group_region_routes_by_peer(target_routes);
|
||||
let mut instructions = HashMap::with_capacity(target_region_routes_by_peer.len());
|
||||
|
||||
for (peer, region_ids) in target_region_routes_by_peer {
|
||||
let mut central_region_instruction = None;
|
||||
for (peer, mut region_ids) in target_region_routes_by_peer {
|
||||
// If the central region is in the target region ids,
|
||||
// remove it and build the instruction for the central region.
|
||||
if region_ids.contains(¢ral_region_id) {
|
||||
region_ids.retain(|r| *r != central_region_id);
|
||||
central_region_instruction = Some((
|
||||
peer.clone(),
|
||||
common_meta::instruction::ApplyStagingManifest {
|
||||
region_id: central_region_id,
|
||||
partition_expr: target_partition_expr_by_region[¢ral_region_id].clone(),
|
||||
central_region_id,
|
||||
manifest_path: staging_manifest_paths[¢ral_region_id].clone(),
|
||||
},
|
||||
));
|
||||
if region_ids.is_empty() {
|
||||
continue;
|
||||
}
|
||||
}
|
||||
let apply_staging_manifests = region_ids
|
||||
.into_iter()
|
||||
.map(|region_id| common_meta::instruction::ApplyStagingManifest {
|
||||
@@ -99,7 +119,10 @@ impl ApplyStagingManifest {
|
||||
instructions.insert(peer.clone(), apply_staging_manifests);
|
||||
}
|
||||
|
||||
Ok(instructions)
|
||||
Ok(ApplyStagingManifestInstructions {
|
||||
instructions,
|
||||
central_region_instruction,
|
||||
})
|
||||
}
|
||||
|
||||
#[allow(dead_code)]
|
||||
@@ -112,7 +135,10 @@ impl ApplyStagingManifest {
|
||||
let targets = &ctx.persistent_ctx.targets;
|
||||
let target_routes = &prepare_result.target_routes;
|
||||
let central_region_id = prepare_result.central_region;
|
||||
let instructions = Self::build_apply_staging_manifest_instructions(
|
||||
let ApplyStagingManifestInstructions {
|
||||
instructions,
|
||||
central_region_instruction,
|
||||
} = Self::build_apply_staging_manifest_instructions(
|
||||
staging_manifest_paths,
|
||||
target_routes,
|
||||
targets,
|
||||
@@ -155,8 +181,10 @@ impl ApplyStagingManifest {
|
||||
let results = join_all(tasks).await;
|
||||
let result = handle_multiple_results(&results);
|
||||
match result {
|
||||
HandleMultipleResult::AllSuccessful => Ok(()),
|
||||
HandleMultipleResult::AllRetryable(retryable_errors) => error::RetryLaterSnafu {
|
||||
HandleMultipleResult::AllSuccessful => {
|
||||
// Coninute
|
||||
}
|
||||
HandleMultipleResult::AllRetryable(retryable_errors) => return error::RetryLaterSnafu {
|
||||
reason: format!(
|
||||
"All retryable errors during applying staging manifests for repartition table {}, group id {}: {:?}",
|
||||
table_id, group_id,
|
||||
@@ -168,7 +196,7 @@ impl ApplyStagingManifest {
|
||||
),
|
||||
}
|
||||
.fail(),
|
||||
HandleMultipleResult::AllNonRetryable(non_retryable_errors) => error::UnexpectedSnafu {
|
||||
HandleMultipleResult::AllNonRetryable(non_retryable_errors) => return error::UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"All non retryable errors during applying staging manifests for repartition table {}, group id {}: {:?}",
|
||||
table_id, group_id,
|
||||
@@ -183,7 +211,7 @@ impl ApplyStagingManifest {
|
||||
HandleMultipleResult::PartialRetryable {
|
||||
retryable_errors,
|
||||
non_retryable_errors,
|
||||
} => error::UnexpectedSnafu {
|
||||
} => return error::UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"Partial retryable errors during applying staging manifests for repartition table {}, group id {}: {:?}, non retryable errors: {:?}",
|
||||
table_id, group_id,
|
||||
@@ -201,6 +229,20 @@ impl ApplyStagingManifest {
|
||||
}
|
||||
.fail(),
|
||||
}
|
||||
|
||||
if let Some((peer, instruction)) = central_region_instruction {
|
||||
info!("Applying staging manifest for central region: {:?}", peer);
|
||||
Self::apply_staging_manifest(
|
||||
&ctx.mailbox,
|
||||
&ctx.server_addr,
|
||||
&peer,
|
||||
&[instruction],
|
||||
operation_timeout,
|
||||
)
|
||||
.await?;
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
async fn apply_staging_manifest(
|
||||
|
||||
@@ -128,14 +128,6 @@ impl RepartitionStart {
|
||||
central_region_datanode,
|
||||
})
|
||||
}
|
||||
|
||||
#[allow(dead_code)]
|
||||
fn next_state() -> (Box<dyn State>, Status) {
|
||||
(
|
||||
Box::new(UpdateMetadata::ApplyStaging),
|
||||
Status::executing(true),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
@@ -157,7 +149,10 @@ impl State for RepartitionStart {
|
||||
_procedure_ctx: &ProcedureContext,
|
||||
) -> Result<(Box<dyn State>, Status)> {
|
||||
if ctx.persistent_ctx.group_prepare_result.is_some() {
|
||||
return Ok(Self::next_state());
|
||||
return Ok((
|
||||
Box::new(UpdateMetadata::ApplyStaging),
|
||||
Status::executing(true),
|
||||
));
|
||||
}
|
||||
let table_id = ctx.persistent_ctx.table_id;
|
||||
let group_id = ctx.persistent_ctx.group_id;
|
||||
@@ -177,7 +172,10 @@ impl State for RepartitionStart {
|
||||
ctx.persistent_ctx.targets.len()
|
||||
);
|
||||
|
||||
Ok(Self::next_state())
|
||||
Ok((
|
||||
Box::new(UpdateMetadata::ApplyStaging),
|
||||
Status::executing(true),
|
||||
))
|
||||
}
|
||||
|
||||
fn as_any(&self) -> &dyn Any {
|
||||
|
||||
@@ -12,11 +12,14 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::{HashMap, HashSet};
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::key::TableMetadataManagerRef;
|
||||
use common_meta::key::datanode_table::{DatanodeTableKey, DatanodeTableValue};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::storage::TableId;
|
||||
use common_meta::rpc::router::RegionRoute;
|
||||
use snafu::{OptionExt, ResultExt, ensure};
|
||||
use store_api::storage::{RegionId, RegionNumber, TableId};
|
||||
|
||||
use crate::error::{self, Result};
|
||||
|
||||
@@ -47,3 +50,207 @@ pub async fn get_datanode_table_value(
|
||||
})?;
|
||||
Ok(datanode_table_value)
|
||||
}
|
||||
|
||||
/// Merges and validates region WAL options for repartition.
|
||||
///
|
||||
/// This function:
|
||||
/// 1. Validates that new WAL options don't overwrite existing ones
|
||||
/// 2. Merges existing `region_wal_options` with new `new_region_wal_options`
|
||||
/// 3. Filters out WAL options for regions that are not in `new_region_routes`
|
||||
/// 4. Validates that every region in `new_region_routes` has a corresponding WAL option
|
||||
///
|
||||
/// # Arguments
|
||||
/// * `region_wal_options` - Existing region WAL options from datanode table
|
||||
/// * `new_region_wal_options` - New region WAL options to merge (should only contain newly allocated regions)
|
||||
/// * `new_region_routes` - The new region routes after repartition
|
||||
/// * `table_id` - Table ID for error reporting
|
||||
///
|
||||
/// # Returns
|
||||
/// Returns the merged and filtered WAL options, ensuring all regions have options.
|
||||
///
|
||||
/// # Errors
|
||||
/// Returns an error if:
|
||||
/// - New WAL options try to overwrite existing ones for the same region
|
||||
/// - Any region in `new_region_routes` is missing a WAL option
|
||||
pub fn merge_and_validate_region_wal_options(
|
||||
region_wal_options: &HashMap<RegionNumber, String>,
|
||||
mut new_region_wal_options: HashMap<RegionNumber, String>,
|
||||
new_region_routes: &[RegionRoute],
|
||||
table_id: TableId,
|
||||
) -> Result<HashMap<RegionNumber, String>> {
|
||||
// Doesn't allow overwriting existing WAL options.
|
||||
for (region_number, _) in new_region_wal_options.iter() {
|
||||
if region_wal_options.contains_key(region_number) {
|
||||
return error::UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"Overwriting existing WAL option for region: {}",
|
||||
RegionId::new(table_id, *region_number)
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
}
|
||||
|
||||
new_region_wal_options.extend(region_wal_options.clone());
|
||||
|
||||
// Extract region numbers from new routes
|
||||
let region_numbers: HashSet<RegionNumber> = new_region_routes
|
||||
.iter()
|
||||
.map(|r| r.region.id.region_number())
|
||||
.collect();
|
||||
|
||||
// Filter out WAL options for regions that are not in new_region_routes
|
||||
new_region_wal_options.retain(|k, _| region_numbers.contains(k));
|
||||
|
||||
// Validate that every region has a WAL option
|
||||
ensure!(
|
||||
region_numbers.len() == new_region_wal_options.len(),
|
||||
error::UnexpectedSnafu {
|
||||
violated: format!(
|
||||
"Mismatch between number of region_numbers ({}) and new_region_wal_options ({}) for table: {}",
|
||||
region_numbers.len(),
|
||||
new_region_wal_options.len(),
|
||||
table_id
|
||||
),
|
||||
}
|
||||
);
|
||||
|
||||
Ok(new_region_wal_options)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use common_meta::peer::Peer;
|
||||
use common_meta::rpc::router::{Region, RegionRoute};
|
||||
use common_wal::options::{KafkaWalOptions, WalOptions};
|
||||
use store_api::storage::RegionId;
|
||||
|
||||
use super::*;
|
||||
|
||||
/// Helper function to create a Kafka WAL option string from a topic name.
|
||||
fn kafka_wal_option(topic: &str) -> String {
|
||||
serde_json::to_string(&WalOptions::Kafka(KafkaWalOptions {
|
||||
topic: topic.to_string(),
|
||||
}))
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
fn new_region_route(region_id: u64, datanode_id: u64) -> RegionRoute {
|
||||
RegionRoute {
|
||||
region: Region {
|
||||
id: RegionId::from_u64(region_id),
|
||||
..Default::default()
|
||||
},
|
||||
leader_peer: Some(Peer::empty(datanode_id)),
|
||||
follower_peers: vec![],
|
||||
leader_state: None,
|
||||
leader_down_since: None,
|
||||
}
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merge_and_validate_region_wal_options_success() {
|
||||
let table_id = 1;
|
||||
let existing_wal_options: HashMap<RegionNumber, String> = vec![
|
||||
(1, kafka_wal_option("topic_1")),
|
||||
(2, kafka_wal_option("topic_2")),
|
||||
]
|
||||
.into_iter()
|
||||
.collect();
|
||||
let new_wal_options: HashMap<RegionNumber, String> =
|
||||
vec![(3, kafka_wal_option("topic_3"))].into_iter().collect();
|
||||
let new_region_routes = vec![
|
||||
new_region_route(1, 1),
|
||||
new_region_route(2, 2),
|
||||
new_region_route(3, 3),
|
||||
];
|
||||
let result = merge_and_validate_region_wal_options(
|
||||
&existing_wal_options,
|
||||
new_wal_options,
|
||||
&new_region_routes,
|
||||
table_id,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// Should have all three regions
|
||||
assert_eq!(result.len(), 3);
|
||||
assert!(result.contains_key(&1));
|
||||
assert!(result.contains_key(&2));
|
||||
assert!(result.contains_key(&3));
|
||||
// Existing options should be preserved
|
||||
assert_eq!(result.get(&1).unwrap(), &kafka_wal_option("topic_1"));
|
||||
assert_eq!(result.get(&2).unwrap(), &kafka_wal_option("topic_2"));
|
||||
// New option should be present
|
||||
assert_eq!(result.get(&3).unwrap(), &kafka_wal_option("topic_3"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merge_and_validate_region_wal_options_new_overrides_existing() {
|
||||
let table_id = 1;
|
||||
let existing_wal_options: HashMap<RegionNumber, String> =
|
||||
vec![(1, kafka_wal_option("topic_1_old"))]
|
||||
.into_iter()
|
||||
.collect();
|
||||
let new_wal_options: HashMap<RegionNumber, String> =
|
||||
vec![(1, kafka_wal_option("topic_1_new"))]
|
||||
.into_iter()
|
||||
.collect();
|
||||
let new_region_routes = vec![new_region_route(1, 1)];
|
||||
merge_and_validate_region_wal_options(
|
||||
&existing_wal_options,
|
||||
new_wal_options,
|
||||
&new_region_routes,
|
||||
table_id,
|
||||
)
|
||||
.unwrap_err();
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merge_and_validate_region_wal_options_filters_removed_regions() {
|
||||
let table_id = 1;
|
||||
let existing_wal_options: HashMap<RegionNumber, String> = vec![
|
||||
(1, kafka_wal_option("topic_1")),
|
||||
(2, kafka_wal_option("topic_2")),
|
||||
(3, kafka_wal_option("topic_3")),
|
||||
]
|
||||
.into_iter()
|
||||
.collect();
|
||||
let new_wal_options = HashMap::new();
|
||||
// Only regions 1 and 2 are in new routes (region 3 removed)
|
||||
let new_region_routes = vec![new_region_route(1, 1), new_region_route(2, 2)];
|
||||
let result = merge_and_validate_region_wal_options(
|
||||
&existing_wal_options,
|
||||
new_wal_options,
|
||||
&new_region_routes,
|
||||
table_id,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// Should only have regions 1 and 2 (region 3 filtered out)
|
||||
assert_eq!(result.len(), 2);
|
||||
assert!(result.contains_key(&1));
|
||||
assert!(result.contains_key(&2));
|
||||
assert!(!result.contains_key(&3));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_merge_and_validate_region_wal_options_missing_option() {
|
||||
let table_id = 1;
|
||||
let existing_wal_options: HashMap<RegionNumber, String> =
|
||||
vec![(1, kafka_wal_option("topic_1"))].into_iter().collect();
|
||||
let new_wal_options = HashMap::new();
|
||||
// Region 2 is in routes but has no WAL option
|
||||
let new_region_routes = vec![new_region_route(1, 1), new_region_route(2, 2)];
|
||||
let result = merge_and_validate_region_wal_options(
|
||||
&existing_wal_options,
|
||||
new_wal_options,
|
||||
&new_region_routes,
|
||||
table_id,
|
||||
);
|
||||
// Should fail validation
|
||||
assert!(result.is_err());
|
||||
let error_msg = result.unwrap_err().to_string();
|
||||
assert!(error_msg.contains("Mismatch"));
|
||||
assert!(error_msg.contains(&table_id.to_string()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -744,6 +744,18 @@ impl RegionSupervisor {
|
||||
result.migrating
|
||||
);
|
||||
}
|
||||
if !result.region_not_found.is_empty() {
|
||||
let detecting_regions = result
|
||||
.region_not_found
|
||||
.iter()
|
||||
.map(|region_id| (from_peer_id, *region_id))
|
||||
.collect::<Vec<_>>();
|
||||
self.deregister_failure_detectors(detecting_regions).await;
|
||||
info!(
|
||||
"Region route not found, removed failover detectors for regions: {:?}",
|
||||
result.region_not_found
|
||||
);
|
||||
}
|
||||
if !result.table_not_found.is_empty() {
|
||||
let detecting_regions = result
|
||||
.table_not_found
|
||||
@@ -1222,6 +1234,34 @@ pub(crate) mod tests {
|
||||
assert!(supervisor.failover_counts.is_empty());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_handle_submit_region_migration_task_result_region_not_found() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let (mut supervisor, _) = new_test_supervisor();
|
||||
let region_id = RegionId::new(1, 1);
|
||||
let detecting_region = (1, region_id);
|
||||
supervisor
|
||||
.register_failure_detectors(vec![detecting_region])
|
||||
.await;
|
||||
supervisor.failover_counts.insert(detecting_region, 1);
|
||||
let result = SubmitRegionMigrationTaskResult {
|
||||
region_not_found: vec![region_id],
|
||||
..Default::default()
|
||||
};
|
||||
supervisor
|
||||
.handle_submit_region_migration_task_result(
|
||||
1,
|
||||
2,
|
||||
Duration::from_millis(1000),
|
||||
RegionMigrationTriggerReason::Manual,
|
||||
result,
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
assert!(!supervisor.failure_detector.contains(&detecting_region));
|
||||
assert!(supervisor.failover_counts.is_empty());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_handle_submit_region_migration_task_result_leader_changed() {
|
||||
common_telemetry::init_default_ut_logging();
|
||||
|
||||
@@ -42,6 +42,8 @@ use store_api::storage::{ConcreteDataType, FileId, RegionId, TimeSeriesRowSelect
|
||||
use crate::cache::cache_size::parquet_meta_size;
|
||||
use crate::cache::file_cache::{FileType, IndexKey};
|
||||
use crate::cache::index::inverted_index::{InvertedIndexCache, InvertedIndexCacheRef};
|
||||
#[cfg(feature = "vector_index")]
|
||||
use crate::cache::index::vector_index::{VectorIndexCache, VectorIndexCacheRef};
|
||||
use crate::cache::write_cache::WriteCacheRef;
|
||||
use crate::metrics::{CACHE_BYTES, CACHE_EVICTION, CACHE_HIT, CACHE_MISS};
|
||||
use crate::read::Batch;
|
||||
@@ -247,6 +249,16 @@ impl CacheStrategy {
|
||||
}
|
||||
}
|
||||
|
||||
/// Calls [CacheManager::vector_index_cache()].
|
||||
/// It returns None if the strategy is [CacheStrategy::Compaction] or [CacheStrategy::Disabled].
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub fn vector_index_cache(&self) -> Option<&VectorIndexCacheRef> {
|
||||
match self {
|
||||
CacheStrategy::EnableAll(cache_manager) => cache_manager.vector_index_cache(),
|
||||
CacheStrategy::Compaction(_) | CacheStrategy::Disabled => None,
|
||||
}
|
||||
}
|
||||
|
||||
/// Calls [CacheManager::puffin_metadata_cache()].
|
||||
/// It returns None if the strategy is [CacheStrategy::Compaction] or [CacheStrategy::Disabled].
|
||||
pub fn puffin_metadata_cache(&self) -> Option<&PuffinMetadataCacheRef> {
|
||||
@@ -303,6 +315,9 @@ pub struct CacheManager {
|
||||
inverted_index_cache: Option<InvertedIndexCacheRef>,
|
||||
/// Cache for bloom filter index.
|
||||
bloom_filter_index_cache: Option<BloomFilterIndexCacheRef>,
|
||||
/// Cache for vector index.
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_cache: Option<VectorIndexCacheRef>,
|
||||
/// Puffin metadata cache.
|
||||
puffin_metadata_cache: Option<PuffinMetadataCacheRef>,
|
||||
/// Cache for time series selectors.
|
||||
@@ -335,7 +350,10 @@ impl CacheManager {
|
||||
// Try to get metadata from write cache
|
||||
let key = IndexKey::new(file_id.region_id(), file_id.file_id(), FileType::Parquet);
|
||||
if let Some(write_cache) = &self.write_cache
|
||||
&& let Some(metadata) = write_cache.file_cache().get_parquet_meta_data(key).await
|
||||
&& let Some(metadata) = write_cache
|
||||
.file_cache()
|
||||
.get_parquet_meta_data(key, metrics)
|
||||
.await
|
||||
{
|
||||
metrics.file_cache_hit += 1;
|
||||
let metadata = Arc::new(metadata);
|
||||
@@ -434,6 +452,11 @@ impl CacheManager {
|
||||
cache.invalidate_file(file_id.file_id());
|
||||
}
|
||||
|
||||
#[cfg(feature = "vector_index")]
|
||||
if let Some(cache) = &self.vector_index_cache {
|
||||
cache.invalidate_file(file_id.file_id());
|
||||
}
|
||||
|
||||
if let Some(cache) = &self.puffin_metadata_cache {
|
||||
cache.remove(&file_id.to_string());
|
||||
}
|
||||
@@ -486,6 +509,11 @@ impl CacheManager {
|
||||
self.bloom_filter_index_cache.as_ref()
|
||||
}
|
||||
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub(crate) fn vector_index_cache(&self) -> Option<&VectorIndexCacheRef> {
|
||||
self.vector_index_cache.as_ref()
|
||||
}
|
||||
|
||||
pub(crate) fn puffin_metadata_cache(&self) -> Option<&PuffinMetadataCacheRef> {
|
||||
self.puffin_metadata_cache.as_ref()
|
||||
}
|
||||
@@ -646,6 +674,9 @@ impl CacheManagerBuilder {
|
||||
self.index_content_size,
|
||||
self.index_content_page_size,
|
||||
);
|
||||
#[cfg(feature = "vector_index")]
|
||||
let vector_index_cache = (self.index_content_size != 0)
|
||||
.then(|| Arc::new(VectorIndexCache::new(self.index_content_size)));
|
||||
let index_result_cache = (self.index_result_cache_size != 0)
|
||||
.then(|| IndexResultCache::new(self.index_result_cache_size));
|
||||
let puffin_metadata_cache =
|
||||
@@ -672,6 +703,8 @@ impl CacheManagerBuilder {
|
||||
write_cache: self.write_cache,
|
||||
inverted_index_cache: Some(Arc::new(inverted_index_cache)),
|
||||
bloom_filter_index_cache: Some(Arc::new(bloom_filter_index_cache)),
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_cache,
|
||||
puffin_metadata_cache: Some(Arc::new(puffin_metadata_cache)),
|
||||
selector_result_cache,
|
||||
index_result_cache,
|
||||
|
||||
9
src/mito2/src/cache/file_cache.rs
vendored
9
src/mito2/src/cache/file_cache.rs
vendored
@@ -43,6 +43,7 @@ use crate::metrics::{
|
||||
use crate::region::opener::RegionLoadCacheTask;
|
||||
use crate::sst::parquet::helper::fetch_byte_ranges;
|
||||
use crate::sst::parquet::metadata::MetadataLoader;
|
||||
use crate::sst::parquet::reader::MetadataCacheMetrics;
|
||||
|
||||
/// Subdirectory of cached files for write.
|
||||
///
|
||||
@@ -566,7 +567,11 @@ impl FileCache {
|
||||
|
||||
/// Get the parquet metadata in file cache.
|
||||
/// If the file is not in the cache or fail to load metadata, return None.
|
||||
pub(crate) async fn get_parquet_meta_data(&self, key: IndexKey) -> Option<ParquetMetaData> {
|
||||
pub(crate) async fn get_parquet_meta_data(
|
||||
&self,
|
||||
key: IndexKey,
|
||||
cache_metrics: &mut MetadataCacheMetrics,
|
||||
) -> Option<ParquetMetaData> {
|
||||
// Check if file cache contains the key
|
||||
if let Some(index_value) = self.inner.parquet_index.get(&key).await {
|
||||
// Load metadata from file cache
|
||||
@@ -575,7 +580,7 @@ impl FileCache {
|
||||
let file_size = index_value.file_size as u64;
|
||||
let metadata_loader = MetadataLoader::new(local_store, &file_path, file_size);
|
||||
|
||||
match metadata_loader.load().await {
|
||||
match metadata_loader.load(cache_metrics).await {
|
||||
Ok(metadata) => {
|
||||
CACHE_HIT
|
||||
.with_label_values(&[key.file_type.metric_label()])
|
||||
|
||||
2
src/mito2/src/cache/index.rs
vendored
2
src/mito2/src/cache/index.rs
vendored
@@ -15,6 +15,8 @@
|
||||
pub mod bloom_filter_index;
|
||||
pub mod inverted_index;
|
||||
pub mod result_cache;
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub mod vector_index;
|
||||
|
||||
use std::future::Future;
|
||||
use std::hash::Hash;
|
||||
|
||||
137
src/mito2/src/cache/index/vector_index.rs
vendored
Normal file
137
src/mito2/src/cache/index/vector_index.rs
vendored
Normal file
@@ -0,0 +1,137 @@
|
||||
// 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 moka::notification::RemovalCause;
|
||||
use moka::sync::Cache;
|
||||
use roaring::RoaringBitmap;
|
||||
use store_api::storage::{ColumnId, FileId, IndexVersion, VectorDistanceMetric, VectorIndexEngine};
|
||||
|
||||
use crate::metrics::{CACHE_BYTES, CACHE_EVICTION};
|
||||
|
||||
const VECTOR_INDEX_CACHE_TYPE: &str = "vector_index";
|
||||
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
|
||||
pub struct VectorIndexCacheKey {
|
||||
file_id: FileId,
|
||||
index_version: IndexVersion,
|
||||
column_id: ColumnId,
|
||||
}
|
||||
|
||||
impl VectorIndexCacheKey {
|
||||
pub fn new(file_id: FileId, index_version: IndexVersion, column_id: ColumnId) -> Self {
|
||||
Self {
|
||||
file_id,
|
||||
index_version,
|
||||
column_id,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub struct CachedVectorIndex {
|
||||
pub engine: Box<dyn VectorIndexEngine>,
|
||||
pub null_bitmap: RoaringBitmap,
|
||||
pub size_bytes: usize,
|
||||
pub dimensions: u32,
|
||||
pub metric: VectorDistanceMetric,
|
||||
pub total_rows: u64,
|
||||
pub indexed_rows: u64,
|
||||
}
|
||||
|
||||
impl CachedVectorIndex {
|
||||
pub fn new(
|
||||
engine: Box<dyn VectorIndexEngine>,
|
||||
null_bitmap: RoaringBitmap,
|
||||
dimensions: u32,
|
||||
metric: VectorDistanceMetric,
|
||||
total_rows: u64,
|
||||
indexed_rows: u64,
|
||||
) -> Self {
|
||||
let size_bytes =
|
||||
engine.memory_usage() + null_bitmap.serialized_size() + std::mem::size_of::<Self>();
|
||||
Self {
|
||||
engine,
|
||||
null_bitmap,
|
||||
size_bytes,
|
||||
dimensions,
|
||||
metric,
|
||||
total_rows,
|
||||
indexed_rows,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for CachedVectorIndex {
|
||||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
f.debug_struct("CachedVectorIndex")
|
||||
.field("size_bytes", &self.size_bytes)
|
||||
.field("dimensions", &self.dimensions)
|
||||
.field("metric", &self.metric)
|
||||
.field("total_rows", &self.total_rows)
|
||||
.field("indexed_rows", &self.indexed_rows)
|
||||
.field("null_bitmap_len", &self.null_bitmap.len())
|
||||
.finish()
|
||||
}
|
||||
}
|
||||
|
||||
pub struct VectorIndexCache {
|
||||
inner: Cache<VectorIndexCacheKey, Arc<CachedVectorIndex>>,
|
||||
}
|
||||
|
||||
pub type VectorIndexCacheRef = Arc<VectorIndexCache>;
|
||||
|
||||
impl VectorIndexCache {
|
||||
pub fn new(capacity: u64) -> Self {
|
||||
fn to_str(cause: RemovalCause) -> &'static str {
|
||||
match cause {
|
||||
RemovalCause::Expired => "expired",
|
||||
RemovalCause::Explicit => "explicit",
|
||||
RemovalCause::Replaced => "replaced",
|
||||
RemovalCause::Size => "size",
|
||||
}
|
||||
}
|
||||
|
||||
let inner = Cache::builder()
|
||||
.max_capacity(capacity)
|
||||
.weigher(|_k, v: &Arc<CachedVectorIndex>| v.size_bytes as u32)
|
||||
.eviction_listener(|_k, v, cause| {
|
||||
CACHE_BYTES
|
||||
.with_label_values(&[VECTOR_INDEX_CACHE_TYPE])
|
||||
.sub(v.size_bytes as i64);
|
||||
CACHE_EVICTION
|
||||
.with_label_values(&[VECTOR_INDEX_CACHE_TYPE, to_str(cause)])
|
||||
.inc();
|
||||
})
|
||||
.build();
|
||||
Self { inner }
|
||||
}
|
||||
|
||||
pub fn get(&self, key: &VectorIndexCacheKey) -> Option<Arc<CachedVectorIndex>> {
|
||||
self.inner.get(key)
|
||||
}
|
||||
|
||||
pub fn insert(&self, key: VectorIndexCacheKey, value: Arc<CachedVectorIndex>) {
|
||||
CACHE_BYTES
|
||||
.with_label_values(&[VECTOR_INDEX_CACHE_TYPE])
|
||||
.add(value.size_bytes as i64);
|
||||
self.inner.insert(key, value);
|
||||
}
|
||||
|
||||
pub fn invalidate_file(&self, file_id: FileId) {
|
||||
let _ = self
|
||||
.inner
|
||||
.invalidate_entries_if(move |k, _| k.file_id == file_id);
|
||||
}
|
||||
}
|
||||
@@ -81,7 +81,7 @@ mod apply_staging_manifest_test;
|
||||
mod puffin_index;
|
||||
|
||||
use std::any::Any;
|
||||
use std::collections::HashMap;
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::sync::Arc;
|
||||
use std::time::Instant;
|
||||
|
||||
@@ -303,7 +303,7 @@ impl MitoEngine {
|
||||
pub async fn get_snapshot_of_file_refs(
|
||||
&self,
|
||||
file_handle_regions: impl IntoIterator<Item = RegionId>,
|
||||
manifest_regions: HashMap<RegionId, Vec<RegionId>>,
|
||||
related_regions: HashMap<RegionId, HashSet<RegionId>>,
|
||||
) -> Result<FileRefsManifest> {
|
||||
let file_ref_mgr = self.file_ref_manager();
|
||||
|
||||
@@ -315,15 +315,30 @@ impl MitoEngine {
|
||||
.filter_map(|region_id| self.find_region(region_id))
|
||||
.collect();
|
||||
|
||||
let related_regions: Vec<(MitoRegionRef, Vec<RegionId>)> = manifest_regions
|
||||
.into_iter()
|
||||
.filter_map(|(related_region, queries)| {
|
||||
self.find_region(related_region).map(|r| (r, queries))
|
||||
})
|
||||
.collect();
|
||||
let dst_region_to_src_regions: Vec<(MitoRegionRef, HashSet<RegionId>)> = {
|
||||
let dst2src = related_regions
|
||||
.into_iter()
|
||||
.flat_map(|(src, dsts)| dsts.into_iter().map(move |dst| (dst, src)))
|
||||
.fold(
|
||||
HashMap::<RegionId, HashSet<RegionId>>::new(),
|
||||
|mut acc, (k, v)| {
|
||||
let entry = acc.entry(k).or_default();
|
||||
entry.insert(v);
|
||||
acc
|
||||
},
|
||||
);
|
||||
let mut dst_region_to_src_regions = Vec::with_capacity(dst2src.len());
|
||||
for (dst_region, srcs) in dst2src {
|
||||
let Some(dst_region) = self.find_region(dst_region) else {
|
||||
continue;
|
||||
};
|
||||
dst_region_to_src_regions.push((dst_region, srcs));
|
||||
}
|
||||
dst_region_to_src_regions
|
||||
};
|
||||
|
||||
file_ref_mgr
|
||||
.get_snapshot_of_file_refs(query_regions, related_regions)
|
||||
.get_snapshot_of_file_refs(query_regions, dst_region_to_src_regions)
|
||||
.await
|
||||
}
|
||||
|
||||
|
||||
@@ -652,6 +652,14 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[cfg(feature = "vector_index")]
|
||||
#[snafu(display("Failed to apply vector index: {}", reason))]
|
||||
ApplyVectorIndex {
|
||||
reason: String,
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to push index value"))]
|
||||
PushIndexValue {
|
||||
source: index::inverted_index::error::Error,
|
||||
@@ -1324,6 +1332,8 @@ impl ErrorExt for Error {
|
||||
| PushIndexValue { source, .. }
|
||||
| ApplyInvertedIndex { source, .. }
|
||||
| IndexFinish { source, .. } => source.status_code(),
|
||||
#[cfg(feature = "vector_index")]
|
||||
ApplyVectorIndex { .. } => StatusCode::Internal,
|
||||
PuffinReadBlob { source, .. }
|
||||
| PuffinAddBlob { source, .. }
|
||||
| PuffinInitStager { source, .. }
|
||||
|
||||
@@ -134,6 +134,7 @@ async fn test_gc_worker_basic_truncate() {
|
||||
let file_ref_manifest = FileRefsManifest {
|
||||
file_refs: Default::default(),
|
||||
manifest_version: [(region_id, version)].into(),
|
||||
cross_region_refs: HashMap::new(),
|
||||
};
|
||||
let gc_worker = create_gc_worker(&engine, regions, &file_ref_manifest, true).await;
|
||||
let report = gc_worker.run().await.unwrap();
|
||||
@@ -232,6 +233,7 @@ async fn test_gc_worker_truncate_with_ref() {
|
||||
)]
|
||||
.into(),
|
||||
manifest_version: [(region_id, version)].into(),
|
||||
cross_region_refs: HashMap::new(),
|
||||
};
|
||||
let gc_worker = create_gc_worker(&engine, regions, &file_ref_manifest, true).await;
|
||||
let report = gc_worker.run().await.unwrap();
|
||||
@@ -313,6 +315,7 @@ async fn test_gc_worker_basic_compact() {
|
||||
let file_ref_manifest = FileRefsManifest {
|
||||
file_refs: Default::default(),
|
||||
manifest_version: [(region_id, version)].into(),
|
||||
cross_region_refs: HashMap::new(),
|
||||
};
|
||||
|
||||
let gc_worker = create_gc_worker(&engine, regions, &file_ref_manifest, true).await;
|
||||
@@ -399,6 +402,7 @@ async fn test_gc_worker_compact_with_ref() {
|
||||
.collect(),
|
||||
)]),
|
||||
manifest_version: [(region_id, version)].into(),
|
||||
cross_region_refs: HashMap::new(),
|
||||
};
|
||||
|
||||
let gc_worker = create_gc_worker(&engine, regions, &file_ref_manifest, true).await;
|
||||
|
||||
@@ -1058,10 +1058,6 @@ impl<T: BatchReader + ?Sized> BatchReader for Box<T> {
|
||||
/// Local metrics for scanners.
|
||||
#[derive(Debug, Default)]
|
||||
pub(crate) struct ScannerMetrics {
|
||||
/// Duration to prepare the scan task.
|
||||
prepare_scan_cost: Duration,
|
||||
/// Duration to build the (merge) reader.
|
||||
build_reader_cost: Duration,
|
||||
/// Duration to scan data.
|
||||
scan_cost: Duration,
|
||||
/// Duration while waiting for `yield`.
|
||||
@@ -1070,10 +1066,6 @@ pub(crate) struct ScannerMetrics {
|
||||
num_batches: usize,
|
||||
/// Number of rows returned.
|
||||
num_rows: usize,
|
||||
/// Number of mem ranges scanned.
|
||||
num_mem_ranges: usize,
|
||||
/// Number of file ranges scanned.
|
||||
num_file_ranges: usize,
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
|
||||
@@ -70,11 +70,15 @@ use crate::sst::index::fulltext_index::applier::FulltextIndexApplierRef;
|
||||
use crate::sst::index::fulltext_index::applier::builder::FulltextIndexApplierBuilder;
|
||||
use crate::sst::index::inverted_index::applier::InvertedIndexApplierRef;
|
||||
use crate::sst::index::inverted_index::applier::builder::InvertedIndexApplierBuilder;
|
||||
#[cfg(feature = "vector_index")]
|
||||
use crate::sst::index::vector_index::applier::{VectorIndexApplier, VectorIndexApplierRef};
|
||||
use crate::sst::parquet::file_range::PreFilterMode;
|
||||
use crate::sst::parquet::reader::ReaderMetrics;
|
||||
|
||||
/// Parallel scan channel size for flat format.
|
||||
const FLAT_SCAN_CHANNEL_SIZE: usize = 2;
|
||||
#[cfg(feature = "vector_index")]
|
||||
const VECTOR_INDEX_OVERFETCH_MULTIPLIER: usize = 2;
|
||||
|
||||
/// A scanner scans a region and returns a [SendableRecordBatchStream].
|
||||
pub(crate) enum Scanner {
|
||||
@@ -498,6 +502,16 @@ impl ScanRegion {
|
||||
self.build_fulltext_index_applier(&non_field_filters),
|
||||
self.build_fulltext_index_applier(&field_filters),
|
||||
];
|
||||
#[cfg(feature = "vector_index")]
|
||||
let vector_index_applier = self.build_vector_index_applier();
|
||||
#[cfg(feature = "vector_index")]
|
||||
let vector_index_k = self.request.vector_search.as_ref().map(|search| {
|
||||
if self.request.filters.is_empty() {
|
||||
search.k
|
||||
} else {
|
||||
search.k.saturating_mul(VECTOR_INDEX_OVERFETCH_MULTIPLIER)
|
||||
}
|
||||
});
|
||||
let predicate = PredicateGroup::new(&self.version.metadata, &self.request.filters)?;
|
||||
|
||||
if flat_format {
|
||||
@@ -523,6 +537,10 @@ impl ScanRegion {
|
||||
.with_series_row_selector(self.request.series_row_selector)
|
||||
.with_distribution(self.request.distribution)
|
||||
.with_flat_format(flat_format);
|
||||
#[cfg(feature = "vector_index")]
|
||||
let input = input
|
||||
.with_vector_index_applier(vector_index_applier)
|
||||
.with_vector_index_k(vector_index_k);
|
||||
|
||||
#[cfg(feature = "enterprise")]
|
||||
let input = if let Some(provider) = self.extension_range_provider {
|
||||
@@ -667,6 +685,31 @@ impl ScanRegion {
|
||||
.flatten()
|
||||
.map(Arc::new)
|
||||
}
|
||||
|
||||
/// Build the vector index applier from vector search request.
|
||||
#[cfg(feature = "vector_index")]
|
||||
fn build_vector_index_applier(&self) -> Option<VectorIndexApplierRef> {
|
||||
let vector_search = self.request.vector_search.as_ref()?;
|
||||
|
||||
let file_cache = self.cache_strategy.write_cache().map(|w| w.file_cache());
|
||||
let puffin_metadata_cache = self.cache_strategy.puffin_metadata_cache().cloned();
|
||||
let vector_index_cache = self.cache_strategy.vector_index_cache().cloned();
|
||||
|
||||
let applier = VectorIndexApplier::new(
|
||||
self.access_layer.table_dir().to_string(),
|
||||
self.access_layer.path_type(),
|
||||
self.access_layer.object_store().clone(),
|
||||
self.access_layer.puffin_manager_factory().clone(),
|
||||
vector_search.column_id,
|
||||
vector_search.query_vector.clone(),
|
||||
vector_search.metric,
|
||||
)
|
||||
.with_file_cache(file_cache)
|
||||
.with_puffin_metadata_cache(puffin_metadata_cache)
|
||||
.with_vector_index_cache(vector_index_cache);
|
||||
|
||||
Some(Arc::new(applier))
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if the time range of a SST `file` matches the `predicate`.
|
||||
@@ -708,6 +751,12 @@ pub struct ScanInput {
|
||||
inverted_index_appliers: [Option<InvertedIndexApplierRef>; 2],
|
||||
bloom_filter_index_appliers: [Option<BloomFilterIndexApplierRef>; 2],
|
||||
fulltext_index_appliers: [Option<FulltextIndexApplierRef>; 2],
|
||||
/// Vector index applier for KNN search.
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub(crate) vector_index_applier: Option<VectorIndexApplierRef>,
|
||||
/// Over-fetched k for vector index scan.
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub(crate) vector_index_k: Option<usize>,
|
||||
/// Start time of the query.
|
||||
pub(crate) query_start: Option<Instant>,
|
||||
/// The region is using append mode.
|
||||
@@ -747,6 +796,10 @@ impl ScanInput {
|
||||
inverted_index_appliers: [None, None],
|
||||
bloom_filter_index_appliers: [None, None],
|
||||
fulltext_index_appliers: [None, None],
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_applier: None,
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_k: None,
|
||||
query_start: None,
|
||||
append_mode: false,
|
||||
filter_deleted: true,
|
||||
@@ -853,6 +906,25 @@ impl ScanInput {
|
||||
self
|
||||
}
|
||||
|
||||
/// Sets vector index applier for KNN search.
|
||||
#[cfg(feature = "vector_index")]
|
||||
#[must_use]
|
||||
pub(crate) fn with_vector_index_applier(
|
||||
mut self,
|
||||
applier: Option<VectorIndexApplierRef>,
|
||||
) -> Self {
|
||||
self.vector_index_applier = applier;
|
||||
self
|
||||
}
|
||||
|
||||
/// Sets over-fetched k for vector index scan.
|
||||
#[cfg(feature = "vector_index")]
|
||||
#[must_use]
|
||||
pub(crate) fn with_vector_index_k(mut self, k: Option<usize>) -> Self {
|
||||
self.vector_index_k = k;
|
||||
self
|
||||
}
|
||||
|
||||
/// Sets start time of the query.
|
||||
#[must_use]
|
||||
pub(crate) fn with_start_time(mut self, now: Option<Instant>) -> Self {
|
||||
@@ -988,7 +1060,7 @@ impl ScanInput {
|
||||
let predicate = self.predicate_for_file(file);
|
||||
let filter_mode = pre_filter_mode(self.append_mode, self.merge_mode);
|
||||
let decode_pk_values = !self.compaction && self.mapper.has_tags();
|
||||
let res = self
|
||||
let reader = self
|
||||
.access_layer
|
||||
.read_sst(file.clone())
|
||||
.predicate(predicate)
|
||||
@@ -996,7 +1068,15 @@ impl ScanInput {
|
||||
.cache(self.cache_strategy.clone())
|
||||
.inverted_index_appliers(self.inverted_index_appliers.clone())
|
||||
.bloom_filter_index_appliers(self.bloom_filter_index_appliers.clone())
|
||||
.fulltext_index_appliers(self.fulltext_index_appliers.clone())
|
||||
.fulltext_index_appliers(self.fulltext_index_appliers.clone());
|
||||
#[cfg(feature = "vector_index")]
|
||||
let reader = {
|
||||
let mut reader = reader;
|
||||
reader =
|
||||
reader.vector_index_applier(self.vector_index_applier.clone(), self.vector_index_k);
|
||||
reader
|
||||
};
|
||||
let res = reader
|
||||
.expected_metadata(Some(self.mapper.metadata().clone()))
|
||||
.flat_format(self.flat_format)
|
||||
.compaction(self.compaction)
|
||||
@@ -1418,6 +1498,7 @@ impl StreamContext {
|
||||
.entries(self.input.files.iter().map(|file| FileWrapper { file }))
|
||||
.finish()?;
|
||||
}
|
||||
write!(f, ", \"flat_format\": {}", self.input.flat_format)?;
|
||||
|
||||
#[cfg(feature = "enterprise")]
|
||||
self.format_extension_ranges(f)?;
|
||||
|
||||
@@ -115,6 +115,8 @@ pub(crate) struct ScanMetricsSet {
|
||||
scan_cost: Duration,
|
||||
/// Duration while waiting for `yield`.
|
||||
yield_cost: Duration,
|
||||
/// Duration to convert [`Batch`]es.
|
||||
convert_cost: Option<Time>,
|
||||
/// Duration of the scan.
|
||||
total_cost: Duration,
|
||||
/// Number of rows returned.
|
||||
@@ -151,6 +153,8 @@ pub(crate) struct ScanMetricsSet {
|
||||
rg_minmax_filtered: usize,
|
||||
/// Number of row groups filtered by bloom filter index.
|
||||
rg_bloom_filtered: usize,
|
||||
/// Number of row groups filtered by vector index.
|
||||
rg_vector_filtered: usize,
|
||||
/// Number of rows in row group before filtering.
|
||||
rows_before_filter: usize,
|
||||
/// Number of rows in row group filtered by fulltext index.
|
||||
@@ -159,8 +163,22 @@ pub(crate) struct ScanMetricsSet {
|
||||
rows_inverted_filtered: usize,
|
||||
/// Number of rows in row group filtered by bloom filter index.
|
||||
rows_bloom_filtered: usize,
|
||||
/// Number of rows filtered by vector index.
|
||||
rows_vector_filtered: usize,
|
||||
/// Number of rows filtered by precise filter.
|
||||
rows_precise_filtered: usize,
|
||||
/// Number of index result cache hits for fulltext index.
|
||||
fulltext_index_cache_hit: usize,
|
||||
/// Number of index result cache misses for fulltext index.
|
||||
fulltext_index_cache_miss: usize,
|
||||
/// Number of index result cache hits for inverted index.
|
||||
inverted_index_cache_hit: usize,
|
||||
/// Number of index result cache misses for inverted index.
|
||||
inverted_index_cache_miss: usize,
|
||||
/// Number of index result cache hits for bloom filter index.
|
||||
bloom_filter_cache_hit: usize,
|
||||
/// Number of index result cache misses for bloom filter index.
|
||||
bloom_filter_cache_miss: usize,
|
||||
/// Number of record batches read from SST.
|
||||
num_sst_record_batches: usize,
|
||||
/// Number of batches decoded from SST.
|
||||
@@ -183,6 +201,8 @@ pub(crate) struct ScanMetricsSet {
|
||||
distributor_scan_cost: Duration,
|
||||
/// Duration of the series distributor to yield.
|
||||
distributor_yield_cost: Duration,
|
||||
/// Duration spent in divider operations.
|
||||
distributor_divider_cost: Duration,
|
||||
|
||||
/// Merge metrics.
|
||||
merge_metrics: MergeMetrics,
|
||||
@@ -243,6 +263,7 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
build_reader_cost,
|
||||
scan_cost,
|
||||
yield_cost,
|
||||
convert_cost,
|
||||
total_cost,
|
||||
num_rows,
|
||||
num_batches,
|
||||
@@ -255,11 +276,19 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
rg_inverted_filtered,
|
||||
rg_minmax_filtered,
|
||||
rg_bloom_filtered,
|
||||
rg_vector_filtered,
|
||||
rows_before_filter,
|
||||
rows_fulltext_filtered,
|
||||
rows_inverted_filtered,
|
||||
rows_bloom_filtered,
|
||||
rows_vector_filtered,
|
||||
rows_precise_filtered,
|
||||
fulltext_index_cache_hit,
|
||||
fulltext_index_cache_miss,
|
||||
inverted_index_cache_hit,
|
||||
inverted_index_cache_miss,
|
||||
bloom_filter_cache_hit,
|
||||
bloom_filter_cache_miss,
|
||||
num_sst_record_batches,
|
||||
num_sst_batches,
|
||||
num_sst_rows,
|
||||
@@ -270,6 +299,7 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
num_distributor_batches,
|
||||
distributor_scan_cost,
|
||||
distributor_yield_cost,
|
||||
distributor_divider_cost,
|
||||
merge_metrics,
|
||||
dedup_metrics,
|
||||
stream_eof,
|
||||
@@ -307,6 +337,12 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
\"first_poll\":\"{first_poll:?}\""
|
||||
)?;
|
||||
|
||||
// Write convert_cost if present
|
||||
if let Some(time) = convert_cost {
|
||||
let duration = Duration::from_nanos(time.value() as u64);
|
||||
write!(f, ", \"convert_cost\":\"{duration:?}\"")?;
|
||||
}
|
||||
|
||||
// Write non-zero filter counters
|
||||
if *rg_fulltext_filtered > 0 {
|
||||
write!(f, ", \"rg_fulltext_filtered\":{rg_fulltext_filtered}")?;
|
||||
@@ -320,6 +356,9 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
if *rg_bloom_filtered > 0 {
|
||||
write!(f, ", \"rg_bloom_filtered\":{rg_bloom_filtered}")?;
|
||||
}
|
||||
if *rg_vector_filtered > 0 {
|
||||
write!(f, ", \"rg_vector_filtered\":{rg_vector_filtered}")?;
|
||||
}
|
||||
if *rows_fulltext_filtered > 0 {
|
||||
write!(f, ", \"rows_fulltext_filtered\":{rows_fulltext_filtered}")?;
|
||||
}
|
||||
@@ -329,9 +368,42 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
if *rows_bloom_filtered > 0 {
|
||||
write!(f, ", \"rows_bloom_filtered\":{rows_bloom_filtered}")?;
|
||||
}
|
||||
if *rows_vector_filtered > 0 {
|
||||
write!(f, ", \"rows_vector_filtered\":{rows_vector_filtered}")?;
|
||||
}
|
||||
if *rows_precise_filtered > 0 {
|
||||
write!(f, ", \"rows_precise_filtered\":{rows_precise_filtered}")?;
|
||||
}
|
||||
if *fulltext_index_cache_hit > 0 {
|
||||
write!(
|
||||
f,
|
||||
", \"fulltext_index_cache_hit\":{fulltext_index_cache_hit}"
|
||||
)?;
|
||||
}
|
||||
if *fulltext_index_cache_miss > 0 {
|
||||
write!(
|
||||
f,
|
||||
", \"fulltext_index_cache_miss\":{fulltext_index_cache_miss}"
|
||||
)?;
|
||||
}
|
||||
if *inverted_index_cache_hit > 0 {
|
||||
write!(
|
||||
f,
|
||||
", \"inverted_index_cache_hit\":{inverted_index_cache_hit}"
|
||||
)?;
|
||||
}
|
||||
if *inverted_index_cache_miss > 0 {
|
||||
write!(
|
||||
f,
|
||||
", \"inverted_index_cache_miss\":{inverted_index_cache_miss}"
|
||||
)?;
|
||||
}
|
||||
if *bloom_filter_cache_hit > 0 {
|
||||
write!(f, ", \"bloom_filter_cache_hit\":{bloom_filter_cache_hit}")?;
|
||||
}
|
||||
if *bloom_filter_cache_miss > 0 {
|
||||
write!(f, ", \"bloom_filter_cache_miss\":{bloom_filter_cache_miss}")?;
|
||||
}
|
||||
|
||||
// Write non-zero distributor metrics
|
||||
if *num_series_send_timeout > 0 {
|
||||
@@ -358,6 +430,12 @@ impl fmt::Debug for ScanMetricsSet {
|
||||
", \"distributor_yield_cost\":\"{distributor_yield_cost:?}\""
|
||||
)?;
|
||||
}
|
||||
if !distributor_divider_cost.is_zero() {
|
||||
write!(
|
||||
f,
|
||||
", \"distributor_divider_cost\":\"{distributor_divider_cost:?}\""
|
||||
)?;
|
||||
}
|
||||
|
||||
// Write non-zero memtable metrics
|
||||
if *mem_rows > 0 {
|
||||
@@ -466,27 +544,25 @@ impl ScanMetricsSet {
|
||||
self
|
||||
}
|
||||
|
||||
/// Attaches the `convert_cost` to the metrics set.
|
||||
fn with_convert_cost(mut self, time: Time) -> Self {
|
||||
self.convert_cost = Some(time);
|
||||
self
|
||||
}
|
||||
|
||||
/// Merges the local scanner metrics.
|
||||
fn merge_scanner_metrics(&mut self, other: &ScannerMetrics) {
|
||||
let ScannerMetrics {
|
||||
prepare_scan_cost,
|
||||
build_reader_cost,
|
||||
scan_cost,
|
||||
yield_cost,
|
||||
num_batches,
|
||||
num_rows,
|
||||
num_mem_ranges,
|
||||
num_file_ranges,
|
||||
} = other;
|
||||
|
||||
self.prepare_scan_cost += *prepare_scan_cost;
|
||||
self.build_reader_cost += *build_reader_cost;
|
||||
self.scan_cost += *scan_cost;
|
||||
self.yield_cost += *yield_cost;
|
||||
self.num_rows += *num_rows;
|
||||
self.num_batches += *num_batches;
|
||||
self.num_mem_ranges += *num_mem_ranges;
|
||||
self.num_file_ranges += *num_file_ranges;
|
||||
}
|
||||
|
||||
/// Merges the local reader metrics.
|
||||
@@ -500,11 +576,19 @@ impl ScanMetricsSet {
|
||||
rg_inverted_filtered,
|
||||
rg_minmax_filtered,
|
||||
rg_bloom_filtered,
|
||||
rg_vector_filtered,
|
||||
rows_total,
|
||||
rows_fulltext_filtered,
|
||||
rows_inverted_filtered,
|
||||
rows_bloom_filtered,
|
||||
rows_vector_filtered,
|
||||
rows_precise_filtered,
|
||||
fulltext_index_cache_hit,
|
||||
fulltext_index_cache_miss,
|
||||
inverted_index_cache_hit,
|
||||
inverted_index_cache_miss,
|
||||
bloom_filter_cache_hit,
|
||||
bloom_filter_cache_miss,
|
||||
inverted_index_apply_metrics,
|
||||
bloom_filter_apply_metrics,
|
||||
fulltext_index_apply_metrics,
|
||||
@@ -525,13 +609,22 @@ impl ScanMetricsSet {
|
||||
self.rg_inverted_filtered += *rg_inverted_filtered;
|
||||
self.rg_minmax_filtered += *rg_minmax_filtered;
|
||||
self.rg_bloom_filtered += *rg_bloom_filtered;
|
||||
self.rg_vector_filtered += *rg_vector_filtered;
|
||||
|
||||
self.rows_before_filter += *rows_total;
|
||||
self.rows_fulltext_filtered += *rows_fulltext_filtered;
|
||||
self.rows_inverted_filtered += *rows_inverted_filtered;
|
||||
self.rows_bloom_filtered += *rows_bloom_filtered;
|
||||
self.rows_vector_filtered += *rows_vector_filtered;
|
||||
self.rows_precise_filtered += *rows_precise_filtered;
|
||||
|
||||
self.fulltext_index_cache_hit += *fulltext_index_cache_hit;
|
||||
self.fulltext_index_cache_miss += *fulltext_index_cache_miss;
|
||||
self.inverted_index_cache_hit += *inverted_index_cache_hit;
|
||||
self.inverted_index_cache_miss += *inverted_index_cache_miss;
|
||||
self.bloom_filter_cache_hit += *bloom_filter_cache_hit;
|
||||
self.bloom_filter_cache_miss += *bloom_filter_cache_miss;
|
||||
|
||||
self.num_sst_record_batches += *num_record_batches;
|
||||
self.num_sst_batches += *num_batches;
|
||||
self.num_sst_rows += *num_rows;
|
||||
@@ -582,6 +675,7 @@ impl ScanMetricsSet {
|
||||
num_batches,
|
||||
scan_cost,
|
||||
yield_cost,
|
||||
divider_cost,
|
||||
} = distributor_metrics;
|
||||
|
||||
self.num_series_send_timeout += *num_series_send_timeout;
|
||||
@@ -590,6 +684,7 @@ impl ScanMetricsSet {
|
||||
self.num_distributor_batches += *num_batches;
|
||||
self.distributor_scan_cost += *scan_cost;
|
||||
self.distributor_yield_cost += *yield_cost;
|
||||
self.distributor_divider_cost += *divider_cost;
|
||||
}
|
||||
|
||||
/// Observes metrics.
|
||||
@@ -606,6 +701,11 @@ impl ScanMetricsSet {
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["yield"])
|
||||
.observe(self.yield_cost.as_secs_f64());
|
||||
if let Some(time) = &self.convert_cost {
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["convert"])
|
||||
.observe(Duration::from_nanos(time.value() as u64).as_secs_f64());
|
||||
}
|
||||
READ_STAGE_ELAPSED
|
||||
.with_label_values(&["total"])
|
||||
.observe(self.total_cost.as_secs_f64());
|
||||
@@ -631,6 +731,10 @@ impl ScanMetricsSet {
|
||||
READ_ROW_GROUPS_TOTAL
|
||||
.with_label_values(&["bloom_filter_index_filtered"])
|
||||
.inc_by(self.rg_bloom_filtered as u64);
|
||||
#[cfg(feature = "vector_index")]
|
||||
READ_ROW_GROUPS_TOTAL
|
||||
.with_label_values(&["vector_index_filtered"])
|
||||
.inc_by(self.rg_vector_filtered as u64);
|
||||
|
||||
PRECISE_FILTER_ROWS_TOTAL
|
||||
.with_label_values(&["parquet"])
|
||||
@@ -647,6 +751,10 @@ impl ScanMetricsSet {
|
||||
READ_ROWS_IN_ROW_GROUP_TOTAL
|
||||
.with_label_values(&["bloom_filter_index_filtered"])
|
||||
.inc_by(self.rows_bloom_filtered as u64);
|
||||
#[cfg(feature = "vector_index")]
|
||||
READ_ROWS_IN_ROW_GROUP_TOTAL
|
||||
.with_label_values(&["vector_index_filtered"])
|
||||
.inc_by(self.rows_vector_filtered as u64);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -722,21 +830,19 @@ impl Drop for PartitionMetricsInner {
|
||||
|
||||
if self.explain_verbose {
|
||||
common_telemetry::info!(
|
||||
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}, convert_batch_costs: {}",
|
||||
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}",
|
||||
self.scanner_type,
|
||||
self.region_id,
|
||||
self.partition,
|
||||
metrics,
|
||||
self.convert_cost,
|
||||
);
|
||||
} else {
|
||||
common_telemetry::debug!(
|
||||
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}, convert_batch_costs: {}",
|
||||
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}",
|
||||
self.scanner_type,
|
||||
self.region_id,
|
||||
self.partition,
|
||||
metrics,
|
||||
self.convert_cost,
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -783,7 +889,10 @@ impl PartitionMetrics {
|
||||
let partition_str = partition.to_string();
|
||||
let in_progress_scan = IN_PROGRESS_SCAN.with_label_values(&[scanner_type, &partition_str]);
|
||||
in_progress_scan.inc();
|
||||
let metrics = ScanMetricsSet::default().with_prepare_scan_cost(query_start.elapsed());
|
||||
let convert_cost = MetricBuilder::new(metrics_set).subset_time("convert_cost", partition);
|
||||
let metrics = ScanMetricsSet::default()
|
||||
.with_prepare_scan_cost(query_start.elapsed())
|
||||
.with_convert_cost(convert_cost.clone());
|
||||
let inner = PartitionMetricsInner {
|
||||
region_id,
|
||||
partition,
|
||||
@@ -798,7 +907,7 @@ impl PartitionMetrics {
|
||||
.subset_time("build_reader_cost", partition),
|
||||
scan_cost: MetricBuilder::new(metrics_set).subset_time("scan_cost", partition),
|
||||
yield_cost: MetricBuilder::new(metrics_set).subset_time("yield_cost", partition),
|
||||
convert_cost: MetricBuilder::new(metrics_set).subset_time("convert_cost", partition),
|
||||
convert_cost,
|
||||
elapsed_compute: MetricBuilder::new(metrics_set).elapsed_compute(partition),
|
||||
};
|
||||
Self(Arc::new(inner))
|
||||
@@ -850,9 +959,6 @@ impl PartitionMetrics {
|
||||
|
||||
/// Merges [ScannerMetrics], `build_reader_cost`, `scan_cost` and `yield_cost`.
|
||||
pub(crate) fn merge_metrics(&self, metrics: &ScannerMetrics) {
|
||||
self.0
|
||||
.build_reader_cost
|
||||
.add_duration(metrics.build_reader_cost);
|
||||
self.0.scan_cost.add_duration(metrics.scan_cost);
|
||||
self.record_elapsed_compute(metrics.scan_cost);
|
||||
self.0.yield_cost.add_duration(metrics.yield_cost);
|
||||
@@ -932,6 +1038,8 @@ pub(crate) struct SeriesDistributorMetrics {
|
||||
pub(crate) scan_cost: Duration,
|
||||
/// Duration of the series distributor to yield.
|
||||
pub(crate) yield_cost: Duration,
|
||||
/// Duration spent in divider operations.
|
||||
pub(crate) divider_cost: Duration,
|
||||
}
|
||||
|
||||
/// Scans memtable ranges at `index`.
|
||||
|
||||
@@ -410,6 +410,9 @@ impl SeqScan {
|
||||
|
||||
let stream = try_stream! {
|
||||
part_metrics.on_first_poll();
|
||||
// Start fetch time before building sources so scan cost contains
|
||||
// build part cost.
|
||||
let mut fetch_start = Instant::now();
|
||||
|
||||
let range_builder_list = Arc::new(RangeBuilderList::new(
|
||||
stream_ctx.input.num_memtables(),
|
||||
@@ -431,8 +434,6 @@ impl SeqScan {
|
||||
file_scan_semaphore.clone(),
|
||||
).await?;
|
||||
|
||||
let mut metrics = ScannerMetrics::default();
|
||||
let mut fetch_start = Instant::now();
|
||||
let mut reader =
|
||||
Self::build_reader_from_sources(&stream_ctx, sources, semaphore.clone(), Some(&part_metrics))
|
||||
.await?;
|
||||
@@ -441,6 +442,12 @@ impl SeqScan {
|
||||
.with_start(Some(part_range.start))
|
||||
.with_end(Some(part_range.end));
|
||||
|
||||
let mut metrics = ScannerMetrics {
|
||||
scan_cost: fetch_start.elapsed(),
|
||||
..Default::default()
|
||||
};
|
||||
fetch_start = Instant::now();
|
||||
|
||||
while let Some(batch) = reader.next_batch().await? {
|
||||
metrics.scan_cost += fetch_start.elapsed();
|
||||
metrics.num_batches += 1;
|
||||
@@ -448,6 +455,7 @@ impl SeqScan {
|
||||
|
||||
debug_assert!(!batch.is_empty());
|
||||
if batch.is_empty() {
|
||||
fetch_start = Instant::now();
|
||||
continue;
|
||||
}
|
||||
|
||||
@@ -476,6 +484,7 @@ impl SeqScan {
|
||||
}
|
||||
|
||||
metrics.scan_cost += fetch_start.elapsed();
|
||||
fetch_start = Instant::now();
|
||||
part_metrics.merge_metrics(&metrics);
|
||||
}
|
||||
|
||||
@@ -516,6 +525,9 @@ impl SeqScan {
|
||||
|
||||
let stream = try_stream! {
|
||||
part_metrics.on_first_poll();
|
||||
// Start fetch time before building sources so scan cost contains
|
||||
// build part cost.
|
||||
let mut fetch_start = Instant::now();
|
||||
|
||||
let range_builder_list = Arc::new(RangeBuilderList::new(
|
||||
stream_ctx.input.num_memtables(),
|
||||
@@ -534,12 +546,16 @@ impl SeqScan {
|
||||
file_scan_semaphore.clone(),
|
||||
).await?;
|
||||
|
||||
let mut metrics = ScannerMetrics::default();
|
||||
let mut fetch_start = Instant::now();
|
||||
let mut reader =
|
||||
Self::build_flat_reader_from_sources(&stream_ctx, sources, semaphore.clone(), Some(&part_metrics))
|
||||
.await?;
|
||||
|
||||
let mut metrics = ScannerMetrics {
|
||||
scan_cost: fetch_start.elapsed(),
|
||||
..Default::default()
|
||||
};
|
||||
fetch_start = Instant::now();
|
||||
|
||||
while let Some(record_batch) = reader.try_next().await? {
|
||||
metrics.scan_cost += fetch_start.elapsed();
|
||||
metrics.num_batches += 1;
|
||||
@@ -547,6 +563,7 @@ impl SeqScan {
|
||||
|
||||
debug_assert!(record_batch.num_rows() > 0);
|
||||
if record_batch.num_rows() == 0 {
|
||||
fetch_start = Instant::now();
|
||||
continue;
|
||||
}
|
||||
|
||||
@@ -558,6 +575,7 @@ impl SeqScan {
|
||||
}
|
||||
|
||||
metrics.scan_cost += fetch_start.elapsed();
|
||||
fetch_start = Instant::now();
|
||||
part_metrics.merge_metrics(&metrics);
|
||||
}
|
||||
|
||||
|
||||
@@ -446,6 +446,9 @@ impl SeriesDistributor {
|
||||
&self.metrics_list,
|
||||
);
|
||||
part_metrics.on_first_poll();
|
||||
// Start fetch time before building sources so scan cost contains
|
||||
// build part cost.
|
||||
let mut fetch_start = Instant::now();
|
||||
|
||||
let range_builder_list = Arc::new(RangeBuilderList::new(
|
||||
self.stream_ctx.input.num_memtables(),
|
||||
@@ -478,7 +481,6 @@ impl SeriesDistributor {
|
||||
)
|
||||
.await?;
|
||||
let mut metrics = SeriesDistributorMetrics::default();
|
||||
let mut fetch_start = Instant::now();
|
||||
|
||||
let mut divider = FlatSeriesBatchDivider::default();
|
||||
while let Some(record_batch) = reader.try_next().await? {
|
||||
@@ -493,7 +495,10 @@ impl SeriesDistributor {
|
||||
}
|
||||
|
||||
// Use divider to split series
|
||||
if let Some(series_batch) = divider.push(record_batch) {
|
||||
let divider_start = Instant::now();
|
||||
let series_batch = divider.push(record_batch);
|
||||
metrics.divider_cost += divider_start.elapsed();
|
||||
if let Some(series_batch) = series_batch {
|
||||
let yield_start = Instant::now();
|
||||
self.senders
|
||||
.send_batch(SeriesBatch::Flat(series_batch))
|
||||
@@ -504,7 +509,10 @@ impl SeriesDistributor {
|
||||
}
|
||||
|
||||
// Send any remaining batch in the divider
|
||||
if let Some(series_batch) = divider.finish() {
|
||||
let divider_start = Instant::now();
|
||||
let series_batch = divider.finish();
|
||||
metrics.divider_cost += divider_start.elapsed();
|
||||
if let Some(series_batch) = series_batch {
|
||||
let yield_start = Instant::now();
|
||||
self.senders
|
||||
.send_batch(SeriesBatch::Flat(series_batch))
|
||||
@@ -536,6 +544,9 @@ impl SeriesDistributor {
|
||||
&self.metrics_list,
|
||||
);
|
||||
part_metrics.on_first_poll();
|
||||
// Start fetch time before building sources so scan cost contains
|
||||
// build part cost.
|
||||
let mut fetch_start = Instant::now();
|
||||
|
||||
let range_builder_list = Arc::new(RangeBuilderList::new(
|
||||
self.stream_ctx.input.num_memtables(),
|
||||
@@ -568,7 +579,6 @@ impl SeriesDistributor {
|
||||
)
|
||||
.await?;
|
||||
let mut metrics = SeriesDistributorMetrics::default();
|
||||
let mut fetch_start = Instant::now();
|
||||
|
||||
let mut current_series = PrimaryKeySeriesBatch::default();
|
||||
while let Some(batch) = reader.next_batch().await? {
|
||||
|
||||
@@ -344,6 +344,12 @@ impl FileMeta {
|
||||
.contains(&IndexType::BloomFilterIndex)
|
||||
}
|
||||
|
||||
/// Returns true if the file has a vector index.
|
||||
#[cfg(feature = "vector_index")]
|
||||
pub fn vector_index_available(&self) -> bool {
|
||||
self.available_indexes.contains(&IndexType::VectorIndex)
|
||||
}
|
||||
|
||||
pub fn index_file_size(&self) -> u64 {
|
||||
self.index_file_size
|
||||
}
|
||||
|
||||
@@ -87,7 +87,7 @@ impl FileReferenceManager {
|
||||
pub(crate) async fn get_snapshot_of_file_refs(
|
||||
&self,
|
||||
query_regions_for_mem: Vec<MitoRegionRef>,
|
||||
related_regions_in_manifest: Vec<(MitoRegionRef, Vec<RegionId>)>,
|
||||
dst_region_to_src_regions: Vec<(MitoRegionRef, HashSet<RegionId>)>,
|
||||
) -> Result<FileRefsManifest> {
|
||||
let mut ref_files = HashMap::new();
|
||||
// get from in memory file handles
|
||||
@@ -99,12 +99,17 @@ impl FileReferenceManager {
|
||||
|
||||
let mut manifest_version = HashMap::new();
|
||||
|
||||
let mut cross_region_refs = HashMap::new();
|
||||
|
||||
// get file refs from related regions' manifests
|
||||
for (related_region, queries) in &related_regions_in_manifest {
|
||||
let queries = queries.iter().cloned().collect::<HashSet<_>>();
|
||||
let manifest = related_region.manifest_ctx.manifest().await;
|
||||
for (dst_region, src_regions) in &dst_region_to_src_regions {
|
||||
let manifest = dst_region.manifest_ctx.manifest().await;
|
||||
for meta in manifest.files.values() {
|
||||
if queries.contains(&meta.region_id) {
|
||||
if src_regions.contains(&meta.region_id) {
|
||||
cross_region_refs
|
||||
.entry(meta.region_id)
|
||||
.or_insert_with(HashSet::new)
|
||||
.insert(dst_region.region_id());
|
||||
// since gc couldn't happen together with repartition
|
||||
// (both the queries and related_region acquire region read lock), no need to worry about
|
||||
// staging manifest in repartition here.
|
||||
@@ -119,7 +124,7 @@ impl FileReferenceManager {
|
||||
}
|
||||
}
|
||||
// not sure if related region's manifest version is needed, but record it for now.
|
||||
manifest_version.insert(related_region.region_id(), manifest.manifest_version);
|
||||
manifest_version.insert(dst_region.region_id(), manifest.manifest_version);
|
||||
}
|
||||
|
||||
for r in &query_regions_for_mem {
|
||||
@@ -138,6 +143,7 @@ impl FileReferenceManager {
|
||||
Ok(FileRefsManifest {
|
||||
file_refs: ref_files,
|
||||
manifest_version,
|
||||
cross_region_refs,
|
||||
})
|
||||
}
|
||||
|
||||
|
||||
650
src/mito2/src/sst/index/vector_index/applier.rs
Normal file
650
src/mito2/src/sst/index/vector_index/applier.rs
Normal file
@@ -0,0 +1,650 @@
|
||||
// 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.
|
||||
|
||||
//! Vector index applier for KNN search.
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_base::range_read::RangeReader;
|
||||
use common_telemetry::warn;
|
||||
use index::vector::distance_metric_to_usearch;
|
||||
use puffin::puffin_manager::cache::PuffinMetadataCacheRef;
|
||||
use puffin::puffin_manager::{PuffinManager, PuffinReader};
|
||||
use roaring::RoaringBitmap;
|
||||
use snafu::ResultExt;
|
||||
use store_api::storage::{ColumnId, VectorDistanceMetric};
|
||||
|
||||
use crate::access_layer::{RegionFilePathFactory, WriteCachePathProvider};
|
||||
use crate::cache::file_cache::{FileCacheRef, FileType, IndexKey};
|
||||
use crate::cache::index::vector_index::{
|
||||
CachedVectorIndex, VectorIndexCacheKey, VectorIndexCacheRef,
|
||||
};
|
||||
use crate::error::{ApplyVectorIndexSnafu, PuffinBuildReaderSnafu, PuffinReadBlobSnafu, Result};
|
||||
use crate::sst::file::RegionIndexId;
|
||||
use crate::sst::index::puffin_manager::{BlobReader, PuffinManagerFactory};
|
||||
use crate::sst::index::trigger_index_background_download;
|
||||
use crate::sst::index::vector_index::creator::VectorIndexConfig;
|
||||
use crate::sst::index::vector_index::format::VectorIndexBlobHeader;
|
||||
use crate::sst::index::vector_index::{INDEX_BLOB_TYPE, engine};
|
||||
|
||||
/// Result of applying vector index.
|
||||
#[derive(Debug)]
|
||||
pub struct VectorIndexApplyOutput {
|
||||
/// Row offsets in the SST file.
|
||||
pub row_offsets: Vec<u64>,
|
||||
}
|
||||
|
||||
/// Vector index applier for KNN search against SST blobs.
|
||||
pub struct VectorIndexApplier {
|
||||
table_dir: String,
|
||||
path_type: store_api::region_request::PathType,
|
||||
object_store: object_store::ObjectStore,
|
||||
puffin_manager_factory: PuffinManagerFactory,
|
||||
file_cache: Option<FileCacheRef>,
|
||||
puffin_metadata_cache: Option<PuffinMetadataCacheRef>,
|
||||
vector_index_cache: Option<VectorIndexCacheRef>,
|
||||
column_id: ColumnId,
|
||||
query_vector: Vec<f32>,
|
||||
metric: VectorDistanceMetric,
|
||||
}
|
||||
|
||||
pub type VectorIndexApplierRef = Arc<VectorIndexApplier>;
|
||||
|
||||
impl VectorIndexApplier {
|
||||
pub fn new(
|
||||
table_dir: String,
|
||||
path_type: store_api::region_request::PathType,
|
||||
object_store: object_store::ObjectStore,
|
||||
puffin_manager_factory: PuffinManagerFactory,
|
||||
column_id: ColumnId,
|
||||
query_vector: Vec<f32>,
|
||||
metric: VectorDistanceMetric,
|
||||
) -> Self {
|
||||
Self {
|
||||
table_dir,
|
||||
path_type,
|
||||
object_store,
|
||||
puffin_manager_factory,
|
||||
file_cache: None,
|
||||
puffin_metadata_cache: None,
|
||||
vector_index_cache: None,
|
||||
column_id,
|
||||
query_vector,
|
||||
metric,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn with_file_cache(mut self, file_cache: Option<FileCacheRef>) -> Self {
|
||||
self.file_cache = file_cache;
|
||||
self
|
||||
}
|
||||
|
||||
pub fn with_puffin_metadata_cache(
|
||||
mut self,
|
||||
puffin_metadata_cache: Option<PuffinMetadataCacheRef>,
|
||||
) -> Self {
|
||||
self.puffin_metadata_cache = puffin_metadata_cache;
|
||||
self
|
||||
}
|
||||
|
||||
pub fn with_vector_index_cache(mut self, cache: Option<VectorIndexCacheRef>) -> Self {
|
||||
self.vector_index_cache = cache;
|
||||
self
|
||||
}
|
||||
|
||||
/// Applies vector index to the file and returns candidates.
|
||||
///
|
||||
/// This method loads the vector index blob (from cache or remote), runs
|
||||
/// a KNN search against the indexed vectors, and maps the HNSW keys back
|
||||
/// to row offsets in the SST file. It returns only row offsets; callers
|
||||
/// are responsible for any higher-level ordering or limit enforcement.
|
||||
pub async fn apply_with_k(
|
||||
&self,
|
||||
file_id: RegionIndexId,
|
||||
file_size_hint: Option<u64>,
|
||||
k: usize,
|
||||
) -> Result<VectorIndexApplyOutput> {
|
||||
if k == 0 {
|
||||
return Ok(VectorIndexApplyOutput {
|
||||
row_offsets: Vec::new(),
|
||||
});
|
||||
}
|
||||
|
||||
let index = self.load_or_read_index(file_id, file_size_hint).await?;
|
||||
let Some(index) = index else {
|
||||
return Ok(VectorIndexApplyOutput {
|
||||
row_offsets: Vec::new(),
|
||||
});
|
||||
};
|
||||
|
||||
if self.query_vector.len() != index.dimensions as usize {
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: format!(
|
||||
"Query vector dimension {} does not match index dimension {}",
|
||||
self.query_vector.len(),
|
||||
index.dimensions
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
if self.metric != index.metric {
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: format!(
|
||||
"Query metric {} does not match index metric {}",
|
||||
self.metric, index.metric
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
if index.indexed_rows == 0 {
|
||||
return Ok(VectorIndexApplyOutput {
|
||||
row_offsets: Vec::new(),
|
||||
});
|
||||
}
|
||||
|
||||
let matches = index
|
||||
.engine
|
||||
.search(&self.query_vector, k.min(index.indexed_rows as usize))
|
||||
.map_err(|e| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: e.to_string(),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
|
||||
let row_offsets = map_hnsw_keys_to_row_offsets(
|
||||
&index.null_bitmap,
|
||||
index.total_rows,
|
||||
index.indexed_rows,
|
||||
matches.keys,
|
||||
)?;
|
||||
|
||||
Ok(VectorIndexApplyOutput { row_offsets })
|
||||
}
|
||||
|
||||
async fn load_or_read_index(
|
||||
&self,
|
||||
file_id: RegionIndexId,
|
||||
file_size_hint: Option<u64>,
|
||||
) -> Result<Option<Arc<CachedVectorIndex>>> {
|
||||
let cache_key =
|
||||
VectorIndexCacheKey::new(file_id.file_id(), file_id.version, self.column_id);
|
||||
if let Some(cache) = &self.vector_index_cache
|
||||
&& let Some(cached) = cache.get(&cache_key)
|
||||
{
|
||||
return Ok(Some(cached));
|
||||
}
|
||||
|
||||
let reader = match self.cached_blob_reader(file_id, file_size_hint).await {
|
||||
Ok(Some(reader)) => reader,
|
||||
Ok(None) => self.remote_blob_reader(file_id, file_size_hint).await?,
|
||||
Err(err) => {
|
||||
if is_blob_not_found(&err) {
|
||||
self.remote_blob_reader(file_id, file_size_hint).await?
|
||||
} else {
|
||||
warn!(err; "Failed to read cached vector index blob, fallback to remote");
|
||||
self.remote_blob_reader(file_id, file_size_hint).await?
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
let blob_data = read_all_blob(reader, file_size_hint).await?;
|
||||
if blob_data.is_empty() {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let cached = Arc::new(parse_vector_index_blob(&blob_data)?);
|
||||
if let Some(cache) = &self.vector_index_cache {
|
||||
cache.insert(cache_key, cached.clone());
|
||||
}
|
||||
|
||||
Ok(Some(cached))
|
||||
}
|
||||
|
||||
async fn cached_blob_reader(
|
||||
&self,
|
||||
file_id: RegionIndexId,
|
||||
file_size_hint: Option<u64>,
|
||||
) -> Result<Option<BlobReader>> {
|
||||
let Some(file_cache) = &self.file_cache else {
|
||||
return Ok(None);
|
||||
};
|
||||
|
||||
let index_key = IndexKey::new(
|
||||
file_id.region_id(),
|
||||
file_id.file_id(),
|
||||
FileType::Puffin(file_id.version),
|
||||
);
|
||||
if file_cache.get(index_key).await.is_none() {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let puffin_manager = self.puffin_manager_factory.build(
|
||||
file_cache.local_store(),
|
||||
WriteCachePathProvider::new(file_cache.clone()),
|
||||
);
|
||||
let blob_name = column_blob_name(self.column_id);
|
||||
|
||||
let reader = puffin_manager
|
||||
.reader(&file_id)
|
||||
.await
|
||||
.context(PuffinBuildReaderSnafu)?
|
||||
.with_file_size_hint(file_size_hint)
|
||||
.blob(&blob_name)
|
||||
.await
|
||||
.context(PuffinReadBlobSnafu)?
|
||||
.reader()
|
||||
.await
|
||||
.context(PuffinBuildReaderSnafu)?;
|
||||
Ok(Some(reader))
|
||||
}
|
||||
|
||||
async fn remote_blob_reader(
|
||||
&self,
|
||||
file_id: RegionIndexId,
|
||||
file_size_hint: Option<u64>,
|
||||
) -> Result<BlobReader> {
|
||||
let path_factory = RegionFilePathFactory::new(self.table_dir.clone(), self.path_type);
|
||||
|
||||
trigger_index_background_download(
|
||||
self.file_cache.as_ref(),
|
||||
&file_id,
|
||||
file_size_hint,
|
||||
&path_factory,
|
||||
&self.object_store,
|
||||
);
|
||||
|
||||
let puffin_manager = self
|
||||
.puffin_manager_factory
|
||||
.build(self.object_store.clone(), path_factory)
|
||||
.with_puffin_metadata_cache(self.puffin_metadata_cache.clone());
|
||||
|
||||
let blob_name = column_blob_name(self.column_id);
|
||||
|
||||
puffin_manager
|
||||
.reader(&file_id)
|
||||
.await
|
||||
.context(PuffinBuildReaderSnafu)?
|
||||
.with_file_size_hint(file_size_hint)
|
||||
.blob(&blob_name)
|
||||
.await
|
||||
.context(PuffinReadBlobSnafu)?
|
||||
.reader()
|
||||
.await
|
||||
.context(PuffinBuildReaderSnafu)
|
||||
}
|
||||
}
|
||||
|
||||
fn column_blob_name(column_id: ColumnId) -> String {
|
||||
format!("{INDEX_BLOB_TYPE}-{}", column_id)
|
||||
}
|
||||
|
||||
fn is_blob_not_found(err: &crate::error::Error) -> bool {
|
||||
matches!(
|
||||
err,
|
||||
crate::error::Error::PuffinReadBlob {
|
||||
source: puffin::error::Error::BlobNotFound { .. },
|
||||
..
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
async fn read_all_blob(reader: BlobReader, file_size_hint: Option<u64>) -> Result<Vec<u8>> {
|
||||
let metadata = reader.metadata().await.map_err(|e| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: format!("Failed to read vector index metadata: {}", e),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
if let Some(limit) = file_size_hint
|
||||
&& metadata.content_length > limit
|
||||
{
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: format!(
|
||||
"Vector index blob size {} exceeds file size hint {}",
|
||||
metadata.content_length, limit
|
||||
),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
let bytes = reader.read(0..metadata.content_length).await.map_err(|e| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: format!("Failed to read vector index data: {}", e),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
Ok(bytes.to_vec())
|
||||
}
|
||||
|
||||
fn parse_vector_index_blob(data: &[u8]) -> Result<CachedVectorIndex> {
|
||||
let (header, mut offset) = VectorIndexBlobHeader::decode(data).map_err(|e| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: e.to_string(),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
let null_bitmap_len = header.null_bitmap_len as usize;
|
||||
|
||||
if data.len() < offset + null_bitmap_len {
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: "Vector index blob truncated while reading null bitmap".to_string(),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
let null_bitmap_bytes = &data[offset..offset + null_bitmap_len];
|
||||
offset += null_bitmap_len;
|
||||
let null_bitmap = RoaringBitmap::deserialize_from(null_bitmap_bytes).map_err(|e| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: format!("Failed to deserialize null bitmap: {}", e),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
|
||||
let index_bytes = &data[offset..];
|
||||
let config = VectorIndexConfig {
|
||||
engine: header.engine_type,
|
||||
dim: header.dim as usize,
|
||||
metric: distance_metric_to_usearch(header.metric),
|
||||
distance_metric: header.metric,
|
||||
connectivity: header.connectivity as usize,
|
||||
expansion_add: header.expansion_add as usize,
|
||||
expansion_search: header.expansion_search as usize,
|
||||
};
|
||||
let engine = engine::load_engine(header.engine_type, &config, index_bytes).map_err(|e| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: e.to_string(),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
|
||||
Ok(CachedVectorIndex::new(
|
||||
engine,
|
||||
null_bitmap,
|
||||
header.dim,
|
||||
header.metric,
|
||||
header.total_rows,
|
||||
header.indexed_rows,
|
||||
))
|
||||
}
|
||||
|
||||
fn map_hnsw_keys_to_row_offsets(
|
||||
null_bitmap: &RoaringBitmap,
|
||||
total_rows: u64,
|
||||
indexed_rows: u64,
|
||||
keys: Vec<u64>,
|
||||
) -> Result<Vec<u64>> {
|
||||
if total_rows == 0 {
|
||||
return Ok(Vec::new());
|
||||
}
|
||||
let total_rows_u32 = u32::try_from(total_rows).map_err(|_| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: format!("Total rows {} exceeds u32::MAX", total_rows),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
|
||||
let mut row_offsets = Vec::with_capacity(keys.len());
|
||||
for key in keys {
|
||||
let offset = hnsw_key_to_row_offset(null_bitmap, total_rows_u32, indexed_rows, key)?;
|
||||
row_offsets.push(offset as u64);
|
||||
}
|
||||
Ok(row_offsets)
|
||||
}
|
||||
|
||||
fn hnsw_key_to_row_offset(
|
||||
null_bitmap: &RoaringBitmap,
|
||||
total_rows: u32,
|
||||
indexed_rows: u64,
|
||||
key: u64,
|
||||
) -> Result<u32> {
|
||||
if total_rows == 0 {
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: "Total rows is zero".to_string(),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
if key >= indexed_rows {
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: format!("HNSW key {} exceeds indexed rows {}", key, indexed_rows),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
if null_bitmap.is_empty() {
|
||||
return Ok(key as u32);
|
||||
}
|
||||
|
||||
let mut left: u32 = 0;
|
||||
let mut right: u32 = total_rows - 1;
|
||||
while left <= right {
|
||||
let mid = left + (right - left) / 2;
|
||||
let nulls_before = null_bitmap.rank(mid);
|
||||
let non_nulls = (mid as u64 + 1).saturating_sub(nulls_before);
|
||||
if non_nulls > key {
|
||||
if mid == 0 {
|
||||
break;
|
||||
}
|
||||
right = mid - 1;
|
||||
} else {
|
||||
left = mid + 1;
|
||||
}
|
||||
}
|
||||
|
||||
if left >= total_rows {
|
||||
return ApplyVectorIndexSnafu {
|
||||
reason: "Failed to map HNSW key to row offset".to_string(),
|
||||
}
|
||||
.fail();
|
||||
}
|
||||
|
||||
Ok(left)
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use common_test_util::temp_dir::TempDir;
|
||||
use futures::io::Cursor;
|
||||
use object_store::ObjectStore;
|
||||
use object_store::services::Memory;
|
||||
use puffin::puffin_manager::PuffinWriter;
|
||||
use store_api::region_request::PathType;
|
||||
use store_api::storage::{ColumnId, FileId, VectorDistanceMetric, VectorIndexEngineType};
|
||||
|
||||
use super::*;
|
||||
use crate::access_layer::RegionFilePathFactory;
|
||||
use crate::sst::file::RegionFileId;
|
||||
use crate::sst::index::puffin_manager::PuffinManagerFactory;
|
||||
use crate::sst::index::vector_index::creator::VectorIndexConfig;
|
||||
|
||||
async fn build_applier_with_blob(
|
||||
blob: Vec<u8>,
|
||||
column_id: ColumnId,
|
||||
query_vector: Vec<f32>,
|
||||
metric: VectorDistanceMetric,
|
||||
) -> (TempDir, VectorIndexApplier, RegionIndexId, u64) {
|
||||
let (dir, puffin_manager_factory) =
|
||||
PuffinManagerFactory::new_for_test_async("test_vector_index_applier_").await;
|
||||
let object_store = ObjectStore::new(Memory::default()).unwrap().finish();
|
||||
let file_id = RegionFileId::new(0.into(), FileId::random());
|
||||
let index_id = RegionIndexId::new(file_id, 0);
|
||||
let table_dir = "table_dir".to_string();
|
||||
|
||||
let puffin_manager = puffin_manager_factory.build(
|
||||
object_store.clone(),
|
||||
RegionFilePathFactory::new(table_dir.clone(), PathType::Bare),
|
||||
);
|
||||
let mut writer = puffin_manager.writer(&index_id).await.unwrap();
|
||||
let blob_name = column_blob_name(column_id);
|
||||
let _bytes_written = writer
|
||||
.put_blob(
|
||||
blob_name.as_str(),
|
||||
Cursor::new(blob),
|
||||
Default::default(),
|
||||
Default::default(),
|
||||
)
|
||||
.await
|
||||
.unwrap();
|
||||
let file_size = writer.finish().await.unwrap();
|
||||
|
||||
let applier = VectorIndexApplier::new(
|
||||
table_dir,
|
||||
PathType::Bare,
|
||||
object_store,
|
||||
puffin_manager_factory,
|
||||
column_id,
|
||||
query_vector,
|
||||
metric,
|
||||
);
|
||||
|
||||
(dir, applier, index_id, file_size)
|
||||
}
|
||||
|
||||
fn build_blob_with_vectors(
|
||||
config: &VectorIndexConfig,
|
||||
vectors: Vec<(u64, Vec<f32>)>,
|
||||
null_bitmap: &RoaringBitmap,
|
||||
total_rows: u64,
|
||||
indexed_rows: u64,
|
||||
) -> Vec<u8> {
|
||||
let mut engine = engine::create_engine(config.engine, config).unwrap();
|
||||
for (key, vector) in vectors {
|
||||
engine.add(key, &vector).unwrap();
|
||||
}
|
||||
let index_size = engine.serialized_length();
|
||||
let mut index_bytes = vec![0u8; index_size];
|
||||
engine.save_to_buffer(&mut index_bytes).unwrap();
|
||||
|
||||
let mut null_bitmap_bytes = Vec::new();
|
||||
null_bitmap.serialize_into(&mut null_bitmap_bytes).unwrap();
|
||||
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
config.engine,
|
||||
config.dim as u32,
|
||||
config.distance_metric,
|
||||
config.connectivity as u16,
|
||||
config.expansion_add as u16,
|
||||
config.expansion_search as u16,
|
||||
total_rows,
|
||||
indexed_rows,
|
||||
null_bitmap_bytes.len() as u32,
|
||||
)
|
||||
.unwrap();
|
||||
let mut blob = Vec::new();
|
||||
header.encode_into(&mut blob);
|
||||
blob.extend_from_slice(&null_bitmap_bytes);
|
||||
blob.extend_from_slice(&index_bytes);
|
||||
blob
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_hnsw_key_to_row_offset_with_nulls() {
|
||||
let mut bitmap = RoaringBitmap::new();
|
||||
bitmap.insert(1);
|
||||
bitmap.insert(3);
|
||||
|
||||
assert_eq!(hnsw_key_to_row_offset(&bitmap, 6, 4, 0).unwrap(), 0);
|
||||
assert_eq!(hnsw_key_to_row_offset(&bitmap, 6, 4, 1).unwrap(), 2);
|
||||
assert_eq!(hnsw_key_to_row_offset(&bitmap, 6, 4, 2).unwrap(), 4);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_hnsw_key_to_row_offset_without_nulls() {
|
||||
let bitmap = RoaringBitmap::new();
|
||||
assert_eq!(hnsw_key_to_row_offset(&bitmap, 4, 4, 3).unwrap(), 3);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_hnsw_key_to_row_offset_out_of_range() {
|
||||
let bitmap = RoaringBitmap::new();
|
||||
assert!(hnsw_key_to_row_offset(&bitmap, 4, 4, 4).is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_map_hnsw_keys_to_row_offsets_multiple_keys() {
|
||||
let bitmap = RoaringBitmap::new();
|
||||
let offsets = map_hnsw_keys_to_row_offsets(&bitmap, 4, 4, vec![0, 2, 3]).unwrap();
|
||||
assert_eq!(offsets, vec![0, 2, 3]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_apply_with_k_returns_offsets() {
|
||||
let config = VectorIndexConfig {
|
||||
engine: VectorIndexEngineType::Usearch,
|
||||
dim: 2,
|
||||
metric: distance_metric_to_usearch(VectorDistanceMetric::L2sq),
|
||||
distance_metric: VectorDistanceMetric::L2sq,
|
||||
connectivity: 16,
|
||||
expansion_add: 128,
|
||||
expansion_search: 64,
|
||||
};
|
||||
let mut null_bitmap = RoaringBitmap::new();
|
||||
null_bitmap.insert(1);
|
||||
let blob = build_blob_with_vectors(
|
||||
&config,
|
||||
vec![(0, vec![1.0, 0.0]), (1, vec![0.0, 1.0])],
|
||||
&null_bitmap,
|
||||
3,
|
||||
2,
|
||||
);
|
||||
let (_dir, applier, index_id, size_bytes) =
|
||||
build_applier_with_blob(blob, 1, vec![1.0, 0.0], VectorDistanceMetric::L2sq).await;
|
||||
let output = applier
|
||||
.apply_with_k(index_id, Some(size_bytes), 2)
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(output.row_offsets, vec![0, 2]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_apply_with_k_dimension_mismatch() {
|
||||
let config = VectorIndexConfig {
|
||||
engine: VectorIndexEngineType::Usearch,
|
||||
dim: 2,
|
||||
metric: distance_metric_to_usearch(VectorDistanceMetric::L2sq),
|
||||
distance_metric: VectorDistanceMetric::L2sq,
|
||||
connectivity: 16,
|
||||
expansion_add: 128,
|
||||
expansion_search: 64,
|
||||
};
|
||||
let null_bitmap = RoaringBitmap::new();
|
||||
let blob = build_blob_with_vectors(&config, vec![(0, vec![1.0, 0.0])], &null_bitmap, 1, 1);
|
||||
let (_dir, applier, index_id, size_bytes) =
|
||||
build_applier_with_blob(blob, 1, vec![1.0, 0.0, 0.0], VectorDistanceMetric::L2sq).await;
|
||||
let res = applier.apply_with_k(index_id, Some(size_bytes), 1).await;
|
||||
assert!(res.is_err());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_apply_with_k_empty_blob() {
|
||||
let config = VectorIndexConfig {
|
||||
engine: VectorIndexEngineType::Usearch,
|
||||
dim: 1,
|
||||
metric: distance_metric_to_usearch(VectorDistanceMetric::L2sq),
|
||||
distance_metric: VectorDistanceMetric::L2sq,
|
||||
connectivity: 16,
|
||||
expansion_add: 128,
|
||||
expansion_search: 64,
|
||||
};
|
||||
let null_bitmap = RoaringBitmap::new();
|
||||
let blob = build_blob_with_vectors(&config, Vec::new(), &null_bitmap, 0, 0);
|
||||
let (_dir, applier, index_id, size_bytes) =
|
||||
build_applier_with_blob(blob, 1, vec![1.0], VectorDistanceMetric::L2sq).await;
|
||||
let output = applier
|
||||
.apply_with_k(index_id, Some(size_bytes), 1)
|
||||
.await
|
||||
.unwrap();
|
||||
assert!(output.row_offsets.is_empty());
|
||||
}
|
||||
}
|
||||
@@ -44,6 +44,9 @@ use crate::sst::index::intermediate::{
|
||||
};
|
||||
use crate::sst::index::puffin_manager::SstPuffinWriter;
|
||||
use crate::sst::index::statistics::{ByteCount, RowCount, Statistics};
|
||||
use crate::sst::index::vector_index::format::{
|
||||
VECTOR_INDEX_BLOB_HEADER_SIZE, VectorIndexBlobHeader,
|
||||
};
|
||||
use crate::sst::index::vector_index::util::bytes_to_f32_slice;
|
||||
use crate::sst::index::vector_index::{INDEX_BLOB_TYPE, engine};
|
||||
|
||||
@@ -323,6 +326,7 @@ impl VectorIndexer {
|
||||
|
||||
for (col_id, creator) in &mut self.creators {
|
||||
let Some(values) = batch.field_col_value(*col_id) else {
|
||||
creator.add_nulls(n);
|
||||
continue;
|
||||
};
|
||||
|
||||
@@ -561,36 +565,12 @@ impl VectorIndexer {
|
||||
creator.save_to_buffer(&mut index_bytes)?;
|
||||
|
||||
// Header size: version(1) + engine(1) + dim(4) + metric(1) +
|
||||
// connectivity(2) + expansion_add(2) + expansion_search(2) +
|
||||
// total_rows(8) + indexed_rows(8) + bitmap_len(4) = 33 bytes
|
||||
/// Size of the vector index blob header in bytes.
|
||||
/// Header format: version(1) + engine(1) + dim(4) + metric(1) +
|
||||
/// connectivity(2) + expansion_add(2) + expansion_search(2) +
|
||||
/// total_rows(8) + indexed_rows(8) + bitmap_len(4) = 33 bytes
|
||||
const VECTOR_INDEX_BLOB_HEADER_SIZE: usize = 33;
|
||||
// connectivity(2) + expansion_add(2) + expansion_search(2) +
|
||||
// total_rows(8) + indexed_rows(8) + bitmap_len(4) = 33 bytes.
|
||||
let total_size =
|
||||
VECTOR_INDEX_BLOB_HEADER_SIZE + null_bitmap_bytes.len() + index_bytes.len();
|
||||
let mut blob_data = Vec::with_capacity(total_size);
|
||||
|
||||
// Write version (1 byte)
|
||||
blob_data.push(1u8);
|
||||
// Write engine type (1 byte)
|
||||
blob_data.push(creator.engine_type().as_u8());
|
||||
// Write dimension (4 bytes, little-endian)
|
||||
blob_data.extend_from_slice(&(creator.config.dim as u32).to_le_bytes());
|
||||
// Write metric (1 byte)
|
||||
blob_data.push(creator.metric().as_u8());
|
||||
// Write connectivity/M (2 bytes, little-endian)
|
||||
blob_data.extend_from_slice(&(creator.config.connectivity as u16).to_le_bytes());
|
||||
// Write expansion_add/ef_construction (2 bytes, little-endian)
|
||||
blob_data.extend_from_slice(&(creator.config.expansion_add as u16).to_le_bytes());
|
||||
// Write expansion_search/ef_search (2 bytes, little-endian)
|
||||
blob_data.extend_from_slice(&(creator.config.expansion_search as u16).to_le_bytes());
|
||||
// Write total_rows (8 bytes, little-endian)
|
||||
blob_data.extend_from_slice(&creator.current_row_offset.to_le_bytes());
|
||||
// Write indexed_rows (8 bytes, little-endian)
|
||||
blob_data.extend_from_slice(&creator.next_hnsw_key.to_le_bytes());
|
||||
// Write NULL bitmap length (4 bytes, little-endian)
|
||||
let bitmap_len: u32 = null_bitmap_bytes.len().try_into().map_err(|_| {
|
||||
VectorIndexBuildSnafu {
|
||||
reason: format!(
|
||||
@@ -601,7 +581,24 @@ impl VectorIndexer {
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
blob_data.extend_from_slice(&bitmap_len.to_le_bytes());
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
creator.engine_type(),
|
||||
creator.config.dim as u32,
|
||||
creator.metric(),
|
||||
creator.config.connectivity as u16,
|
||||
creator.config.expansion_add as u16,
|
||||
creator.config.expansion_search as u16,
|
||||
creator.current_row_offset,
|
||||
creator.next_hnsw_key,
|
||||
bitmap_len,
|
||||
)
|
||||
.map_err(|e| {
|
||||
VectorIndexFinishSnafu {
|
||||
reason: e.to_string(),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
header.encode_into(&mut blob_data);
|
||||
// Write NULL bitmap
|
||||
blob_data.extend_from_slice(&null_bitmap_bytes);
|
||||
// Write vector index
|
||||
@@ -686,7 +683,78 @@ impl VectorIndexer {
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use api::v1::SemanticType;
|
||||
use datatypes::data_type::ConcreteDataType;
|
||||
use datatypes::schema::ColumnSchema;
|
||||
use datatypes::value::ValueRef;
|
||||
use datatypes::vectors::{TimestampMillisecondVector, UInt8Vector, UInt64Vector};
|
||||
use mito_codec::row_converter::{DensePrimaryKeyCodec, PrimaryKeyCodecExt, SortField};
|
||||
use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder, RegionMetadataRef};
|
||||
use store_api::storage::{ColumnId, FileId, RegionId};
|
||||
|
||||
use super::*;
|
||||
use crate::read::BatchColumn;
|
||||
|
||||
fn mock_region_metadata_with_vector() -> RegionMetadataRef {
|
||||
let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1));
|
||||
builder
|
||||
.push_column_metadata(ColumnMetadata {
|
||||
column_schema: ColumnSchema::new(
|
||||
"ts",
|
||||
ConcreteDataType::timestamp_millisecond_datatype(),
|
||||
false,
|
||||
),
|
||||
semantic_type: SemanticType::Timestamp,
|
||||
column_id: 1,
|
||||
})
|
||||
.push_column_metadata(ColumnMetadata {
|
||||
column_schema: ColumnSchema::new("tag", ConcreteDataType::int64_datatype(), true),
|
||||
semantic_type: SemanticType::Tag,
|
||||
column_id: 2,
|
||||
})
|
||||
.push_column_metadata(ColumnMetadata {
|
||||
column_schema: ColumnSchema::new("vec", ConcreteDataType::vector_datatype(2), true),
|
||||
semantic_type: SemanticType::Field,
|
||||
column_id: 3,
|
||||
})
|
||||
.push_column_metadata(ColumnMetadata {
|
||||
column_schema: ColumnSchema::new(
|
||||
"field_u64",
|
||||
ConcreteDataType::uint64_datatype(),
|
||||
true,
|
||||
),
|
||||
semantic_type: SemanticType::Field,
|
||||
column_id: 4,
|
||||
})
|
||||
.primary_key(vec![2]);
|
||||
|
||||
Arc::new(builder.build().unwrap())
|
||||
}
|
||||
|
||||
fn new_batch_missing_vector_column(column_id: ColumnId, rows: usize) -> Batch {
|
||||
let fields = vec![(0, SortField::new(ConcreteDataType::int64_datatype()))];
|
||||
let codec = DensePrimaryKeyCodec::with_fields(fields);
|
||||
let primary_key = codec.encode([ValueRef::Int64(1)].into_iter()).unwrap();
|
||||
|
||||
let field = BatchColumn {
|
||||
column_id,
|
||||
data: Arc::new(UInt64Vector::from_iter_values(0..rows as u64)),
|
||||
};
|
||||
|
||||
Batch::new(
|
||||
primary_key,
|
||||
Arc::new(TimestampMillisecondVector::from_values(
|
||||
(0..rows).map(|i| i as i64).collect::<Vec<_>>(),
|
||||
)),
|
||||
Arc::new(UInt64Vector::from_iter_values(std::iter::repeat_n(0, rows))),
|
||||
Arc::new(UInt8Vector::from_iter_values(std::iter::repeat_n(1, rows))),
|
||||
vec![field],
|
||||
)
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_index_creator() {
|
||||
@@ -837,23 +905,24 @@ mod tests {
|
||||
let mut index_bytes = vec![0u8; index_size];
|
||||
creator.save_to_buffer(&mut index_bytes).unwrap();
|
||||
|
||||
// Header: 33 bytes
|
||||
let header_size = 33;
|
||||
let total_size = header_size + null_bitmap_bytes.len() + index_bytes.len();
|
||||
let total_size =
|
||||
VECTOR_INDEX_BLOB_HEADER_SIZE + null_bitmap_bytes.len() + index_bytes.len();
|
||||
let mut blob_data = Vec::with_capacity(total_size);
|
||||
|
||||
// Write header fields
|
||||
blob_data.push(1u8); // version
|
||||
blob_data.push(creator.engine_type().as_u8()); // engine type
|
||||
blob_data.extend_from_slice(&(creator.config.dim as u32).to_le_bytes()); // dimension
|
||||
blob_data.push(creator.metric().as_u8()); // metric
|
||||
blob_data.extend_from_slice(&(creator.config.connectivity as u16).to_le_bytes());
|
||||
blob_data.extend_from_slice(&(creator.config.expansion_add as u16).to_le_bytes());
|
||||
blob_data.extend_from_slice(&(creator.config.expansion_search as u16).to_le_bytes());
|
||||
blob_data.extend_from_slice(&creator.current_row_offset.to_le_bytes()); // total_rows
|
||||
blob_data.extend_from_slice(&creator.next_hnsw_key.to_le_bytes()); // indexed_rows
|
||||
let bitmap_len: u32 = null_bitmap_bytes.len().try_into().unwrap();
|
||||
blob_data.extend_from_slice(&bitmap_len.to_le_bytes());
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
creator.engine_type(),
|
||||
creator.config.dim as u32,
|
||||
creator.metric(),
|
||||
creator.config.connectivity as u16,
|
||||
creator.config.expansion_add as u16,
|
||||
creator.config.expansion_search as u16,
|
||||
creator.current_row_offset,
|
||||
creator.next_hnsw_key,
|
||||
bitmap_len,
|
||||
)
|
||||
.unwrap();
|
||||
header.encode_into(&mut blob_data);
|
||||
blob_data.extend_from_slice(&null_bitmap_bytes);
|
||||
blob_data.extend_from_slice(&index_bytes);
|
||||
|
||||
@@ -861,60 +930,62 @@ mod tests {
|
||||
assert_eq!(blob_data.len(), total_size);
|
||||
|
||||
// Parse header and verify values
|
||||
assert_eq!(blob_data[0], 1); // version
|
||||
assert_eq!(blob_data[1], VectorIndexEngineType::Usearch.as_u8()); // engine
|
||||
|
||||
let dim = u32::from_le_bytes([blob_data[2], blob_data[3], blob_data[4], blob_data[5]]);
|
||||
assert_eq!(dim, 4);
|
||||
|
||||
let metric = blob_data[6];
|
||||
let (decoded, header_size) = VectorIndexBlobHeader::decode(&blob_data).unwrap();
|
||||
assert_eq!(header_size, VECTOR_INDEX_BLOB_HEADER_SIZE);
|
||||
assert_eq!(decoded.engine_type, VectorIndexEngineType::Usearch);
|
||||
assert_eq!(decoded.dim, 4);
|
||||
assert_eq!(
|
||||
metric,
|
||||
datatypes::schema::VectorDistanceMetric::L2sq.as_u8()
|
||||
decoded.metric,
|
||||
datatypes::schema::VectorDistanceMetric::L2sq
|
||||
);
|
||||
|
||||
let connectivity = u16::from_le_bytes([blob_data[7], blob_data[8]]);
|
||||
assert_eq!(connectivity, 24);
|
||||
|
||||
let expansion_add = u16::from_le_bytes([blob_data[9], blob_data[10]]);
|
||||
assert_eq!(expansion_add, 200);
|
||||
|
||||
let expansion_search = u16::from_le_bytes([blob_data[11], blob_data[12]]);
|
||||
assert_eq!(expansion_search, 100);
|
||||
|
||||
let total_rows = u64::from_le_bytes([
|
||||
blob_data[13],
|
||||
blob_data[14],
|
||||
blob_data[15],
|
||||
blob_data[16],
|
||||
blob_data[17],
|
||||
blob_data[18],
|
||||
blob_data[19],
|
||||
blob_data[20],
|
||||
]);
|
||||
assert_eq!(total_rows, 5);
|
||||
|
||||
let indexed_rows = u64::from_le_bytes([
|
||||
blob_data[21],
|
||||
blob_data[22],
|
||||
blob_data[23],
|
||||
blob_data[24],
|
||||
blob_data[25],
|
||||
blob_data[26],
|
||||
blob_data[27],
|
||||
blob_data[28],
|
||||
]);
|
||||
assert_eq!(indexed_rows, 3);
|
||||
|
||||
let null_bitmap_len =
|
||||
u32::from_le_bytes([blob_data[29], blob_data[30], blob_data[31], blob_data[32]]);
|
||||
assert_eq!(null_bitmap_len as usize, null_bitmap_bytes.len());
|
||||
assert_eq!(decoded.connectivity, 24);
|
||||
assert_eq!(decoded.expansion_add, 200);
|
||||
assert_eq!(decoded.expansion_search, 100);
|
||||
assert_eq!(decoded.total_rows, 5);
|
||||
assert_eq!(decoded.indexed_rows, 3);
|
||||
assert_eq!(decoded.null_bitmap_len as usize, null_bitmap_bytes.len());
|
||||
|
||||
// Verify null bitmap can be deserialized
|
||||
let null_bitmap_data = &blob_data[header_size..header_size + null_bitmap_len as usize];
|
||||
let null_bitmap_data =
|
||||
&blob_data[header_size..header_size + decoded.null_bitmap_len as usize];
|
||||
let restored_bitmap = RoaringBitmap::deserialize_from(null_bitmap_data).unwrap();
|
||||
assert_eq!(restored_bitmap.len(), 2); // 2 nulls
|
||||
assert!(restored_bitmap.contains(1));
|
||||
assert!(restored_bitmap.contains(3));
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_vector_index_missing_column_as_nulls() {
|
||||
let tempdir = common_test_util::temp_dir::create_temp_dir(
|
||||
"test_vector_index_missing_column_as_nulls_",
|
||||
);
|
||||
let intm_mgr = IntermediateManager::init_fs(tempdir.path().to_string_lossy())
|
||||
.await
|
||||
.unwrap();
|
||||
let region_metadata = mock_region_metadata_with_vector();
|
||||
|
||||
let mut vector_index_options = HashMap::new();
|
||||
vector_index_options.insert(3, VectorIndexOptions::default());
|
||||
|
||||
let mut indexer = VectorIndexer::new(
|
||||
FileId::random(),
|
||||
®ion_metadata,
|
||||
intm_mgr,
|
||||
None,
|
||||
&vector_index_options,
|
||||
)
|
||||
.unwrap()
|
||||
.unwrap();
|
||||
|
||||
let mut batch = new_batch_missing_vector_column(4, 3);
|
||||
indexer.update(&mut batch).await.unwrap();
|
||||
|
||||
let creator = indexer.creators.get(&3).unwrap();
|
||||
assert_eq!(creator.size(), 0);
|
||||
assert_eq!(creator.current_row_offset, 3);
|
||||
assert_eq!(creator.null_bitmap.len(), 3);
|
||||
for idx in 0..3 {
|
||||
assert!(creator.null_bitmap.contains(idx as u32));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
324
src/mito2/src/sst/index/vector_index/format.rs
Normal file
324
src/mito2/src/sst/index/vector_index/format.rs
Normal file
@@ -0,0 +1,324 @@
|
||||
// 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.
|
||||
|
||||
//! Vector index blob format helpers.
|
||||
|
||||
use std::fmt;
|
||||
|
||||
#[cfg(test)]
|
||||
use datatypes::schema::VectorDistanceMetric as SchemaVectorDistanceMetric;
|
||||
#[cfg(test)]
|
||||
use index::vector::distance_metric_to_usearch;
|
||||
use store_api::storage::{VectorDistanceMetric, VectorIndexEngineType};
|
||||
|
||||
pub(crate) const VECTOR_INDEX_BLOB_VERSION: u8 = 1;
|
||||
pub(crate) const VECTOR_INDEX_BLOB_HEADER_SIZE: usize = 33;
|
||||
|
||||
#[derive(Debug, Clone, Copy)]
|
||||
pub(crate) struct VectorIndexBlobHeader {
|
||||
pub engine_type: VectorIndexEngineType,
|
||||
pub dim: u32,
|
||||
pub metric: VectorDistanceMetric,
|
||||
pub connectivity: u16,
|
||||
pub expansion_add: u16,
|
||||
pub expansion_search: u16,
|
||||
pub total_rows: u64,
|
||||
pub indexed_rows: u64,
|
||||
pub null_bitmap_len: u32,
|
||||
}
|
||||
|
||||
impl VectorIndexBlobHeader {
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
pub(crate) fn new(
|
||||
engine_type: VectorIndexEngineType,
|
||||
dim: u32,
|
||||
metric: VectorDistanceMetric,
|
||||
connectivity: u16,
|
||||
expansion_add: u16,
|
||||
expansion_search: u16,
|
||||
total_rows: u64,
|
||||
indexed_rows: u64,
|
||||
null_bitmap_len: u32,
|
||||
) -> Result<Self, VectorIndexBlobFormatError> {
|
||||
if total_rows < indexed_rows {
|
||||
return Err(VectorIndexBlobFormatError::InvalidRowCounts {
|
||||
total: total_rows,
|
||||
indexed: indexed_rows,
|
||||
});
|
||||
}
|
||||
if total_rows > u64::from(u32::MAX) || indexed_rows > u64::from(u32::MAX) {
|
||||
return Err(VectorIndexBlobFormatError::RowsExceedU32 {
|
||||
total: total_rows,
|
||||
indexed: indexed_rows,
|
||||
});
|
||||
}
|
||||
Ok(Self {
|
||||
engine_type,
|
||||
dim,
|
||||
metric,
|
||||
connectivity,
|
||||
expansion_add,
|
||||
expansion_search,
|
||||
total_rows,
|
||||
indexed_rows,
|
||||
null_bitmap_len,
|
||||
})
|
||||
}
|
||||
|
||||
pub(crate) fn encode_into(&self, buf: &mut Vec<u8>) {
|
||||
buf.push(VECTOR_INDEX_BLOB_VERSION);
|
||||
buf.push(self.engine_type.as_u8());
|
||||
buf.extend_from_slice(&self.dim.to_le_bytes());
|
||||
buf.push(self.metric.as_u8());
|
||||
buf.extend_from_slice(&self.connectivity.to_le_bytes());
|
||||
buf.extend_from_slice(&self.expansion_add.to_le_bytes());
|
||||
buf.extend_from_slice(&self.expansion_search.to_le_bytes());
|
||||
buf.extend_from_slice(&self.total_rows.to_le_bytes());
|
||||
buf.extend_from_slice(&self.indexed_rows.to_le_bytes());
|
||||
buf.extend_from_slice(&self.null_bitmap_len.to_le_bytes());
|
||||
}
|
||||
|
||||
pub(crate) fn decode(data: &[u8]) -> Result<(Self, usize), VectorIndexBlobFormatError> {
|
||||
if data.len() < VECTOR_INDEX_BLOB_HEADER_SIZE {
|
||||
return Err(VectorIndexBlobFormatError::Truncated("header"));
|
||||
}
|
||||
|
||||
let mut offset = 0;
|
||||
let version = read_u8(data, &mut offset)?;
|
||||
if version != VECTOR_INDEX_BLOB_VERSION {
|
||||
return Err(VectorIndexBlobFormatError::UnsupportedVersion(version));
|
||||
}
|
||||
|
||||
let engine_type = VectorIndexEngineType::try_from_u8(read_u8(data, &mut offset)?)
|
||||
.ok_or_else(|| VectorIndexBlobFormatError::UnknownEngine(data[offset - 1]))?;
|
||||
let dim = read_u32(data, &mut offset)?;
|
||||
let metric = VectorDistanceMetric::try_from_u8(read_u8(data, &mut offset)?)
|
||||
.ok_or_else(|| VectorIndexBlobFormatError::UnknownMetric(data[offset - 1]))?;
|
||||
let connectivity = read_u16(data, &mut offset)?;
|
||||
let expansion_add = read_u16(data, &mut offset)?;
|
||||
let expansion_search = read_u16(data, &mut offset)?;
|
||||
let total_rows = read_u64(data, &mut offset)?;
|
||||
let indexed_rows = read_u64(data, &mut offset)?;
|
||||
let null_bitmap_len = read_u32(data, &mut offset)?;
|
||||
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
engine_type,
|
||||
dim,
|
||||
metric,
|
||||
connectivity,
|
||||
expansion_add,
|
||||
expansion_search,
|
||||
total_rows,
|
||||
indexed_rows,
|
||||
null_bitmap_len,
|
||||
)?;
|
||||
Ok((header, offset))
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub(crate) enum VectorIndexBlobFormatError {
|
||||
Truncated(&'static str),
|
||||
UnsupportedVersion(u8),
|
||||
UnknownEngine(u8),
|
||||
UnknownMetric(u8),
|
||||
InvalidRowCounts { total: u64, indexed: u64 },
|
||||
RowsExceedU32 { total: u64, indexed: u64 },
|
||||
}
|
||||
|
||||
impl fmt::Display for VectorIndexBlobFormatError {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
match self {
|
||||
Self::Truncated(label) => {
|
||||
write!(f, "Vector index blob truncated while reading {}", label)
|
||||
}
|
||||
Self::UnsupportedVersion(version) => {
|
||||
write!(f, "Unsupported vector index version {}", version)
|
||||
}
|
||||
Self::UnknownEngine(value) => write!(f, "Unknown vector index engine type {}", value),
|
||||
Self::UnknownMetric(value) => write!(f, "Unknown vector index metric {}", value),
|
||||
Self::InvalidRowCounts { total, indexed } => {
|
||||
write!(
|
||||
f,
|
||||
"Total rows {} is smaller than indexed rows {}",
|
||||
total, indexed
|
||||
)
|
||||
}
|
||||
Self::RowsExceedU32 { total, indexed } => {
|
||||
write!(
|
||||
f,
|
||||
"Vector index rows exceed u32::MAX (total={}, indexed={})",
|
||||
total, indexed
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn read_exact<const N: usize>(
|
||||
data: &[u8],
|
||||
offset: &mut usize,
|
||||
label: &'static str,
|
||||
) -> Result<[u8; N], VectorIndexBlobFormatError> {
|
||||
if *offset + N > data.len() {
|
||||
return Err(VectorIndexBlobFormatError::Truncated(label));
|
||||
}
|
||||
let mut buf = [0u8; N];
|
||||
buf.copy_from_slice(&data[*offset..*offset + N]);
|
||||
*offset += N;
|
||||
Ok(buf)
|
||||
}
|
||||
|
||||
fn read_u8(data: &[u8], offset: &mut usize) -> Result<u8, VectorIndexBlobFormatError> {
|
||||
Ok(read_exact::<1>(data, offset, "u8")?[0])
|
||||
}
|
||||
|
||||
fn read_u16(data: &[u8], offset: &mut usize) -> Result<u16, VectorIndexBlobFormatError> {
|
||||
Ok(u16::from_le_bytes(read_exact::<2>(data, offset, "u16")?))
|
||||
}
|
||||
|
||||
fn read_u32(data: &[u8], offset: &mut usize) -> Result<u32, VectorIndexBlobFormatError> {
|
||||
Ok(u32::from_le_bytes(read_exact::<4>(data, offset, "u32")?))
|
||||
}
|
||||
|
||||
fn read_u64(data: &[u8], offset: &mut usize) -> Result<u64, VectorIndexBlobFormatError> {
|
||||
Ok(u64::from_le_bytes(read_exact::<8>(data, offset, "u64")?))
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use roaring::RoaringBitmap;
|
||||
use store_api::storage::VectorIndexEngineType;
|
||||
|
||||
use super::*;
|
||||
use crate::sst::index::vector_index::creator::VectorIndexConfig;
|
||||
use crate::sst::index::vector_index::engine;
|
||||
|
||||
#[test]
|
||||
fn test_vector_index_blob_header_roundtrip() {
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
VectorIndexEngineType::Usearch,
|
||||
4,
|
||||
VectorDistanceMetric::L2sq,
|
||||
24,
|
||||
200,
|
||||
100,
|
||||
5,
|
||||
3,
|
||||
16,
|
||||
)
|
||||
.unwrap();
|
||||
let mut bytes = Vec::new();
|
||||
header.encode_into(&mut bytes);
|
||||
|
||||
let (decoded, offset) = VectorIndexBlobHeader::decode(&bytes).unwrap();
|
||||
assert_eq!(offset, VECTOR_INDEX_BLOB_HEADER_SIZE);
|
||||
assert_eq!(decoded.engine_type, header.engine_type);
|
||||
assert_eq!(decoded.dim, header.dim);
|
||||
assert_eq!(decoded.metric, header.metric);
|
||||
assert_eq!(decoded.connectivity, header.connectivity);
|
||||
assert_eq!(decoded.expansion_add, header.expansion_add);
|
||||
assert_eq!(decoded.expansion_search, header.expansion_search);
|
||||
assert_eq!(decoded.total_rows, header.total_rows);
|
||||
assert_eq!(decoded.indexed_rows, header.indexed_rows);
|
||||
assert_eq!(decoded.null_bitmap_len, header.null_bitmap_len);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_index_blob_header_invalid_version() {
|
||||
let mut blob = vec![0u8; VECTOR_INDEX_BLOB_HEADER_SIZE];
|
||||
blob[0] = 2;
|
||||
assert!(VectorIndexBlobHeader::decode(&blob).is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_index_blob_header_truncated() {
|
||||
let blob = vec![0u8; VECTOR_INDEX_BLOB_HEADER_SIZE - 1];
|
||||
assert!(VectorIndexBlobHeader::decode(&blob).is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_index_blob_parse_roundtrip() {
|
||||
let config = VectorIndexConfig {
|
||||
engine: VectorIndexEngineType::Usearch,
|
||||
dim: 2,
|
||||
metric: distance_metric_to_usearch(VectorDistanceMetric::L2sq),
|
||||
distance_metric: VectorDistanceMetric::L2sq,
|
||||
connectivity: 16,
|
||||
expansion_add: 128,
|
||||
expansion_search: 64,
|
||||
};
|
||||
let mut engine = engine::create_engine(config.engine, &config).unwrap();
|
||||
engine.add(0, &[0.0, 1.0]).unwrap();
|
||||
let index_size = engine.serialized_length();
|
||||
let mut index_bytes = vec![0u8; index_size];
|
||||
engine.save_to_buffer(&mut index_bytes).unwrap();
|
||||
|
||||
let null_bitmap = RoaringBitmap::new();
|
||||
let mut null_bitmap_bytes = Vec::new();
|
||||
null_bitmap.serialize_into(&mut null_bitmap_bytes).unwrap();
|
||||
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
config.engine,
|
||||
config.dim as u32,
|
||||
VectorDistanceMetric::L2sq,
|
||||
config.connectivity as u16,
|
||||
config.expansion_add as u16,
|
||||
config.expansion_search as u16,
|
||||
1,
|
||||
1,
|
||||
null_bitmap_bytes.len() as u32,
|
||||
)
|
||||
.unwrap();
|
||||
let mut blob = Vec::new();
|
||||
header.encode_into(&mut blob);
|
||||
blob.extend_from_slice(&null_bitmap_bytes);
|
||||
blob.extend_from_slice(&index_bytes);
|
||||
|
||||
let (decoded, offset) = VectorIndexBlobHeader::decode(&blob).unwrap();
|
||||
let null_bitmap_len = decoded.null_bitmap_len as usize;
|
||||
let null_bitmap_data = &blob[offset..offset + null_bitmap_len];
|
||||
let restored_bitmap = RoaringBitmap::deserialize_from(null_bitmap_data).unwrap();
|
||||
|
||||
assert_eq!(decoded.metric, VectorDistanceMetric::L2sq);
|
||||
assert_eq!(decoded.total_rows, 1);
|
||||
assert_eq!(decoded.indexed_rows, 1);
|
||||
assert_eq!(restored_bitmap.len(), 0);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_index_blob_header_format_matches_creator() {
|
||||
let header = VectorIndexBlobHeader::new(
|
||||
VectorIndexEngineType::Usearch,
|
||||
4,
|
||||
VectorDistanceMetric::L2sq,
|
||||
24,
|
||||
200,
|
||||
100,
|
||||
5,
|
||||
3,
|
||||
2,
|
||||
)
|
||||
.unwrap();
|
||||
let mut bytes = Vec::new();
|
||||
header.encode_into(&mut bytes);
|
||||
|
||||
let (decoded, header_size) = VectorIndexBlobHeader::decode(&bytes).unwrap();
|
||||
assert_eq!(header_size, VECTOR_INDEX_BLOB_HEADER_SIZE);
|
||||
assert_eq!(decoded.metric, SchemaVectorDistanceMetric::L2sq);
|
||||
assert_eq!(decoded.total_rows, 5);
|
||||
assert_eq!(decoded.indexed_rows, 3);
|
||||
assert_eq!(decoded.null_bitmap_len, 2);
|
||||
}
|
||||
}
|
||||
@@ -14,8 +14,10 @@
|
||||
|
||||
//! Vector index module for HNSW-based approximate nearest neighbor search.
|
||||
|
||||
pub(crate) mod applier;
|
||||
pub(crate) mod creator;
|
||||
pub(crate) mod engine;
|
||||
pub(crate) mod format;
|
||||
pub(crate) mod util;
|
||||
|
||||
/// The blob type identifier for vector index in puffin files.
|
||||
|
||||
@@ -13,6 +13,7 @@
|
||||
// limitations under the License.
|
||||
|
||||
use std::result::Result as StdResult;
|
||||
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
|
||||
|
||||
use bytes::Bytes;
|
||||
use futures::FutureExt;
|
||||
@@ -24,6 +25,7 @@ use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader};
|
||||
use snafu::{IntoError as _, ResultExt};
|
||||
|
||||
use crate::error::{self, Result};
|
||||
use crate::sst::parquet::reader::MetadataCacheMetrics;
|
||||
|
||||
/// The estimated size of the footer and metadata need to read from the end of parquet file.
|
||||
const DEFAULT_PREFETCH_SIZE: u64 = 64 * 1024;
|
||||
@@ -65,24 +67,33 @@ impl<'a> MetadataLoader<'a> {
|
||||
Ok(file_size)
|
||||
}
|
||||
|
||||
pub async fn load(&self) -> Result<ParquetMetaData> {
|
||||
pub async fn load(&self, cache_metrics: &mut MetadataCacheMetrics) -> Result<ParquetMetaData> {
|
||||
let path = self.file_path;
|
||||
let file_size = self.get_file_size().await?;
|
||||
let reader =
|
||||
ParquetMetaDataReader::new().with_prefetch_hint(Some(DEFAULT_PREFETCH_SIZE as usize));
|
||||
|
||||
let num_reads = AtomicUsize::new(0);
|
||||
let bytes_read = AtomicU64::new(0);
|
||||
let fetch = ObjectStoreFetch {
|
||||
object_store: &self.object_store,
|
||||
file_path: self.file_path,
|
||||
num_reads: &num_reads,
|
||||
bytes_read: &bytes_read,
|
||||
};
|
||||
|
||||
reader
|
||||
let metadata = reader
|
||||
.load_and_finish(fetch, file_size)
|
||||
.await
|
||||
.map_err(|e| match unbox_external_error(e) {
|
||||
Ok(os_err) => error::OpenDalSnafu {}.into_error(os_err),
|
||||
Err(parquet_err) => error::ReadParquetSnafu { path }.into_error(parquet_err),
|
||||
})
|
||||
})?;
|
||||
|
||||
cache_metrics.num_reads = num_reads.into_inner();
|
||||
cache_metrics.bytes_read = bytes_read.into_inner();
|
||||
|
||||
Ok(metadata)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -100,10 +111,13 @@ fn unbox_external_error(e: ParquetError) -> StdResult<object_store::Error, Parqu
|
||||
struct ObjectStoreFetch<'a> {
|
||||
object_store: &'a ObjectStore,
|
||||
file_path: &'a str,
|
||||
num_reads: &'a AtomicUsize,
|
||||
bytes_read: &'a AtomicU64,
|
||||
}
|
||||
|
||||
impl MetadataFetch for ObjectStoreFetch<'_> {
|
||||
fn fetch(&mut self, range: std::ops::Range<u64>) -> BoxFuture<'_, ParquetResult<Bytes>> {
|
||||
let bytes_to_read = range.end - range.start;
|
||||
async move {
|
||||
let data = self
|
||||
.object_store
|
||||
@@ -111,6 +125,8 @@ impl MetadataFetch for ObjectStoreFetch<'_> {
|
||||
.range(range)
|
||||
.await
|
||||
.map_err(|e| ParquetError::External(Box::new(e)))?;
|
||||
self.num_reads.fetch_add(1, Ordering::Relaxed);
|
||||
self.bytes_read.fetch_add(bytes_to_read, Ordering::Relaxed);
|
||||
Ok(data.to_bytes())
|
||||
}
|
||||
.boxed()
|
||||
|
||||
@@ -14,6 +14,8 @@
|
||||
|
||||
//! Parquet reader.
|
||||
|
||||
#[cfg(feature = "vector_index")]
|
||||
use std::collections::BTreeSet;
|
||||
use std::collections::VecDeque;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant};
|
||||
@@ -41,6 +43,8 @@ use table::predicate::Predicate;
|
||||
|
||||
use crate::cache::CacheStrategy;
|
||||
use crate::cache::index::result_cache::PredicateKey;
|
||||
#[cfg(feature = "vector_index")]
|
||||
use crate::error::ApplyVectorIndexSnafu;
|
||||
use crate::error::{
|
||||
ArrowReaderSnafu, InvalidMetadataSnafu, InvalidParquetSnafu, ReadDataPartSnafu,
|
||||
ReadParquetSnafu, Result,
|
||||
@@ -61,6 +65,8 @@ use crate::sst::index::fulltext_index::applier::{
|
||||
use crate::sst::index::inverted_index::applier::{
|
||||
InvertedIndexApplierRef, InvertedIndexApplyMetrics,
|
||||
};
|
||||
#[cfg(feature = "vector_index")]
|
||||
use crate::sst::index::vector_index::applier::VectorIndexApplierRef;
|
||||
use crate::sst::parquet::file_range::{
|
||||
FileRangeContext, FileRangeContextRef, PreFilterMode, RangeBase, row_group_contains_delete,
|
||||
};
|
||||
@@ -74,6 +80,7 @@ use crate::sst::parquet::{DEFAULT_READ_BATCH_SIZE, PARQUET_METADATA_KEY};
|
||||
const INDEX_TYPE_FULLTEXT: &str = "fulltext";
|
||||
const INDEX_TYPE_INVERTED: &str = "inverted";
|
||||
const INDEX_TYPE_BLOOM: &str = "bloom filter";
|
||||
const INDEX_TYPE_VECTOR: &str = "vector";
|
||||
|
||||
macro_rules! handle_index_error {
|
||||
($err:expr, $file_handle:expr, $index_type:expr) => {
|
||||
@@ -117,6 +124,12 @@ pub struct ParquetReaderBuilder {
|
||||
inverted_index_appliers: [Option<InvertedIndexApplierRef>; 2],
|
||||
bloom_filter_index_appliers: [Option<BloomFilterIndexApplierRef>; 2],
|
||||
fulltext_index_appliers: [Option<FulltextIndexApplierRef>; 2],
|
||||
/// Vector index applier for KNN search.
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_applier: Option<VectorIndexApplierRef>,
|
||||
/// Over-fetched k for vector index scan.
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_k: Option<usize>,
|
||||
/// Expected metadata of the region while reading the SST.
|
||||
/// This is usually the latest metadata of the region. The reader use
|
||||
/// it get the correct column id of a column by name.
|
||||
@@ -150,6 +163,10 @@ impl ParquetReaderBuilder {
|
||||
inverted_index_appliers: [None, None],
|
||||
bloom_filter_index_appliers: [None, None],
|
||||
fulltext_index_appliers: [None, None],
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_applier: None,
|
||||
#[cfg(feature = "vector_index")]
|
||||
vector_index_k: None,
|
||||
expected_metadata: None,
|
||||
flat_format: false,
|
||||
compaction: false,
|
||||
@@ -211,6 +228,19 @@ impl ParquetReaderBuilder {
|
||||
self
|
||||
}
|
||||
|
||||
/// Attaches the vector index applier to the builder.
|
||||
#[cfg(feature = "vector_index")]
|
||||
#[must_use]
|
||||
pub(crate) fn vector_index_applier(
|
||||
mut self,
|
||||
applier: Option<VectorIndexApplierRef>,
|
||||
k: Option<usize>,
|
||||
) -> Self {
|
||||
self.vector_index_applier = applier;
|
||||
self.vector_index_k = k;
|
||||
self
|
||||
}
|
||||
|
||||
/// Attaches the expected metadata to the builder.
|
||||
#[must_use]
|
||||
pub fn expected_metadata(mut self, expected_metadata: Option<RegionMetadataRef>) -> Self {
|
||||
@@ -468,7 +498,7 @@ impl ParquetReaderBuilder {
|
||||
|
||||
// Cache miss, load metadata directly.
|
||||
let metadata_loader = MetadataLoader::new(self.object_store.clone(), file_path, file_size);
|
||||
let metadata = metadata_loader.load().await?;
|
||||
let metadata = metadata_loader.load(cache_metrics).await?;
|
||||
|
||||
let metadata = Arc::new(metadata);
|
||||
// Cache the metadata.
|
||||
@@ -572,6 +602,19 @@ impl ParquetReaderBuilder {
|
||||
)
|
||||
.await;
|
||||
}
|
||||
#[cfg(feature = "vector_index")]
|
||||
{
|
||||
self.prune_row_groups_by_vector_index(
|
||||
row_group_size,
|
||||
num_row_groups,
|
||||
&mut output,
|
||||
metrics,
|
||||
)
|
||||
.await;
|
||||
if output.is_empty() {
|
||||
return output;
|
||||
}
|
||||
}
|
||||
output
|
||||
}
|
||||
|
||||
@@ -606,11 +649,13 @@ impl ParquetReaderBuilder {
|
||||
&& all_required_row_groups_searched(output, result)
|
||||
{
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_FULLTEXT);
|
||||
metrics.fulltext_index_cache_hit += 1;
|
||||
pruned = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
metrics.fulltext_index_cache_miss += 1;
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let apply_res = index_applier
|
||||
.apply_fine(
|
||||
@@ -678,11 +723,13 @@ impl ParquetReaderBuilder {
|
||||
&& all_required_row_groups_searched(output, result)
|
||||
{
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_INVERTED);
|
||||
metrics.inverted_index_cache_hit += 1;
|
||||
pruned = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
metrics.inverted_index_cache_miss += 1;
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let apply_res = index_applier
|
||||
.apply(
|
||||
@@ -746,11 +793,13 @@ impl ParquetReaderBuilder {
|
||||
&& all_required_row_groups_searched(output, result)
|
||||
{
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_BLOOM);
|
||||
metrics.bloom_filter_cache_hit += 1;
|
||||
pruned = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
metrics.bloom_filter_cache_miss += 1;
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let rgs = parquet_meta.row_groups().iter().enumerate().map(|(i, rg)| {
|
||||
(
|
||||
@@ -799,6 +848,48 @@ impl ParquetReaderBuilder {
|
||||
pruned
|
||||
}
|
||||
|
||||
/// Prunes row groups by vector index results.
|
||||
#[cfg(feature = "vector_index")]
|
||||
async fn prune_row_groups_by_vector_index(
|
||||
&self,
|
||||
row_group_size: usize,
|
||||
num_row_groups: usize,
|
||||
output: &mut RowGroupSelection,
|
||||
metrics: &mut ReaderFilterMetrics,
|
||||
) {
|
||||
let Some(applier) = &self.vector_index_applier else {
|
||||
return;
|
||||
};
|
||||
let Some(k) = self.vector_index_k else {
|
||||
return;
|
||||
};
|
||||
if !self.file_handle.meta_ref().vector_index_available() {
|
||||
return;
|
||||
}
|
||||
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let apply_res = applier
|
||||
.apply_with_k(self.file_handle.index_id(), Some(file_size_hint), k)
|
||||
.await;
|
||||
let row_ids = match apply_res {
|
||||
Ok(res) => res.row_offsets,
|
||||
Err(err) => {
|
||||
handle_index_error!(err, self.file_handle, INDEX_TYPE_VECTOR);
|
||||
return;
|
||||
}
|
||||
};
|
||||
|
||||
let selection = match vector_selection_from_offsets(row_ids, row_group_size, num_row_groups)
|
||||
{
|
||||
Ok(selection) => selection,
|
||||
Err(err) => {
|
||||
handle_index_error!(err, self.file_handle, INDEX_TYPE_VECTOR);
|
||||
return;
|
||||
}
|
||||
};
|
||||
apply_selection_and_update_metrics(output, &selection, metrics, INDEX_TYPE_VECTOR);
|
||||
}
|
||||
|
||||
async fn prune_row_groups_by_fulltext_bloom(
|
||||
&self,
|
||||
row_group_size: usize,
|
||||
@@ -829,11 +920,13 @@ impl ParquetReaderBuilder {
|
||||
&& all_required_row_groups_searched(output, result)
|
||||
{
|
||||
apply_selection_and_update_metrics(output, result, metrics, INDEX_TYPE_FULLTEXT);
|
||||
metrics.fulltext_index_cache_hit += 1;
|
||||
pruned = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
// Slow path: apply the index from the file.
|
||||
metrics.fulltext_index_cache_miss += 1;
|
||||
let file_size_hint = self.file_handle.meta_ref().index_file_size();
|
||||
let rgs = parquet_meta.row_groups().iter().enumerate().map(|(i, rg)| {
|
||||
(
|
||||
@@ -983,6 +1076,29 @@ fn apply_selection_and_update_metrics(
|
||||
*output = intersection;
|
||||
}
|
||||
|
||||
#[cfg(feature = "vector_index")]
|
||||
fn vector_selection_from_offsets(
|
||||
row_offsets: Vec<u64>,
|
||||
row_group_size: usize,
|
||||
num_row_groups: usize,
|
||||
) -> Result<RowGroupSelection> {
|
||||
let mut row_ids = BTreeSet::new();
|
||||
for offset in row_offsets {
|
||||
let row_id = u32::try_from(offset).map_err(|_| {
|
||||
ApplyVectorIndexSnafu {
|
||||
reason: format!("Row offset {} exceeds u32::MAX", offset),
|
||||
}
|
||||
.build()
|
||||
})?;
|
||||
row_ids.insert(row_id);
|
||||
}
|
||||
Ok(RowGroupSelection::from_row_ids(
|
||||
row_ids,
|
||||
row_group_size,
|
||||
num_row_groups,
|
||||
))
|
||||
}
|
||||
|
||||
fn all_required_row_groups_searched(
|
||||
required_row_groups: &RowGroupSelection,
|
||||
cached_row_groups: &RowGroupSelection,
|
||||
@@ -1008,6 +1124,8 @@ pub(crate) struct ReaderFilterMetrics {
|
||||
pub(crate) rg_minmax_filtered: usize,
|
||||
/// Number of row groups filtered by bloom filter index.
|
||||
pub(crate) rg_bloom_filtered: usize,
|
||||
/// Number of row groups filtered by vector index.
|
||||
pub(crate) rg_vector_filtered: usize,
|
||||
|
||||
/// Number of rows in row group before filtering.
|
||||
pub(crate) rows_total: usize,
|
||||
@@ -1017,9 +1135,24 @@ pub(crate) struct ReaderFilterMetrics {
|
||||
pub(crate) rows_inverted_filtered: usize,
|
||||
/// Number of rows in row group filtered by bloom filter index.
|
||||
pub(crate) rows_bloom_filtered: usize,
|
||||
/// Number of rows filtered by vector index.
|
||||
pub(crate) rows_vector_filtered: usize,
|
||||
/// Number of rows filtered by precise filter.
|
||||
pub(crate) rows_precise_filtered: usize,
|
||||
|
||||
/// Number of index result cache hits for fulltext index.
|
||||
pub(crate) fulltext_index_cache_hit: usize,
|
||||
/// Number of index result cache misses for fulltext index.
|
||||
pub(crate) fulltext_index_cache_miss: usize,
|
||||
/// Number of index result cache hits for inverted index.
|
||||
pub(crate) inverted_index_cache_hit: usize,
|
||||
/// Number of index result cache misses for inverted index.
|
||||
pub(crate) inverted_index_cache_miss: usize,
|
||||
/// Number of index result cache hits for bloom filter index.
|
||||
pub(crate) bloom_filter_cache_hit: usize,
|
||||
/// Number of index result cache misses for bloom filter index.
|
||||
pub(crate) bloom_filter_cache_miss: usize,
|
||||
|
||||
/// Optional metrics for inverted index applier.
|
||||
pub(crate) inverted_index_apply_metrics: Option<InvertedIndexApplyMetrics>,
|
||||
/// Optional metrics for bloom filter index applier.
|
||||
@@ -1036,13 +1169,22 @@ impl ReaderFilterMetrics {
|
||||
self.rg_inverted_filtered += other.rg_inverted_filtered;
|
||||
self.rg_minmax_filtered += other.rg_minmax_filtered;
|
||||
self.rg_bloom_filtered += other.rg_bloom_filtered;
|
||||
self.rg_vector_filtered += other.rg_vector_filtered;
|
||||
|
||||
self.rows_total += other.rows_total;
|
||||
self.rows_fulltext_filtered += other.rows_fulltext_filtered;
|
||||
self.rows_inverted_filtered += other.rows_inverted_filtered;
|
||||
self.rows_bloom_filtered += other.rows_bloom_filtered;
|
||||
self.rows_vector_filtered += other.rows_vector_filtered;
|
||||
self.rows_precise_filtered += other.rows_precise_filtered;
|
||||
|
||||
self.fulltext_index_cache_hit += other.fulltext_index_cache_hit;
|
||||
self.fulltext_index_cache_miss += other.fulltext_index_cache_miss;
|
||||
self.inverted_index_cache_hit += other.inverted_index_cache_hit;
|
||||
self.inverted_index_cache_miss += other.inverted_index_cache_miss;
|
||||
self.bloom_filter_cache_hit += other.bloom_filter_cache_hit;
|
||||
self.bloom_filter_cache_miss += other.bloom_filter_cache_miss;
|
||||
|
||||
// Merge optional applier metrics
|
||||
if let Some(other_metrics) = &other.inverted_index_apply_metrics {
|
||||
self.inverted_index_apply_metrics
|
||||
@@ -1078,6 +1220,9 @@ impl ReaderFilterMetrics {
|
||||
READ_ROW_GROUPS_TOTAL
|
||||
.with_label_values(&["bloom_filter_index_filtered"])
|
||||
.inc_by(self.rg_bloom_filtered as u64);
|
||||
READ_ROW_GROUPS_TOTAL
|
||||
.with_label_values(&["vector_index_filtered"])
|
||||
.inc_by(self.rg_vector_filtered as u64);
|
||||
|
||||
PRECISE_FILTER_ROWS_TOTAL
|
||||
.with_label_values(&["parquet"])
|
||||
@@ -1094,6 +1239,9 @@ impl ReaderFilterMetrics {
|
||||
READ_ROWS_IN_ROW_GROUP_TOTAL
|
||||
.with_label_values(&["bloom_filter_index_filtered"])
|
||||
.inc_by(self.rows_bloom_filtered as u64);
|
||||
READ_ROWS_IN_ROW_GROUP_TOTAL
|
||||
.with_label_values(&["vector_index_filtered"])
|
||||
.inc_by(self.rows_vector_filtered as u64);
|
||||
}
|
||||
|
||||
fn update_index_metrics(&mut self, index_type: &str, row_group_count: usize, row_count: usize) {
|
||||
@@ -1110,11 +1258,67 @@ impl ReaderFilterMetrics {
|
||||
self.rg_bloom_filtered += row_group_count;
|
||||
self.rows_bloom_filtered += row_count;
|
||||
}
|
||||
INDEX_TYPE_VECTOR => {
|
||||
self.rg_vector_filtered += row_group_count;
|
||||
self.rows_vector_filtered += row_count;
|
||||
}
|
||||
_ => {}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(all(test, feature = "vector_index"))]
|
||||
mod tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_vector_selection_from_offsets() {
|
||||
let row_group_size = 4;
|
||||
let num_row_groups = 3;
|
||||
let selection =
|
||||
vector_selection_from_offsets(vec![0, 1, 5, 9], row_group_size, num_row_groups)
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(selection.row_group_count(), 3);
|
||||
assert_eq!(selection.row_count(), 4);
|
||||
assert!(selection.contains_non_empty_row_group(0));
|
||||
assert!(selection.contains_non_empty_row_group(1));
|
||||
assert!(selection.contains_non_empty_row_group(2));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_selection_from_offsets_out_of_range() {
|
||||
let row_group_size = 4;
|
||||
let num_row_groups = 2;
|
||||
let selection = vector_selection_from_offsets(
|
||||
vec![0, 7, u64::from(u32::MAX) + 1],
|
||||
row_group_size,
|
||||
num_row_groups,
|
||||
);
|
||||
assert!(selection.is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_vector_selection_updates_metrics() {
|
||||
let row_group_size = 4;
|
||||
let total_rows = 8;
|
||||
let mut output = RowGroupSelection::new(row_group_size, total_rows);
|
||||
let selection = vector_selection_from_offsets(vec![1], row_group_size, 2).unwrap();
|
||||
let mut metrics = ReaderFilterMetrics::default();
|
||||
|
||||
apply_selection_and_update_metrics(
|
||||
&mut output,
|
||||
&selection,
|
||||
&mut metrics,
|
||||
INDEX_TYPE_VECTOR,
|
||||
);
|
||||
|
||||
assert_eq!(metrics.rg_vector_filtered, 1);
|
||||
assert_eq!(metrics.rows_vector_filtered, 7);
|
||||
assert_eq!(output.row_count(), 1);
|
||||
}
|
||||
}
|
||||
|
||||
/// Metrics for parquet metadata cache operations.
|
||||
#[derive(Default, Clone, Copy)]
|
||||
pub(crate) struct MetadataCacheMetrics {
|
||||
@@ -1126,6 +1330,10 @@ pub(crate) struct MetadataCacheMetrics {
|
||||
pub(crate) cache_miss: usize,
|
||||
/// Duration to load parquet metadata.
|
||||
pub(crate) metadata_load_cost: Duration,
|
||||
/// Number of read operations performed.
|
||||
pub(crate) num_reads: usize,
|
||||
/// Total bytes read from storage.
|
||||
pub(crate) bytes_read: u64,
|
||||
}
|
||||
|
||||
impl std::fmt::Debug for MetadataCacheMetrics {
|
||||
@@ -1135,6 +1343,8 @@ impl std::fmt::Debug for MetadataCacheMetrics {
|
||||
file_cache_hit,
|
||||
cache_miss,
|
||||
metadata_load_cost,
|
||||
num_reads,
|
||||
bytes_read,
|
||||
} = self;
|
||||
|
||||
if self.is_empty() {
|
||||
@@ -1153,6 +1363,12 @@ impl std::fmt::Debug for MetadataCacheMetrics {
|
||||
if *cache_miss > 0 {
|
||||
write!(f, ", \"cache_miss\":{}", cache_miss)?;
|
||||
}
|
||||
if *num_reads > 0 {
|
||||
write!(f, ", \"num_reads\":{}", num_reads)?;
|
||||
}
|
||||
if *bytes_read > 0 {
|
||||
write!(f, ", \"bytes_read\":{}", bytes_read)?;
|
||||
}
|
||||
|
||||
write!(f, "}}")
|
||||
}
|
||||
@@ -1170,6 +1386,8 @@ impl MetadataCacheMetrics {
|
||||
self.file_cache_hit += other.file_cache_hit;
|
||||
self.cache_miss += other.cache_miss;
|
||||
self.metadata_load_cost += other.metadata_load_cost;
|
||||
self.num_reads += other.num_reads;
|
||||
self.bytes_read += other.bytes_read;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -520,9 +520,10 @@ async fn edit_region(
|
||||
{
|
||||
// Triggers the filling of the parquet metadata cache.
|
||||
// The parquet file is already downloaded.
|
||||
let mut cache_metrics = Default::default();
|
||||
let _ = write_cache
|
||||
.file_cache()
|
||||
.get_parquet_meta_data(index_key)
|
||||
.get_parquet_meta_data(index_key, &mut cache_metrics)
|
||||
.await;
|
||||
|
||||
listener.on_file_cache_filled(index_key.file_id);
|
||||
|
||||
@@ -736,7 +736,14 @@ pub enum Error {
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to deserialize partition expression: {}", source))]
|
||||
#[snafu(display("Failed to serialize partition expression"))]
|
||||
SerializePartitionExpr {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
source: partition::error::Error,
|
||||
},
|
||||
|
||||
#[snafu(display("Failed to deserialize partition expression"))]
|
||||
DeserializePartitionExpr {
|
||||
#[snafu(source)]
|
||||
source: partition::error::Error,
|
||||
@@ -983,7 +990,8 @@ impl ErrorExt for Error {
|
||||
| Error::MissingInsertBody { .. } => StatusCode::Internal,
|
||||
Error::ExecuteAdminFunction { .. }
|
||||
| Error::EncodeJson { .. }
|
||||
| Error::DeserializePartitionExpr { .. } => StatusCode::Unexpected,
|
||||
| Error::DeserializePartitionExpr { .. }
|
||||
| Error::SerializePartitionExpr { .. } => StatusCode::Unexpected,
|
||||
Error::ViewNotFound { .. }
|
||||
| Error::ViewInfoNotFound { .. }
|
||||
| Error::TableNotFound { .. } => StatusCode::TableNotFound,
|
||||
|
||||
@@ -16,9 +16,11 @@ use std::collections::{HashMap, HashSet};
|
||||
use std::sync::Arc;
|
||||
|
||||
use api::helper::ColumnDataTypeWrapper;
|
||||
use api::v1::alter_table_expr::Kind;
|
||||
use api::v1::meta::CreateFlowTask as PbCreateFlowTask;
|
||||
use api::v1::{
|
||||
AlterDatabaseExpr, AlterTableExpr, CreateFlowExpr, CreateTableExpr, CreateViewExpr, column_def,
|
||||
AlterDatabaseExpr, AlterTableExpr, CreateFlowExpr, CreateTableExpr, CreateViewExpr,
|
||||
Repartition, column_def,
|
||||
};
|
||||
#[cfg(feature = "enterprise")]
|
||||
use api::v1::{
|
||||
@@ -88,8 +90,9 @@ use crate::error::{
|
||||
FlowNotFoundSnafu, InvalidPartitionRuleSnafu, InvalidPartitionSnafu, InvalidSqlSnafu,
|
||||
InvalidTableNameSnafu, InvalidViewNameSnafu, InvalidViewStmtSnafu, NotSupportedSnafu,
|
||||
PartitionExprToPbSnafu, Result, SchemaInUseSnafu, SchemaNotFoundSnafu, SchemaReadOnlySnafu,
|
||||
SubstraitCodecSnafu, TableAlreadyExistsSnafu, TableMetadataManagerSnafu, TableNotFoundSnafu,
|
||||
UnrecognizedTableOptionSnafu, ViewAlreadyExistsSnafu,
|
||||
SerializePartitionExprSnafu, SubstraitCodecSnafu, TableAlreadyExistsSnafu,
|
||||
TableMetadataManagerSnafu, TableNotFoundSnafu, UnrecognizedTableOptionSnafu,
|
||||
ViewAlreadyExistsSnafu,
|
||||
};
|
||||
use crate::expr_helper::{self, RepartitionRequest};
|
||||
use crate::statement::StatementExecutor;
|
||||
@@ -1408,20 +1411,56 @@ impl StatementExecutor {
|
||||
.context(InvalidPartitionSnafu)?;
|
||||
|
||||
info!(
|
||||
"Repartition table {} (table_id={}) from {:?} to {:?}, new partition count: {}",
|
||||
"Submitting repartition task for table {} (table_id={}), from {} to {} partitions",
|
||||
table_ref,
|
||||
table_id,
|
||||
from_partition_exprs,
|
||||
into_partition_exprs,
|
||||
from_partition_exprs.len(),
|
||||
new_partition_exprs_len
|
||||
);
|
||||
|
||||
// TODO(weny): Implement the repartition procedure submission.
|
||||
// The repartition procedure infrastructure is not yet fully integrated with the DDL task system.
|
||||
NotSupportedSnafu {
|
||||
feat: "ALTER TABLE REPARTITION",
|
||||
}
|
||||
.fail()
|
||||
let serialize_exprs = |exprs: Vec<PartitionExpr>| -> Result<Vec<String>> {
|
||||
let mut json_exprs = Vec::with_capacity(exprs.len());
|
||||
for expr in exprs {
|
||||
json_exprs.push(expr.as_json_str().context(SerializePartitionExprSnafu)?);
|
||||
}
|
||||
Ok(json_exprs)
|
||||
};
|
||||
let from_partition_exprs_json = serialize_exprs(from_partition_exprs)?;
|
||||
let into_partition_exprs_json = serialize_exprs(into_partition_exprs)?;
|
||||
let req = SubmitDdlTaskRequest {
|
||||
query_context: query_context.clone(),
|
||||
task: DdlTask::new_alter_table(AlterTableExpr {
|
||||
catalog_name: request.catalog_name.clone(),
|
||||
schema_name: request.schema_name.clone(),
|
||||
table_name: request.table_name.clone(),
|
||||
kind: Some(Kind::Repartition(Repartition {
|
||||
from_partition_exprs: from_partition_exprs_json,
|
||||
into_partition_exprs: into_partition_exprs_json,
|
||||
// TODO(weny): allow passing 'wait' from SQL options or QueryContext
|
||||
wait: true,
|
||||
})),
|
||||
}),
|
||||
};
|
||||
let invalidate_keys = vec![
|
||||
CacheIdent::TableId(table_id),
|
||||
CacheIdent::TableName(TableName::new(
|
||||
request.catalog_name,
|
||||
request.schema_name,
|
||||
request.table_name,
|
||||
)),
|
||||
];
|
||||
self.procedure_executor
|
||||
.submit_ddl_task(&ExecutorContext::default(), req)
|
||||
.await
|
||||
.context(error::ExecuteDdlSnafu)?;
|
||||
|
||||
// Invalidates local cache ASAP.
|
||||
self.cache_invalidator
|
||||
.invalidate(&Context::default(), &invalidate_keys)
|
||||
.await
|
||||
.context(error::InvalidateTableCacheSnafu)?;
|
||||
|
||||
Ok(Output::new_with_affected_rows(0))
|
||||
}
|
||||
|
||||
#[tracing::instrument(skip_all)]
|
||||
@@ -2169,6 +2208,7 @@ mod test {
|
||||
use sql::dialect::GreptimeDbDialect;
|
||||
use sql::parser::{ParseOptions, ParserContext};
|
||||
use sql::statements::statement::Statement;
|
||||
use sqlparser::parser::Parser;
|
||||
|
||||
use super::*;
|
||||
use crate::expr_helper;
|
||||
@@ -2186,6 +2226,39 @@ mod test {
|
||||
assert!(!NAME_PATTERN_REG.is_match("#"));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_partition_expr_equivalence_with_swapped_operands() {
|
||||
let column_name = "device_id".to_string();
|
||||
let column_name_and_type =
|
||||
HashMap::from([(&column_name, ConcreteDataType::int32_datatype())]);
|
||||
let timezone = Timezone::from_tz_string("UTC").unwrap();
|
||||
let dialect = GreptimeDbDialect {};
|
||||
|
||||
let mut parser = Parser::new(&dialect)
|
||||
.try_with_sql("device_id < 100")
|
||||
.unwrap();
|
||||
let expr_left = parser.parse_expr().unwrap();
|
||||
|
||||
let mut parser = Parser::new(&dialect)
|
||||
.try_with_sql("100 > device_id")
|
||||
.unwrap();
|
||||
let expr_right = parser.parse_expr().unwrap();
|
||||
|
||||
let partition_left =
|
||||
convert_one_expr(&expr_left, &column_name_and_type, &timezone).unwrap();
|
||||
let partition_right =
|
||||
convert_one_expr(&expr_right, &column_name_and_type, &timezone).unwrap();
|
||||
|
||||
assert_eq!(partition_left, partition_right);
|
||||
assert!([partition_left.clone()].contains(&partition_right));
|
||||
|
||||
let mut physical_partition_exprs = vec![partition_left];
|
||||
let mut logical_partition_exprs = vec![partition_right];
|
||||
physical_partition_exprs.sort_unstable();
|
||||
logical_partition_exprs.sort_unstable();
|
||||
assert_eq!(physical_partition_exprs, logical_partition_exprs);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
#[ignore = "TODO(ruihang): WIP new partition rule"]
|
||||
async fn test_parse_partitions() {
|
||||
|
||||
@@ -185,6 +185,19 @@ impl RestrictedOp {
|
||||
Self::Or => ParserBinaryOperator::Or,
|
||||
}
|
||||
}
|
||||
|
||||
fn invert_for_swap(&self) -> Self {
|
||||
match self {
|
||||
Self::Eq => Self::Eq,
|
||||
Self::NotEq => Self::NotEq,
|
||||
Self::Lt => Self::Gt,
|
||||
Self::LtEq => Self::GtEq,
|
||||
Self::Gt => Self::Lt,
|
||||
Self::GtEq => Self::LtEq,
|
||||
Self::And => Self::And,
|
||||
Self::Or => Self::Or,
|
||||
}
|
||||
}
|
||||
}
|
||||
impl Display for RestrictedOp {
|
||||
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
|
||||
@@ -208,6 +221,32 @@ impl PartitionExpr {
|
||||
op,
|
||||
rhs: Box::new(rhs),
|
||||
}
|
||||
.canonicalize()
|
||||
}
|
||||
|
||||
/// Canonicalize to `Column op Value` form when possible for consistent equality checks.
|
||||
pub fn canonicalize(self) -> Self {
|
||||
let lhs = Self::canonicalize_operand(*self.lhs);
|
||||
let rhs = Self::canonicalize_operand(*self.rhs);
|
||||
let mut expr = Self {
|
||||
lhs: Box::new(lhs),
|
||||
op: self.op,
|
||||
rhs: Box::new(rhs),
|
||||
};
|
||||
|
||||
if matches!(&*expr.lhs, Operand::Value(_)) && matches!(&*expr.rhs, Operand::Column(_)) {
|
||||
std::mem::swap(&mut expr.lhs, &mut expr.rhs);
|
||||
expr.op = expr.op.invert_for_swap();
|
||||
}
|
||||
|
||||
expr
|
||||
}
|
||||
|
||||
fn canonicalize_operand(operand: Operand) -> Operand {
|
||||
match operand {
|
||||
Operand::Expr(expr) => Operand::Expr(expr.canonicalize()),
|
||||
other => other,
|
||||
}
|
||||
}
|
||||
|
||||
/// Convert [Self] back to sqlparser's [Expr]
|
||||
@@ -354,7 +393,7 @@ impl PartitionExpr {
|
||||
|
||||
let bound: PartitionBound = serde_json::from_str(s).context(error::DeserializeJsonSnafu)?;
|
||||
match bound {
|
||||
PartitionBound::Expr(expr) => Ok(Some(expr)),
|
||||
PartitionBound::Expr(expr) => Ok(Some(expr.canonicalize())),
|
||||
_ => Ok(None),
|
||||
}
|
||||
}
|
||||
@@ -494,7 +533,7 @@ mod tests {
|
||||
.try_as_logical_expr()
|
||||
.unwrap()
|
||||
.to_string(),
|
||||
"Int64(10) < a OR a IS NULL"
|
||||
"a > Int64(10) OR a IS NULL"
|
||||
);
|
||||
|
||||
// Test Gt with column on LHS
|
||||
@@ -519,7 +558,7 @@ mod tests {
|
||||
.try_as_logical_expr()
|
||||
.unwrap()
|
||||
.to_string(),
|
||||
"Int64(10) > a OR a IS NULL"
|
||||
"a < Int64(10) OR a IS NULL"
|
||||
);
|
||||
|
||||
// Test GtEq with column on LHS
|
||||
|
||||
@@ -34,7 +34,7 @@ use datafusion::physical_plan::{
|
||||
RecordBatchStream, SendableRecordBatchStream,
|
||||
};
|
||||
use datafusion_common::DFSchema;
|
||||
use datafusion_expr::{EmptyRelation, col};
|
||||
use datafusion_expr::EmptyRelation;
|
||||
use datatypes::arrow;
|
||||
use datatypes::arrow::array::{ArrayRef, Float64Array, TimestampMillisecondArray};
|
||||
use datatypes::arrow::datatypes::{DataType, Field, SchemaRef, TimeUnit};
|
||||
@@ -107,21 +107,7 @@ impl UserDefinedLogicalNodeCore for Absent {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
if self.unfix.is_some() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
vec![col(&self.time_index_column)]
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, _output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
if self.unfix.is_some() {
|
||||
return None;
|
||||
}
|
||||
|
||||
let input_schema = self.input.schema();
|
||||
let time_index_idx = input_schema.index_of_column_by_name(None, &self.time_index_column)?;
|
||||
Some(vec![vec![time_index_idx]])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
|
||||
@@ -40,7 +40,6 @@ use datafusion::physical_plan::{
|
||||
Partitioning, PhysicalExpr, PlanProperties, RecordBatchStream, SendableRecordBatchStream,
|
||||
};
|
||||
use datafusion::prelude::{Column, Expr};
|
||||
use datafusion_expr::col;
|
||||
use datatypes::prelude::{ConcreteDataType, DataType as GtDataType};
|
||||
use datatypes::value::{OrderedF64, Value, ValueRef};
|
||||
use datatypes::vectors::{Helper, MutableVector, VectorRef};
|
||||
@@ -89,45 +88,7 @@ impl UserDefinedLogicalNodeCore for HistogramFold {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
let mut exprs = vec![
|
||||
col(&self.le_column),
|
||||
col(&self.ts_column),
|
||||
col(&self.field_column),
|
||||
];
|
||||
exprs.extend(self.input.schema().fields().iter().filter_map(|f| {
|
||||
let name = f.name();
|
||||
if name != &self.le_column && name != &self.ts_column && name != &self.field_column {
|
||||
Some(col(name))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}));
|
||||
exprs
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
let input_schema = self.input.schema();
|
||||
let le_column_index = input_schema.index_of_column_by_name(None, &self.le_column)?;
|
||||
|
||||
if output_columns.is_empty() {
|
||||
let indices = (0..input_schema.fields().len()).collect::<Vec<_>>();
|
||||
return Some(vec![indices]);
|
||||
}
|
||||
|
||||
let mut necessary_indices = output_columns
|
||||
.iter()
|
||||
.map(|&output_column| {
|
||||
if output_column < le_column_index {
|
||||
output_column
|
||||
} else {
|
||||
output_column + 1
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
necessary_indices.push(le_column_index);
|
||||
necessary_indices.sort_unstable();
|
||||
necessary_indices.dedup();
|
||||
Some(vec![necessary_indices])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
@@ -1037,26 +998,11 @@ mod test {
|
||||
use datafusion::common::ToDFSchema;
|
||||
use datafusion::datasource::memory::MemorySourceConfig;
|
||||
use datafusion::datasource::source::DataSourceExec;
|
||||
use datafusion::logical_expr::EmptyRelation;
|
||||
use datafusion::prelude::SessionContext;
|
||||
use datatypes::arrow_array::StringArray;
|
||||
use futures::FutureExt;
|
||||
|
||||
use super::*;
|
||||
|
||||
fn project_batch(batch: &RecordBatch, indices: &[usize]) -> RecordBatch {
|
||||
let fields = indices
|
||||
.iter()
|
||||
.map(|&idx| batch.schema().field(idx).clone())
|
||||
.collect::<Vec<_>>();
|
||||
let columns = indices
|
||||
.iter()
|
||||
.map(|&idx| batch.column(idx).clone())
|
||||
.collect::<Vec<_>>();
|
||||
let schema = Arc::new(Schema::new(fields));
|
||||
RecordBatch::try_new(schema, columns).unwrap()
|
||||
}
|
||||
|
||||
fn prepare_test_data() -> DataSourceExec {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("host", DataType::Utf8, true),
|
||||
@@ -1244,100 +1190,6 @@ mod test {
|
||||
assert_eq!(result_literal, expected);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn pruning_should_keep_le_column_for_exec() {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("ts", DataType::Timestamp(TimeUnit::Millisecond, None), true),
|
||||
Field::new("le", DataType::Utf8, true),
|
||||
Field::new("val", DataType::Float64, true),
|
||||
]));
|
||||
let df_schema = schema.clone().to_dfschema_ref().unwrap();
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema,
|
||||
});
|
||||
let plan = HistogramFold::new(
|
||||
"le".to_string(),
|
||||
"val".to_string(),
|
||||
"ts".to_string(),
|
||||
0.5,
|
||||
input,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let output_columns = [0usize, 1usize];
|
||||
let required = plan.necessary_children_exprs(&output_columns).unwrap();
|
||||
let required = &required[0];
|
||||
assert_eq!(required.as_slice(), &[0, 1, 2]);
|
||||
|
||||
let input_batch = RecordBatch::try_new(
|
||||
schema,
|
||||
vec![
|
||||
Arc::new(TimestampMillisecondArray::from(vec![0, 0])),
|
||||
Arc::new(StringArray::from(vec!["0.1", "+Inf"])),
|
||||
Arc::new(Float64Array::from(vec![1.0, 2.0])),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
let projected = project_batch(&input_batch, required);
|
||||
let projected_schema = projected.schema();
|
||||
let memory_exec = Arc::new(DataSourceExec::new(Arc::new(
|
||||
MemorySourceConfig::try_new(&[vec![projected]], projected_schema, None).unwrap(),
|
||||
)));
|
||||
|
||||
let fold_exec = plan.to_execution_plan(memory_exec);
|
||||
let session_context = SessionContext::default();
|
||||
let output_batches =
|
||||
datafusion::physical_plan::collect(fold_exec, session_context.task_ctx())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(output_batches.len(), 1);
|
||||
|
||||
let output_batch = &output_batches[0];
|
||||
assert_eq!(output_batch.num_rows(), 1);
|
||||
|
||||
let ts = output_batch
|
||||
.column(0)
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampMillisecondArray>()
|
||||
.unwrap();
|
||||
assert_eq!(ts.values(), &[0i64]);
|
||||
|
||||
let values = output_batch
|
||||
.column(1)
|
||||
.as_any()
|
||||
.downcast_ref::<Float64Array>()
|
||||
.unwrap();
|
||||
assert!((values.value(0) - 0.1).abs() < 1e-12);
|
||||
|
||||
// Simulate the pre-fix pruning behavior: omit the `le` column from the child input.
|
||||
let le_index = 1usize;
|
||||
let broken_required = output_columns
|
||||
.iter()
|
||||
.map(|&output_column| {
|
||||
if output_column < le_index {
|
||||
output_column
|
||||
} else {
|
||||
output_column + 1
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let broken = project_batch(&input_batch, &broken_required);
|
||||
let broken_schema = broken.schema();
|
||||
let broken_exec = Arc::new(DataSourceExec::new(Arc::new(
|
||||
MemorySourceConfig::try_new(&[vec![broken]], broken_schema, None).unwrap(),
|
||||
)));
|
||||
let broken_fold_exec = plan.to_execution_plan(broken_exec);
|
||||
let session_context = SessionContext::default();
|
||||
let broken_result = std::panic::AssertUnwindSafe(async {
|
||||
datafusion::physical_plan::collect(broken_fold_exec, session_context.task_ctx()).await
|
||||
})
|
||||
.catch_unwind()
|
||||
.await;
|
||||
assert!(broken_result.is_err());
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn confirm_schema() {
|
||||
let input_schema = Schema::new(vec![
|
||||
|
||||
@@ -33,8 +33,8 @@ use datafusion::physical_plan::{
|
||||
DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanProperties, RecordBatchStream,
|
||||
SendableRecordBatchStream, Statistics,
|
||||
};
|
||||
use datafusion_expr::col;
|
||||
use datatypes::arrow::compute;
|
||||
use datatypes::arrow::error::Result as ArrowResult;
|
||||
use futures::{Stream, StreamExt, ready};
|
||||
use greptime_proto::substrait_extension as pb;
|
||||
use prost::Message;
|
||||
@@ -84,37 +84,7 @@ impl UserDefinedLogicalNodeCore for InstantManipulate {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
if self.unfix.is_some() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
let mut exprs = vec![col(&self.time_index_column)];
|
||||
if let Some(field) = &self.field_column {
|
||||
exprs.push(col(field));
|
||||
}
|
||||
exprs
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
if self.unfix.is_some() {
|
||||
return None;
|
||||
}
|
||||
|
||||
let input_schema = self.input.schema();
|
||||
if output_columns.is_empty() {
|
||||
let indices = (0..input_schema.fields().len()).collect::<Vec<_>>();
|
||||
return Some(vec![indices]);
|
||||
}
|
||||
|
||||
let mut required = output_columns.to_vec();
|
||||
required.push(input_schema.index_of_column_by_name(None, &self.time_index_column)?);
|
||||
if let Some(field) = &self.field_column {
|
||||
required.push(input_schema.index_of_column_by_name(None, field)?);
|
||||
}
|
||||
|
||||
required.sort_unstable();
|
||||
required.dedup();
|
||||
Some(vec![required])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
@@ -470,6 +440,8 @@ impl InstantManipulateStream {
|
||||
// refer to Go version: https://github.com/prometheus/prometheus/blob/e934d0f01158a1d55fa0ebb035346b195fcc1260/promql/engine.go#L1571
|
||||
// and the function `vectorSelectorSingle`
|
||||
pub fn manipulate(&self, input: RecordBatch) -> DataFusionResult<RecordBatch> {
|
||||
let mut take_indices = vec![];
|
||||
|
||||
let ts_column = input
|
||||
.column(self.time_index)
|
||||
.as_any()
|
||||
@@ -501,14 +473,6 @@ impl InstantManipulateStream {
|
||||
let aligned_start = self.start + (max_start - self.start) / self.interval * self.interval;
|
||||
let aligned_end = self.end - (self.end - min_end) / self.interval * self.interval;
|
||||
|
||||
if let Some(batch) =
|
||||
self.try_aligned_slice(&input, ts_column, field_column, aligned_start, aligned_end)?
|
||||
{
|
||||
return Ok(batch);
|
||||
}
|
||||
|
||||
let mut take_indices = vec![];
|
||||
|
||||
let mut cursor = 0;
|
||||
|
||||
let aligned_ts_iter = (aligned_start..=aligned_end).step_by(self.interval as usize);
|
||||
@@ -577,104 +541,10 @@ impl InstantManipulateStream {
|
||||
}
|
||||
}
|
||||
|
||||
// If the selected points are a contiguous slice, avoid `take` on all columns:
|
||||
// slice all columns and only replace the time index column.
|
||||
if let Some(start) = take_indices.first().copied() {
|
||||
let start = start as usize;
|
||||
let len = take_indices.len();
|
||||
let contiguous = take_indices
|
||||
.iter()
|
||||
.enumerate()
|
||||
.all(|(i, idx)| *idx == take_indices[0] + i as u64);
|
||||
if contiguous && start + len <= input.num_rows() && len == aligned_ts.len() {
|
||||
let sliced = input.slice(start, len);
|
||||
let mut arrays = sliced.columns().to_vec();
|
||||
arrays[self.time_index] = Arc::new(TimestampMillisecondArray::from(aligned_ts));
|
||||
return RecordBatch::try_new(sliced.schema(), arrays)
|
||||
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None));
|
||||
}
|
||||
}
|
||||
|
||||
// take record batch and replace the time index column
|
||||
self.take_record_batch_optional(input, take_indices, aligned_ts)
|
||||
}
|
||||
|
||||
fn try_aligned_slice(
|
||||
&self,
|
||||
record_batch: &RecordBatch,
|
||||
ts_column: &TimestampMillisecondArray,
|
||||
field_column: Option<&Float64Array>,
|
||||
aligned_start: Millisecond,
|
||||
aligned_end: Millisecond,
|
||||
) -> DataFusionResult<Option<RecordBatch>> {
|
||||
if aligned_start > aligned_end {
|
||||
return Ok(Some(record_batch.slice(0, 0)));
|
||||
}
|
||||
|
||||
if self.interval <= 0 {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let range = aligned_end - aligned_start;
|
||||
if range < 0 || range % self.interval != 0 {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let expected_len = (range / self.interval) + 1;
|
||||
let expected_len: usize = match expected_len.try_into() {
|
||||
Ok(v) => v,
|
||||
Err(_) => return Ok(None),
|
||||
};
|
||||
if expected_len == 0 {
|
||||
return Ok(Some(record_batch.slice(0, 0)));
|
||||
}
|
||||
|
||||
let first_ts = ts_column.value(0);
|
||||
if aligned_start < first_ts {
|
||||
// Needs lookback or timestamp replacement.
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let start_delta = aligned_start - first_ts;
|
||||
if start_delta % self.interval != 0 {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
let start_idx: usize = match (start_delta / self.interval).try_into() {
|
||||
Ok(v) => v,
|
||||
Err(_) => return Ok(None),
|
||||
};
|
||||
if start_idx >= ts_column.len() {
|
||||
return Ok(None);
|
||||
}
|
||||
if start_idx + expected_len > ts_column.len() {
|
||||
return Ok(None);
|
||||
}
|
||||
let end_idx = start_idx + expected_len - 1;
|
||||
|
||||
if ts_column.value(start_idx) != aligned_start || ts_column.value(end_idx) != aligned_end {
|
||||
return Ok(None);
|
||||
}
|
||||
|
||||
// Ensure no gaps within the slice.
|
||||
for (i, expected_ts) in (aligned_start..=aligned_end)
|
||||
.step_by(self.interval as usize)
|
||||
.enumerate()
|
||||
{
|
||||
if ts_column.value(start_idx + i) != expected_ts {
|
||||
return Ok(None);
|
||||
}
|
||||
if let Some(field_column) = &field_column
|
||||
&& field_column.value(start_idx + i).is_nan()
|
||||
{
|
||||
// Needs filtering out NaNs, cannot slice.
|
||||
return Ok(None);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(Some(record_batch.slice(start_idx, expected_len)))
|
||||
}
|
||||
|
||||
/// Helper function to apply "take" on record batch.
|
||||
fn take_record_batch_optional(
|
||||
&self,
|
||||
@@ -685,13 +555,11 @@ impl InstantManipulateStream {
|
||||
assert_eq!(take_indices.len(), aligned_ts.len());
|
||||
|
||||
let indices_array = UInt64Array::from(take_indices);
|
||||
let mut arrays = record_batch.columns().to_vec();
|
||||
for (idx, array) in arrays.iter_mut().enumerate() {
|
||||
if idx == self.time_index {
|
||||
continue;
|
||||
}
|
||||
*array = compute::take(array.as_ref(), &indices_array, None)?;
|
||||
}
|
||||
let mut arrays = record_batch
|
||||
.columns()
|
||||
.iter()
|
||||
.map(|array| compute::take(array, &indices_array, None))
|
||||
.collect::<ArrowResult<Vec<_>>>()?;
|
||||
arrays[self.time_index] = Arc::new(TimestampMillisecondArray::from(aligned_ts));
|
||||
|
||||
let result = RecordBatch::try_new(record_batch.schema(), arrays)
|
||||
@@ -702,8 +570,6 @@ impl InstantManipulateStream {
|
||||
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use datafusion::common::ToDFSchema;
|
||||
use datafusion::logical_expr::{EmptyRelation, LogicalPlan};
|
||||
use datafusion::prelude::SessionContext;
|
||||
|
||||
use super::*;
|
||||
@@ -745,30 +611,6 @@ mod test {
|
||||
assert_eq!(result_literal, expected);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn pruning_should_keep_time_and_field_columns_for_exec() {
|
||||
let df_schema = prepare_test_data().schema().to_dfschema_ref().unwrap();
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema,
|
||||
});
|
||||
let plan = InstantManipulate::new(
|
||||
0,
|
||||
0,
|
||||
0,
|
||||
0,
|
||||
TIME_INDEX_COLUMN.to_string(),
|
||||
Some("value".to_string()),
|
||||
input,
|
||||
);
|
||||
|
||||
// Simulate a parent projection requesting only the `path` column.
|
||||
let output_columns = [2usize];
|
||||
let required = plan.necessary_children_exprs(&output_columns).unwrap();
|
||||
let required = &required[0];
|
||||
assert_eq!(required.as_slice(), &[0, 1, 2]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn lookback_10s_interval_30s() {
|
||||
let expected = String::from(
|
||||
|
||||
@@ -31,7 +31,6 @@ use datafusion::physical_plan::{
|
||||
DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanProperties, RecordBatchStream,
|
||||
SendableRecordBatchStream,
|
||||
};
|
||||
use datafusion_expr::col;
|
||||
use datatypes::arrow::array::TimestampMillisecondArray;
|
||||
use datatypes::arrow::datatypes::SchemaRef;
|
||||
use datatypes::arrow::record_batch::RecordBatch;
|
||||
@@ -84,38 +83,7 @@ impl UserDefinedLogicalNodeCore for SeriesNormalize {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<datafusion::logical_expr::Expr> {
|
||||
if self.unfix.is_some() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
self.tag_columns
|
||||
.iter()
|
||||
.map(col)
|
||||
.chain(std::iter::once(col(&self.time_index_column_name)))
|
||||
.collect()
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
if self.unfix.is_some() {
|
||||
return None;
|
||||
}
|
||||
|
||||
let input_schema = self.input.schema();
|
||||
if output_columns.is_empty() {
|
||||
let indices = (0..input_schema.fields().len()).collect::<Vec<_>>();
|
||||
return Some(vec![indices]);
|
||||
}
|
||||
|
||||
let mut required = Vec::with_capacity(output_columns.len() + 1 + self.tag_columns.len());
|
||||
required.extend_from_slice(output_columns);
|
||||
required.push(input_schema.index_of_column_by_name(None, &self.time_index_column_name)?);
|
||||
for tag in &self.tag_columns {
|
||||
required.push(input_schema.index_of_column_by_name(None, tag)?);
|
||||
}
|
||||
|
||||
required.sort_unstable();
|
||||
required.dedup();
|
||||
Some(vec![required])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
@@ -461,10 +429,8 @@ mod test {
|
||||
use datafusion::arrow::datatypes::{
|
||||
ArrowPrimitiveType, DataType, Field, Schema, TimestampMillisecondType,
|
||||
};
|
||||
use datafusion::common::ToDFSchema;
|
||||
use datafusion::datasource::memory::MemorySourceConfig;
|
||||
use datafusion::datasource::source::DataSourceExec;
|
||||
use datafusion::logical_expr::{EmptyRelation, LogicalPlan};
|
||||
use datafusion::prelude::SessionContext;
|
||||
use datatypes::arrow::array::TimestampMillisecondArray;
|
||||
use datatypes::arrow_array::StringArray;
|
||||
@@ -495,23 +461,6 @@ mod test {
|
||||
))
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn pruning_should_keep_time_and_tag_columns_for_exec() {
|
||||
let df_schema = prepare_test_data().schema().to_dfschema_ref().unwrap();
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema,
|
||||
});
|
||||
let plan =
|
||||
SeriesNormalize::new(0, TIME_INDEX_COLUMN, true, vec!["path".to_string()], input);
|
||||
|
||||
// Simulate a parent projection requesting only the `value` column.
|
||||
let output_columns = [1usize];
|
||||
let required = plan.necessary_children_exprs(&output_columns).unwrap();
|
||||
let required = &required[0];
|
||||
assert_eq!(required.as_slice(), &[0, 1, 2]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_sort_record_batch() {
|
||||
let memory_exec = Arc::new(prepare_test_data());
|
||||
|
||||
@@ -18,7 +18,7 @@ use std::pin::Pin;
|
||||
use std::sync::Arc;
|
||||
use std::task::{Context, Poll};
|
||||
|
||||
use common_telemetry::{debug, warn};
|
||||
use common_telemetry::debug;
|
||||
use datafusion::arrow::array::{Array, ArrayRef, Int64Array, TimestampMillisecondArray};
|
||||
use datafusion::arrow::compute;
|
||||
use datafusion::arrow::datatypes::{Field, SchemaRef};
|
||||
@@ -38,7 +38,6 @@ use datafusion::physical_plan::{
|
||||
SendableRecordBatchStream, Statistics,
|
||||
};
|
||||
use datafusion::sql::TableReference;
|
||||
use datafusion_expr::col;
|
||||
use futures::{Stream, StreamExt, ready};
|
||||
use greptime_proto::substrait_extension as pb;
|
||||
use prost::Message;
|
||||
@@ -289,53 +288,7 @@ impl UserDefinedLogicalNodeCore for RangeManipulate {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
if self.unfix.is_some() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
let mut exprs = Vec::with_capacity(1 + self.field_columns.len());
|
||||
exprs.push(col(&self.time_index));
|
||||
exprs.extend(self.field_columns.iter().map(col));
|
||||
exprs
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
if self.unfix.is_some() {
|
||||
return None;
|
||||
}
|
||||
|
||||
let input_schema = self.input.schema();
|
||||
let input_len = input_schema.fields().len();
|
||||
let time_index_idx = input_schema.index_of_column_by_name(None, &self.time_index)?;
|
||||
|
||||
if output_columns.is_empty() {
|
||||
let indices = (0..input_len).collect::<Vec<_>>();
|
||||
return Some(vec![indices]);
|
||||
}
|
||||
|
||||
let mut required = Vec::with_capacity(output_columns.len() + 1 + self.field_columns.len());
|
||||
required.push(time_index_idx);
|
||||
for value_column in &self.field_columns {
|
||||
required.push(input_schema.index_of_column_by_name(None, value_column)?);
|
||||
}
|
||||
for &idx in output_columns {
|
||||
if idx < input_len {
|
||||
required.push(idx);
|
||||
} else if idx == input_len {
|
||||
// Derived timestamp range column.
|
||||
required.push(time_index_idx);
|
||||
} else {
|
||||
warn!(
|
||||
"Output column index {} is out of bounds for input schema with length {}",
|
||||
idx, input_len
|
||||
);
|
||||
return None;
|
||||
}
|
||||
}
|
||||
|
||||
required.sort_unstable();
|
||||
required.dedup();
|
||||
Some(vec![required])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
@@ -781,31 +734,16 @@ mod test {
|
||||
use datafusion::common::ToDFSchema;
|
||||
use datafusion::datasource::memory::MemorySourceConfig;
|
||||
use datafusion::datasource::source::DataSourceExec;
|
||||
use datafusion::logical_expr::{EmptyRelation, LogicalPlan};
|
||||
use datafusion::physical_expr::Partitioning;
|
||||
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
|
||||
use datafusion::physical_plan::memory::MemoryStream;
|
||||
use datafusion::prelude::SessionContext;
|
||||
use datatypes::arrow::array::TimestampMillisecondArray;
|
||||
use futures::FutureExt;
|
||||
|
||||
use super::*;
|
||||
|
||||
const TIME_INDEX_COLUMN: &str = "timestamp";
|
||||
|
||||
fn project_batch(batch: &RecordBatch, indices: &[usize]) -> RecordBatch {
|
||||
let fields = indices
|
||||
.iter()
|
||||
.map(|&idx| batch.schema().field(idx).clone())
|
||||
.collect::<Vec<_>>();
|
||||
let columns = indices
|
||||
.iter()
|
||||
.map(|&idx| batch.column(idx).clone())
|
||||
.collect::<Vec<_>>();
|
||||
let schema = Arc::new(Schema::new(fields));
|
||||
RecordBatch::try_new(schema, columns).unwrap()
|
||||
}
|
||||
|
||||
fn prepare_test_data() -> DataSourceExec {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
|
||||
@@ -906,92 +844,6 @@ mod test {
|
||||
assert_eq!(result_literal, expected);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn pruning_should_keep_time_and_value_columns_for_exec() {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
|
||||
Field::new("value_1", DataType::Float64, true),
|
||||
Field::new("value_2", DataType::Float64, true),
|
||||
Field::new("path", DataType::Utf8, true),
|
||||
]));
|
||||
let df_schema = schema.clone().to_dfschema_ref().unwrap();
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema,
|
||||
});
|
||||
let plan = RangeManipulate::new(
|
||||
0,
|
||||
310_000,
|
||||
30_000,
|
||||
90_000,
|
||||
TIME_INDEX_COLUMN.to_string(),
|
||||
vec!["value_1".to_string(), "value_2".to_string()],
|
||||
input,
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// Simulate a parent projection requesting only the `path` column.
|
||||
let output_columns = [3usize];
|
||||
let required = plan.necessary_children_exprs(&output_columns).unwrap();
|
||||
let required = &required[0];
|
||||
assert_eq!(required.as_slice(), &[0, 1, 2, 3]);
|
||||
|
||||
let timestamp_column = Arc::new(TimestampMillisecondArray::from(vec![
|
||||
0, 30_000, 60_000, 90_000, 120_000, // every 30s
|
||||
180_000, 240_000, // every 60s
|
||||
241_000, 271_000, 291_000, // others
|
||||
])) as _;
|
||||
let field_column: ArrayRef = Arc::new(Float64Array::from(vec![1.0; 10])) as _;
|
||||
let path_column = Arc::new(StringArray::from(vec!["foo"; 10])) as _;
|
||||
let input_batch = RecordBatch::try_new(
|
||||
schema,
|
||||
vec![
|
||||
timestamp_column,
|
||||
field_column.clone(),
|
||||
field_column,
|
||||
path_column,
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let projected = project_batch(&input_batch, required);
|
||||
let projected_schema = projected.schema();
|
||||
let memory_exec = Arc::new(DataSourceExec::new(Arc::new(
|
||||
MemorySourceConfig::try_new(&[vec![projected]], projected_schema, None).unwrap(),
|
||||
)));
|
||||
let range_exec = plan.to_execution_plan(memory_exec);
|
||||
let session_context = SessionContext::default();
|
||||
let output_batches =
|
||||
datafusion::physical_plan::collect(range_exec, session_context.task_ctx())
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(output_batches.len(), 1);
|
||||
|
||||
let output_batch = &output_batches[0];
|
||||
let path = output_batch
|
||||
.column(3)
|
||||
.as_any()
|
||||
.downcast_ref::<StringArray>()
|
||||
.unwrap();
|
||||
assert!(path.iter().all(|v| v == Some("foo")));
|
||||
|
||||
// Simulate the pre-fix pruning behavior: omit the timestamp/value columns from the child.
|
||||
let broken_required = [3usize];
|
||||
let broken = project_batch(&input_batch, &broken_required);
|
||||
let broken_schema = broken.schema();
|
||||
let broken_exec = Arc::new(DataSourceExec::new(Arc::new(
|
||||
MemorySourceConfig::try_new(&[vec![broken]], broken_schema, None).unwrap(),
|
||||
)));
|
||||
let broken_range_exec = plan.to_execution_plan(broken_exec);
|
||||
let session_context = SessionContext::default();
|
||||
let broken_result = std::panic::AssertUnwindSafe(async {
|
||||
datafusion::physical_plan::collect(broken_range_exec, session_context.task_ctx()).await
|
||||
})
|
||||
.catch_unwind()
|
||||
.await;
|
||||
assert!(broken_result.is_err());
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn interval_30s_range_90s() {
|
||||
let expected = String::from(
|
||||
|
||||
@@ -31,7 +31,6 @@ use datafusion::physical_plan::{
|
||||
};
|
||||
use datafusion::prelude::Expr;
|
||||
use datafusion::sql::TableReference;
|
||||
use datafusion_expr::col;
|
||||
use datatypes::arrow::array::{Array, Float64Array, StringArray, TimestampMillisecondArray};
|
||||
use datatypes::arrow::compute::{CastOptions, cast_with_options, concat_batches};
|
||||
use datatypes::arrow::datatypes::{DataType, Field, Schema, SchemaRef, TimeUnit};
|
||||
@@ -267,36 +266,7 @@ impl UserDefinedLogicalNodeCore for ScalarCalculate {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
if self.unfix.is_some() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
self.tag_columns
|
||||
.iter()
|
||||
.map(col)
|
||||
.chain(std::iter::once(col(&self.time_index)))
|
||||
.chain(std::iter::once(col(&self.field_column)))
|
||||
.collect()
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, _output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
if self.unfix.is_some() {
|
||||
return None;
|
||||
}
|
||||
|
||||
let input_schema = self.input.schema();
|
||||
let time_index_idx = input_schema.index_of_column_by_name(None, &self.time_index)?;
|
||||
let field_column_idx = input_schema.index_of_column_by_name(None, &self.field_column)?;
|
||||
|
||||
let mut required = Vec::with_capacity(2 + self.tag_columns.len());
|
||||
required.extend([time_index_idx, field_column_idx]);
|
||||
for tag in &self.tag_columns {
|
||||
required.push(input_schema.index_of_column_by_name(None, tag)?);
|
||||
}
|
||||
|
||||
required.sort_unstable();
|
||||
required.dedup();
|
||||
Some(vec![required])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
@@ -305,9 +275,15 @@ impl UserDefinedLogicalNodeCore for ScalarCalculate {
|
||||
|
||||
fn with_exprs_and_inputs(
|
||||
&self,
|
||||
_exprs: Vec<Expr>,
|
||||
exprs: Vec<Expr>,
|
||||
inputs: Vec<LogicalPlan>,
|
||||
) -> DataFusionResult<Self> {
|
||||
if !exprs.is_empty() {
|
||||
return Err(DataFusionError::Internal(
|
||||
"ScalarCalculate should not have any expressions".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
let input: LogicalPlan = inputs.into_iter().next().unwrap();
|
||||
let input_schema = input.schema();
|
||||
|
||||
@@ -648,109 +624,6 @@ mod test {
|
||||
|
||||
use super::*;
|
||||
|
||||
fn project_batch(batch: &RecordBatch, indices: &[usize]) -> RecordBatch {
|
||||
let fields = indices
|
||||
.iter()
|
||||
.map(|&idx| batch.schema().field(idx).clone())
|
||||
.collect::<Vec<_>>();
|
||||
let columns = indices
|
||||
.iter()
|
||||
.map(|&idx| batch.column(idx).clone())
|
||||
.collect::<Vec<_>>();
|
||||
let schema = Arc::new(Schema::new(fields));
|
||||
RecordBatch::try_new(schema, columns).unwrap()
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn necessary_children_exprs_preserve_tag_columns() {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("ts", DataType::Timestamp(TimeUnit::Millisecond, None), true),
|
||||
Field::new("tag1", DataType::Utf8, true),
|
||||
Field::new("tag2", DataType::Utf8, true),
|
||||
Field::new("val", DataType::Float64, true),
|
||||
Field::new("extra", DataType::Utf8, true),
|
||||
]));
|
||||
let schema = Arc::new(DFSchema::try_from(schema).unwrap());
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema,
|
||||
});
|
||||
let tag_columns = vec!["tag1".to_string(), "tag2".to_string()];
|
||||
let plan = ScalarCalculate::new(0, 1, 1, input, "ts", &tag_columns, "val", None).unwrap();
|
||||
|
||||
let required = plan.necessary_children_exprs(&[0, 1]).unwrap();
|
||||
assert_eq!(required, vec![vec![0, 1, 2, 3]]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn pruning_should_keep_tag_columns_for_exec() {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("ts", DataType::Timestamp(TimeUnit::Millisecond, None), true),
|
||||
Field::new("tag1", DataType::Utf8, true),
|
||||
Field::new("tag2", DataType::Utf8, true),
|
||||
Field::new("val", DataType::Float64, true),
|
||||
Field::new("extra", DataType::Utf8, true),
|
||||
]));
|
||||
let df_schema = Arc::new(DFSchema::try_from(schema.clone()).unwrap());
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema,
|
||||
});
|
||||
let tag_columns = vec!["tag1".to_string(), "tag2".to_string()];
|
||||
let plan =
|
||||
ScalarCalculate::new(0, 15_000, 5000, input, "ts", &tag_columns, "val", None).unwrap();
|
||||
|
||||
let required = plan.necessary_children_exprs(&[0, 1]).unwrap();
|
||||
let required = &required[0];
|
||||
|
||||
let batch = RecordBatch::try_new(
|
||||
schema,
|
||||
vec![
|
||||
Arc::new(TimestampMillisecondArray::from(vec![
|
||||
0, 5_000, 10_000, 15_000,
|
||||
])),
|
||||
Arc::new(StringArray::from(vec!["foo", "foo", "foo", "foo"])),
|
||||
Arc::new(StringArray::from(vec!["bar", "bar", "bar", "bar"])),
|
||||
Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])),
|
||||
Arc::new(StringArray::from(vec!["x", "x", "x", "x"])),
|
||||
],
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let projected_batch = project_batch(&batch, required);
|
||||
let projected_schema = projected_batch.schema();
|
||||
let memory_exec = Arc::new(DataSourceExec::new(Arc::new(
|
||||
MemorySourceConfig::try_new(&[vec![projected_batch]], projected_schema, None).unwrap(),
|
||||
)));
|
||||
let scalar_exec = plan.to_execution_plan(memory_exec).unwrap();
|
||||
|
||||
let session_context = SessionContext::default();
|
||||
let result = datafusion::physical_plan::collect(scalar_exec, session_context.task_ctx())
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
assert_eq!(result.len(), 1);
|
||||
let batch = &result[0];
|
||||
assert_eq!(batch.num_columns(), 2);
|
||||
assert_eq!(batch.num_rows(), 4);
|
||||
assert_eq!(batch.schema().field(0).name(), "ts");
|
||||
assert_eq!(batch.schema().field(1).name(), "scalar(val)");
|
||||
|
||||
let ts = batch
|
||||
.column(0)
|
||||
.as_any()
|
||||
.downcast_ref::<TimestampMillisecondArray>()
|
||||
.unwrap();
|
||||
assert_eq!(ts.values(), &[0i64, 5_000, 10_000, 15_000]);
|
||||
|
||||
let values = batch
|
||||
.column(1)
|
||||
.as_any()
|
||||
.downcast_ref::<Float64Array>()
|
||||
.unwrap();
|
||||
assert_eq!(values.values(), &[1.0f64, 2.0, 3.0, 4.0]);
|
||||
}
|
||||
|
||||
fn prepare_test_data(series: Vec<RecordBatch>) -> DataSourceExec {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("ts", DataType::Timestamp(TimeUnit::Millisecond, None), true),
|
||||
|
||||
@@ -33,7 +33,6 @@ use datafusion::physical_plan::{
|
||||
DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, PlanProperties, RecordBatchStream,
|
||||
SendableRecordBatchStream,
|
||||
};
|
||||
use datafusion_expr::col;
|
||||
use datatypes::arrow::compute;
|
||||
use datatypes::compute::SortOptions;
|
||||
use futures::{Stream, StreamExt, ready};
|
||||
@@ -77,38 +76,7 @@ impl UserDefinedLogicalNodeCore for SeriesDivide {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
if self.unfix.is_some() {
|
||||
return vec![];
|
||||
}
|
||||
|
||||
self.tag_columns
|
||||
.iter()
|
||||
.map(col)
|
||||
.chain(std::iter::once(col(&self.time_index_column)))
|
||||
.collect()
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
if self.unfix.is_some() {
|
||||
return None;
|
||||
}
|
||||
|
||||
let input_schema = self.input.schema();
|
||||
if output_columns.is_empty() {
|
||||
let indices = (0..input_schema.fields().len()).collect::<Vec<_>>();
|
||||
return Some(vec![indices]);
|
||||
}
|
||||
|
||||
let mut required = Vec::with_capacity(output_columns.len() + 1 + self.tag_columns.len());
|
||||
required.extend_from_slice(output_columns);
|
||||
for tag in &self.tag_columns {
|
||||
required.push(input_schema.index_of_column_by_name(None, tag)?);
|
||||
}
|
||||
required.push(input_schema.index_of_column_by_name(None, &self.time_index_column)?);
|
||||
|
||||
required.sort_unstable();
|
||||
required.dedup();
|
||||
Some(vec![required])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
|
||||
@@ -576,10 +544,8 @@ impl SeriesDivideStream {
|
||||
#[cfg(test)]
|
||||
mod test {
|
||||
use datafusion::arrow::datatypes::{DataType, Field, Schema};
|
||||
use datafusion::common::ToDFSchema;
|
||||
use datafusion::datasource::memory::MemorySourceConfig;
|
||||
use datafusion::datasource::source::DataSourceExec;
|
||||
use datafusion::logical_expr::{EmptyRelation, LogicalPlan};
|
||||
use datafusion::prelude::SessionContext;
|
||||
|
||||
use super::*;
|
||||
@@ -645,26 +611,6 @@ mod test {
|
||||
))
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn pruning_should_keep_tags_and_time_index_columns_for_exec() {
|
||||
let df_schema = prepare_test_data().schema().to_dfschema_ref().unwrap();
|
||||
let input = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema,
|
||||
});
|
||||
let plan = SeriesDivide::new(
|
||||
vec!["host".to_string(), "path".to_string()],
|
||||
"time_index".to_string(),
|
||||
input,
|
||||
);
|
||||
|
||||
// Simulate a parent projection requesting only the `host` column.
|
||||
let output_columns = [0usize];
|
||||
let required = plan.necessary_children_exprs(&output_columns).unwrap();
|
||||
let required = &required[0];
|
||||
assert_eq!(required.as_slice(), &[0, 1, 2]);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn overall_data() {
|
||||
let memory_exec = Arc::new(prepare_test_data());
|
||||
|
||||
@@ -32,7 +32,6 @@ use datafusion::physical_plan::{
|
||||
DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PlanProperties,
|
||||
RecordBatchStream, SendableRecordBatchStream, hash_utils,
|
||||
};
|
||||
use datafusion_expr::col;
|
||||
use datatypes::arrow::compute;
|
||||
use futures::future::BoxFuture;
|
||||
use futures::{Stream, StreamExt, TryStreamExt, ready};
|
||||
@@ -146,20 +145,7 @@ impl UserDefinedLogicalNodeCore for UnionDistinctOn {
|
||||
}
|
||||
|
||||
fn expressions(&self) -> Vec<Expr> {
|
||||
let mut exprs: Vec<Expr> = self.compare_keys.iter().map(col).collect();
|
||||
if !self.compare_keys.iter().any(|key| key == &self.ts_col) {
|
||||
exprs.push(col(&self.ts_col));
|
||||
}
|
||||
exprs
|
||||
}
|
||||
|
||||
fn necessary_children_exprs(&self, _output_columns: &[usize]) -> Option<Vec<Vec<usize>>> {
|
||||
let left_len = self.left.schema().fields().len();
|
||||
let right_len = self.right.schema().fields().len();
|
||||
Some(vec![
|
||||
(0..left_len).collect::<Vec<_>>(),
|
||||
(0..right_len).collect::<Vec<_>>(),
|
||||
])
|
||||
vec![]
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
|
||||
@@ -554,43 +540,9 @@ mod test {
|
||||
|
||||
use datafusion::arrow::array::Int32Array;
|
||||
use datafusion::arrow::datatypes::{DataType, Field, Schema};
|
||||
use datafusion::common::ToDFSchema;
|
||||
use datafusion::logical_expr::{EmptyRelation, LogicalPlan};
|
||||
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn pruning_should_keep_all_columns_for_exec() {
|
||||
let schema = Arc::new(Schema::new(vec![
|
||||
Field::new("ts", DataType::Int32, false),
|
||||
Field::new("k", DataType::Int32, false),
|
||||
Field::new("v", DataType::Int32, false),
|
||||
]));
|
||||
let df_schema = schema.to_dfschema_ref().unwrap();
|
||||
let left = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema.clone(),
|
||||
});
|
||||
let right = LogicalPlan::EmptyRelation(EmptyRelation {
|
||||
produce_one_row: false,
|
||||
schema: df_schema.clone(),
|
||||
});
|
||||
let plan = UnionDistinctOn::new(
|
||||
left,
|
||||
right,
|
||||
vec!["k".to_string()],
|
||||
"ts".to_string(),
|
||||
df_schema,
|
||||
);
|
||||
|
||||
// Simulate a parent projection requesting only one output column.
|
||||
let output_columns = [2usize];
|
||||
let required = plan.necessary_children_exprs(&output_columns).unwrap();
|
||||
assert_eq!(required.len(), 2);
|
||||
assert_eq!(required[0].as_slice(), &[0, 1, 2]);
|
||||
assert_eq!(required[1].as_slice(), &[0, 1, 2]);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_interleave_batches() {
|
||||
let schema = Schema::new(vec![
|
||||
|
||||
@@ -25,6 +25,7 @@ pub type PuffinMetadataCacheRef = Arc<PuffinMetadataCache>;
|
||||
/// A cache for storing the metadata of the index files.
|
||||
pub struct PuffinMetadataCache {
|
||||
cache: moka::sync::Cache<String, Arc<FileMetadata>>,
|
||||
cache_bytes: IntGaugeVec,
|
||||
}
|
||||
|
||||
fn puffin_metadata_weight(k: &str, v: &Arc<FileMetadata>) -> u32 {
|
||||
@@ -38,13 +39,14 @@ impl PuffinMetadataCache {
|
||||
cache: moka::sync::CacheBuilder::new(capacity)
|
||||
.name("puffin_metadata")
|
||||
.weigher(|k: &String, v| puffin_metadata_weight(k, v))
|
||||
.eviction_listener(|k, v, _cause| {
|
||||
.eviction_listener(move |k, v, _cause| {
|
||||
let size = puffin_metadata_weight(&k, &v);
|
||||
cache_bytes
|
||||
.with_label_values(&[PUFFIN_METADATA_TYPE])
|
||||
.sub(size.into());
|
||||
})
|
||||
.build(),
|
||||
cache_bytes: cache_bytes.clone(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -55,6 +57,10 @@ impl PuffinMetadataCache {
|
||||
|
||||
/// Puts the metadata into the cache.
|
||||
pub fn put_metadata(&self, file_id: String, metadata: Arc<FileMetadata>) {
|
||||
let size = puffin_metadata_weight(&file_id, &metadata);
|
||||
self.cache_bytes
|
||||
.with_label_values(&[PUFFIN_METADATA_TYPE])
|
||||
.add(size.into());
|
||||
self.cache.insert(file_id, metadata);
|
||||
}
|
||||
|
||||
|
||||
@@ -37,4 +37,5 @@ serde.workspace = true
|
||||
servers.workspace = true
|
||||
snafu.workspace = true
|
||||
store-api.workspace = true
|
||||
table.workspace = true
|
||||
tokio.workspace = true
|
||||
|
||||
@@ -36,6 +36,12 @@ pub enum Error {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
|
||||
#[snafu(display("Repartition procedure is not supported in standalone mode"))]
|
||||
NoSupportRepartitionProcedure {
|
||||
#[snafu(implicit)]
|
||||
location: Location,
|
||||
},
|
||||
}
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
@@ -45,6 +51,7 @@ impl ErrorExt for Error {
|
||||
match self {
|
||||
Error::OpenMetadataKvBackend { source, .. } => source.status_code(),
|
||||
Error::External { source, .. } => source.status_code(),
|
||||
Error::NoSupportRepartitionProcedure { .. } => StatusCode::Unsupported,
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -20,4 +20,4 @@ pub mod procedure;
|
||||
|
||||
pub use information_extension::StandaloneInformationExtension;
|
||||
pub use metadata::build_metadata_kvbackend;
|
||||
pub use procedure::build_procedure_manager;
|
||||
pub use procedure::{StandaloneRepartitionProcedureFactory, build_procedure_manager};
|
||||
|
||||
@@ -14,12 +14,19 @@
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_error::ext::BoxedError;
|
||||
use common_meta::ddl::DdlContext;
|
||||
use common_meta::ddl_manager::RepartitionProcedureFactory;
|
||||
use common_meta::key::runtime_switch::RuntimeSwitchManager;
|
||||
use common_meta::kv_backend::KvBackendRef;
|
||||
use common_meta::state_store::KvStateStore;
|
||||
use common_procedure::ProcedureManagerRef;
|
||||
use common_procedure::local::{LocalManager, ManagerConfig};
|
||||
use common_procedure::options::ProcedureConfig;
|
||||
use common_procedure::{BoxedProcedure, ProcedureManagerRef};
|
||||
use store_api::storage::TableId;
|
||||
use table::table_name::TableName;
|
||||
|
||||
use crate::error::NoSupportRepartitionProcedureSnafu;
|
||||
|
||||
/// Builds the procedure manager.
|
||||
pub fn build_procedure_manager(
|
||||
@@ -43,3 +50,32 @@ pub fn build_procedure_manager(
|
||||
None,
|
||||
))
|
||||
}
|
||||
|
||||
/// No-op implementation of [`RepartitionProcedureFactory`] for standalone mode.
|
||||
///
|
||||
/// In standalone deployments, repartition operations are not supported, so
|
||||
/// this factory always returns a `NoSupportRepartitionProcedure` error
|
||||
/// from [`RepartitionProcedureFactory::create`] and performs no registration
|
||||
/// work in [`RepartitionProcedureFactory::register_loaders`].
|
||||
pub struct StandaloneRepartitionProcedureFactory;
|
||||
|
||||
impl RepartitionProcedureFactory for StandaloneRepartitionProcedureFactory {
|
||||
fn create(
|
||||
&self,
|
||||
_ddl_ctx: &DdlContext,
|
||||
_table_name: TableName,
|
||||
_table_id: TableId,
|
||||
_from_exprs: Vec<String>,
|
||||
_to_exprs: Vec<String>,
|
||||
) -> std::result::Result<BoxedProcedure, BoxedError> {
|
||||
Err(BoxedError::new(NoSupportRepartitionProcedureSnafu.build()))
|
||||
}
|
||||
|
||||
fn register_loaders(
|
||||
&self,
|
||||
_ddl_ctx: &DdlContext,
|
||||
_procedure_manager: &ProcedureManagerRef,
|
||||
) -> std::result::Result<(), BoxedError> {
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -95,8 +95,15 @@ impl FileRef {
|
||||
#[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize)]
|
||||
pub struct FileRefsManifest {
|
||||
pub file_refs: HashMap<RegionId, HashSet<FileRef>>,
|
||||
/// Manifest version when this manifest is read for it's files
|
||||
/// Manifest version when this manifest is read for its files
|
||||
pub manifest_version: HashMap<RegionId, ManifestVersion>,
|
||||
/// Cross-region file ownership mapping.
|
||||
///
|
||||
/// Key is the source/original region id (before repartition); value is the set of
|
||||
/// target/destination region ids (after repartition) that currently hold files
|
||||
/// originally coming from that source region.
|
||||
///
|
||||
pub cross_region_refs: HashMap<RegionId, HashSet<RegionId>>,
|
||||
}
|
||||
|
||||
#[derive(Clone, Default, Debug, PartialEq, Eq, Serialize, Deserialize)]
|
||||
@@ -179,6 +186,8 @@ mod tests {
|
||||
.insert(r1, [FileRef::new(r1, FileId::random(), None)].into());
|
||||
manifest.manifest_version.insert(r0, 10);
|
||||
manifest.manifest_version.insert(r1, 20);
|
||||
manifest.cross_region_refs.insert(r0, [r1].into());
|
||||
manifest.cross_region_refs.insert(r1, [r0].into());
|
||||
|
||||
let json = serde_json::to_string(&manifest).unwrap();
|
||||
let parsed: FileRefsManifest = serde_json::from_str(&json).unwrap();
|
||||
|
||||
@@ -6,6 +6,7 @@ license.workspace = true
|
||||
|
||||
[features]
|
||||
dashboard = []
|
||||
vector_index = []
|
||||
|
||||
[lints]
|
||||
workspace = true
|
||||
|
||||
@@ -52,6 +52,7 @@ use frontend::server::Services;
|
||||
use meta_srv::metasrv::{FLOW_ID_SEQ, TABLE_ID_SEQ};
|
||||
use servers::grpc::GrpcOptions;
|
||||
use snafu::ResultExt;
|
||||
use standalone::StandaloneRepartitionProcedureFactory;
|
||||
use standalone::options::StandaloneOptions;
|
||||
|
||||
use crate::test_util::{self, StorageType, TestGuard, create_tmp_dir_and_datanode_opts};
|
||||
@@ -215,6 +216,7 @@ impl GreptimeDbStandaloneBuilder {
|
||||
flow_id_sequence,
|
||||
));
|
||||
|
||||
let repartition_procedure_factory = Arc::new(StandaloneRepartitionProcedureFactory);
|
||||
let ddl_manager = Arc::new(
|
||||
DdlManager::try_new(
|
||||
DdlContext {
|
||||
@@ -229,6 +231,7 @@ impl GreptimeDbStandaloneBuilder {
|
||||
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
|
||||
},
|
||||
procedure_manager.clone(),
|
||||
repartition_procedure_factory,
|
||||
register_procedure_loaders,
|
||||
)
|
||||
.unwrap(),
|
||||
|
||||
@@ -12,7 +12,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::collections::{HashMap, HashSet};
|
||||
use std::collections::HashSet;
|
||||
use std::time::Duration;
|
||||
|
||||
use common_meta::key::TableMetadataManagerRef;
|
||||
@@ -105,8 +105,10 @@ async fn distributed_with_gc(store_type: &StorageType) -> (TestContext, TempDirG
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_gc_basic_different_store() {
|
||||
let _ = dotenv::dotenv();
|
||||
common_telemetry::init_default_ut_logging();
|
||||
let store_type = StorageType::build_storage_types_based_on_env();
|
||||
info!("store type: {:?}", store_type);
|
||||
for store in store_type {
|
||||
if store == StorageType::File {
|
||||
continue; // no point in test gc in fs storage
|
||||
@@ -190,17 +192,16 @@ async fn test_gc_basic(store_type: &StorageType) {
|
||||
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) =
|
||||
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.table_metadata_manager().clone(),
|
||||
metasrv.options().grpc.server_addr.clone(),
|
||||
regions.clone(),
|
||||
false, // full_file_listing
|
||||
region_routes,
|
||||
HashMap::new(), // related_regions (empty for this simple test)
|
||||
false, // full_file_listing
|
||||
Duration::from_secs(10), // timeout
|
||||
);
|
||||
|
||||
|
||||
@@ -1379,6 +1379,19 @@ providers = []"#,
|
||||
)
|
||||
};
|
||||
|
||||
let vector_index_config = if cfg!(feature = "vector_index") {
|
||||
r#"
|
||||
[region_engine.mito.vector_index]
|
||||
create_on_flush = "auto"
|
||||
create_on_compaction = "auto"
|
||||
apply_on_query = "auto"
|
||||
mem_threshold_on_create = "auto"
|
||||
|
||||
"#
|
||||
} else {
|
||||
"\n"
|
||||
};
|
||||
|
||||
let expected_toml_str = format!(
|
||||
r#"
|
||||
enable_telemetry = true
|
||||
@@ -1545,14 +1558,7 @@ create_on_flush = "auto"
|
||||
create_on_compaction = "auto"
|
||||
apply_on_query = "auto"
|
||||
mem_threshold_on_create = "auto"
|
||||
|
||||
[region_engine.mito.vector_index]
|
||||
create_on_flush = "auto"
|
||||
create_on_compaction = "auto"
|
||||
apply_on_query = "auto"
|
||||
mem_threshold_on_create = "auto"
|
||||
|
||||
[region_engine.mito.memtable]
|
||||
{vector_index_config}[region_engine.mito.memtable]
|
||||
type = "time_series"
|
||||
|
||||
[region_engine.mito.gc]
|
||||
|
||||
@@ -544,7 +544,8 @@ tql explain (1752591864, 1752592164, '30s') sum by (a, b, c) (rate(aggr_optimize
|
||||
| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] |
|
||||
| | PromSeriesDivide: tags=["a", "b", "c", "d"] |
|
||||
| | Sort: aggr_optimize_not_count.a ASC NULLS FIRST, aggr_optimize_not_count.b ASC NULLS FIRST, aggr_optimize_not_count.c ASC NULLS FIRST, aggr_optimize_not_count.d ASC NULLS FIRST, aggr_optimize_not_count.greptime_timestamp ASC NULLS FIRST |
|
||||
| | MergeScan [is_placeholder=false, remote_input=[ |
|
||||
| | Projection: aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.d, aggr_optimize_not_count.greptime_timestamp, aggr_optimize_not_count.greptime_value |
|
||||
| | MergeScan [is_placeholder=false, remote_input=[ |
|
||||
| | Filter: aggr_optimize_not_count.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not_count.greptime_timestamp <= TimestampMillisecond(300000, None) |
|
||||
| | TableScan: aggr_optimize_not_count |
|
||||
| | ]] |
|
||||
|
||||
101
tests/cases/distributed/repartition/repartition.result
Normal file
101
tests/cases/distributed/repartition/repartition.result
Normal file
@@ -0,0 +1,101 @@
|
||||
CREATE TABLE alter_repartition_table(
|
||||
device_id INT,
|
||||
area STRING,
|
||||
ty STRING,
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY(device_id)
|
||||
) PARTITION ON COLUMNS (device_id, area) (
|
||||
device_id < 100,
|
||||
device_id >= 100 AND device_id < 200,
|
||||
device_id >= 200
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
ALTER TABLE alter_repartition_table REPARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
device_id < 100 AND area < 'South',
|
||||
device_id < 100 AND area >= 'South'
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
SHOW CREATE TABLE alter_repartition_table;
|
||||
|
||||
+-------------------------+--------------------------------------------------------+
|
||||
| Table | Create Table |
|
||||
+-------------------------+--------------------------------------------------------+
|
||||
| alter_repartition_table | CREATE TABLE IF NOT EXISTS "alter_repartition_table" ( |
|
||||
| | "device_id" INT NULL, |
|
||||
| | "area" STRING NULL, |
|
||||
| | "ty" STRING NULL, |
|
||||
| | "ts" TIMESTAMP(3) NOT NULL, |
|
||||
| | TIME INDEX ("ts"), |
|
||||
| | PRIMARY KEY ("device_id") |
|
||||
| | ) |
|
||||
| | PARTITION ON COLUMNS ("device_id", "area") ( |
|
||||
| | device_id < 100 AND area < 'South', |
|
||||
| | device_id >= 100 AND device_id < 200, |
|
||||
| | device_id >= 200, |
|
||||
| | device_id < 100 AND area >= 'South' |
|
||||
| | ) |
|
||||
| | ENGINE=mito |
|
||||
| | |
|
||||
+-------------------------+--------------------------------------------------------+
|
||||
|
||||
ALTER TABLE alter_repartition_table MERGE PARTITION (
|
||||
device_id < 100 AND area < 'South',
|
||||
device_id < 100 AND area >= 'South'
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
SHOW CREATE TABLE alter_repartition_table;
|
||||
|
||||
+-------------------------+------------------------------------------------------------------------------+
|
||||
| Table | Create Table |
|
||||
+-------------------------+------------------------------------------------------------------------------+
|
||||
| alter_repartition_table | CREATE TABLE IF NOT EXISTS "alter_repartition_table" ( |
|
||||
| | "device_id" INT NULL, |
|
||||
| | "area" STRING NULL, |
|
||||
| | "ty" STRING NULL, |
|
||||
| | "ts" TIMESTAMP(3) NOT NULL, |
|
||||
| | TIME INDEX ("ts"), |
|
||||
| | PRIMARY KEY ("device_id") |
|
||||
| | ) |
|
||||
| | PARTITION ON COLUMNS ("device_id", "area") ( |
|
||||
| | device_id < 100 AND area < 'South' OR device_id < 100 AND area >= 'South', |
|
||||
| | device_id >= 100 AND device_id < 200, |
|
||||
| | device_id >= 200 |
|
||||
| | ) |
|
||||
| | ENGINE=mito |
|
||||
| | |
|
||||
+-------------------------+------------------------------------------------------------------------------+
|
||||
|
||||
-- FIXME(weny): Object store is not configured for the test environment,
|
||||
-- so staging manifest may not be applied in some cases.
|
||||
-- invalid: empty source clause
|
||||
ALTER TABLE alter_repartition_table REPARTITION () INTO (
|
||||
device_id < 100
|
||||
);
|
||||
|
||||
Error: 2000(InvalidSyntax), Invalid SQL syntax: sql parser error: Expected expression inside REPARTITION clause, found: )
|
||||
|
||||
-- invalid: more than one INTO clause
|
||||
ALTER TABLE alter_repartition_table REPARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
device_id < 50
|
||||
), (
|
||||
device_id >= 50
|
||||
) INTO (
|
||||
device_id >= 50
|
||||
);
|
||||
|
||||
Error: 2000(InvalidSyntax), Invalid SQL syntax: sql parser error: Expected end of REPARTITION clause, found: ,
|
||||
|
||||
DROP TABLE alter_repartition_table;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
@@ -10,7 +10,6 @@ CREATE TABLE alter_repartition_table(
|
||||
device_id >= 200
|
||||
);
|
||||
|
||||
-- valid grammar, currently not implemented
|
||||
ALTER TABLE alter_repartition_table REPARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
@@ -18,19 +17,17 @@ ALTER TABLE alter_repartition_table REPARTITION (
|
||||
device_id < 100 AND area >= 'South'
|
||||
);
|
||||
|
||||
-- valid grammar, currently not implemented
|
||||
ALTER TABLE alter_repartition_table SPLIT PARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
SHOW CREATE TABLE alter_repartition_table;
|
||||
|
||||
ALTER TABLE alter_repartition_table MERGE PARTITION (
|
||||
device_id < 100 AND area < 'South',
|
||||
device_id < 100 AND area >= 'South'
|
||||
);
|
||||
|
||||
-- valid grammar, currently not implemented
|
||||
ALTER TABLE alter_repartition_table MERGE PARTITION (
|
||||
device_id < 100,
|
||||
device_id >= 100 AND device_id < 200
|
||||
);
|
||||
SHOW CREATE TABLE alter_repartition_table;
|
||||
|
||||
-- FIXME(weny): Object store is not configured for the test environment,
|
||||
-- so staging manifest may not be applied in some cases.
|
||||
|
||||
-- invalid: empty source clause
|
||||
ALTER TABLE alter_repartition_table REPARTITION () INTO (
|
||||
@@ -90,6 +90,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|
||||
-- SQLNESS REPLACE (Duration.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 10, '5s') test;
|
||||
|
||||
+-+-+-+
|
||||
|
||||
@@ -44,6 +44,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|
||||
-- SQLNESS REPLACE (Duration.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 10, '5s') test;
|
||||
|
||||
DROP TABLE test;
|
||||
|
||||
@@ -1,66 +0,0 @@
|
||||
CREATE TABLE alter_repartition_table(
|
||||
device_id INT,
|
||||
area STRING,
|
||||
ty STRING,
|
||||
ts TIMESTAMP TIME INDEX,
|
||||
PRIMARY KEY(device_id)
|
||||
) PARTITION ON COLUMNS (device_id, area) (
|
||||
device_id < 100,
|
||||
device_id >= 100 AND device_id < 200,
|
||||
device_id >= 200
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
-- valid grammar, currently not implemented
|
||||
ALTER TABLE alter_repartition_table REPARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
device_id < 100 AND area < 'South',
|
||||
device_id < 100 AND area >= 'South'
|
||||
);
|
||||
|
||||
Error: 1001(Unsupported), Not supported: ALTER TABLE REPARTITION
|
||||
|
||||
-- valid grammar, currently not implemented
|
||||
ALTER TABLE alter_repartition_table SPLIT PARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
device_id < 100 AND area < 'South',
|
||||
device_id < 100 AND area >= 'South'
|
||||
);
|
||||
|
||||
Error: 1001(Unsupported), Not supported: ALTER TABLE REPARTITION
|
||||
|
||||
-- valid grammar, currently not implemented
|
||||
ALTER TABLE alter_repartition_table MERGE PARTITION (
|
||||
device_id < 100,
|
||||
device_id >= 100 AND device_id < 200
|
||||
);
|
||||
|
||||
Error: 1001(Unsupported), Not supported: ALTER TABLE REPARTITION
|
||||
|
||||
-- invalid: empty source clause
|
||||
ALTER TABLE alter_repartition_table REPARTITION () INTO (
|
||||
device_id < 100
|
||||
);
|
||||
|
||||
Error: 2000(InvalidSyntax), Invalid SQL syntax: sql parser error: Expected expression inside REPARTITION clause, found: )
|
||||
|
||||
-- invalid: more than one INTO clause
|
||||
ALTER TABLE alter_repartition_table REPARTITION (
|
||||
device_id < 100
|
||||
) INTO (
|
||||
device_id < 50
|
||||
), (
|
||||
device_id >= 50
|
||||
) INTO (
|
||||
device_id >= 50
|
||||
);
|
||||
|
||||
Error: 2000(InvalidSyntax), Invalid SQL syntax: sql parser error: Expected end of REPARTITION clause, found: ,
|
||||
|
||||
DROP TABLE alter_repartition_table;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
@@ -63,6 +63,7 @@ TQL EVAL (0, 100, '15s') test{host=~"(10\\.0\\.160\\.237:8080|10\\.0\\.160\\.237
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".*"};
|
||||
|
||||
+-+-+-+
|
||||
@@ -86,6 +87,7 @@ TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".*"};
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".+"};
|
||||
|
||||
+-+-+-+
|
||||
@@ -109,6 +111,7 @@ TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".+"};
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host!~".*"};
|
||||
|
||||
+-+-+-+
|
||||
@@ -134,6 +137,7 @@ TQL ANALYZE VERBOSE (0, 0, '1s') test{host!~".*"};
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host!~".+"};
|
||||
|
||||
+-+-+-+
|
||||
|
||||
@@ -26,6 +26,7 @@ TQL EVAL (0, 100, '15s') test{host=~"(10\\.0\\.160\\.237:8080|10\\.0\\.160\\.237
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".*"};
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
@@ -35,6 +36,7 @@ TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".*"};
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".+"};
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
@@ -44,6 +46,7 @@ TQL ANALYZE VERBOSE (0, 0, '1s') test{host=~".+"};
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host!~".*"};
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
@@ -53,6 +56,7 @@ TQL ANALYZE VERBOSE (0, 0, '1s') test{host!~".*"};
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 0, '1s') test{host!~".+"};
|
||||
|
||||
DROP TABLE test;
|
||||
|
||||
@@ -24,46 +24,40 @@ VALUES
|
||||
Affected Rows: 10
|
||||
|
||||
-- should not fail with mismatch timezone
|
||||
-- SQLNESS REPLACE \d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d+Z NOW
|
||||
SELECT count(*) FROM (
|
||||
SELECT
|
||||
now()
|
||||
FROM
|
||||
ngx_access_log;
|
||||
ngx_access_log);
|
||||
|
||||
+--------------------------------+
|
||||
| now() |
|
||||
+--------------------------------+
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
| NOW |
|
||||
+--------------------------------+
|
||||
+----------+
|
||||
| count(*) |
|
||||
+----------+
|
||||
| 10 |
|
||||
+----------+
|
||||
|
||||
-- SQLNESS REPLACE TimestampNanosecond\(\d+ TimestampNanosecond(NOW
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE (partitioning.*) REDACTED
|
||||
EXPLAIN SELECT
|
||||
EXPLAIN SELECT count(*) FROM (
|
||||
SELECT
|
||||
now()
|
||||
FROM
|
||||
ngx_access_log;
|
||||
ngx_access_log);
|
||||
|
||||
+---------------+-------------------------------------------------------------------------------+
|
||||
| plan_type | plan |
|
||||
+---------------+-------------------------------------------------------------------------------+
|
||||
| logical_plan | MergeScan [is_placeholder=false, remote_input=[ |
|
||||
| | Projection: TimestampNanosecond(NOW, Some("+00:00")) AS now() |
|
||||
| | TableScan: ngx_access_log |
|
||||
| | ]] |
|
||||
| physical_plan | CooperativeExec |
|
||||
+---------------+-----------------------------------------------------------------------------------+
|
||||
| plan_type | plan |
|
||||
+---------------+-----------------------------------------------------------------------------------+
|
||||
| logical_plan | MergeScan [is_placeholder=false, remote_input=[ |
|
||||
| | Projection: count(Int64(1)) AS count(*) |
|
||||
| | Aggregate: groupBy=[[]], aggr=[[count(Int64(1))]] |
|
||||
| | Projection: TimestampNanosecond(NOW, Some("+00:00")) AS now() |
|
||||
| | TableScan: ngx_access_log |
|
||||
| | ]] |
|
||||
| physical_plan | CooperativeExec |
|
||||
| | MergeScanExec: REDACTED
|
||||
| | |
|
||||
+---------------+-------------------------------------------------------------------------------+
|
||||
| | |
|
||||
+---------------+-----------------------------------------------------------------------------------+
|
||||
|
||||
DROP TABLE ngx_access_log;
|
||||
|
||||
|
||||
@@ -21,18 +21,19 @@ VALUES
|
||||
("client10", "KR", "2022-01-01 00:00:09");
|
||||
|
||||
-- should not fail with mismatch timezone
|
||||
-- SQLNESS REPLACE \d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d+Z NOW
|
||||
SELECT count(*) FROM (
|
||||
SELECT
|
||||
now()
|
||||
FROM
|
||||
ngx_access_log;
|
||||
ngx_access_log);
|
||||
|
||||
-- SQLNESS REPLACE TimestampNanosecond\(\d+ TimestampNanosecond(NOW
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE (partitioning.*) REDACTED
|
||||
EXPLAIN SELECT
|
||||
EXPLAIN SELECT count(*) FROM (
|
||||
SELECT
|
||||
now()
|
||||
FROM
|
||||
ngx_access_log;
|
||||
ngx_access_log);
|
||||
|
||||
DROP TABLE ngx_access_log;
|
||||
@@ -90,6 +90,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|
||||
-- SQLNESS REPLACE (Duration.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 10, '5s') test;
|
||||
|
||||
+-+-+-+
|
||||
@@ -208,6 +209,7 @@ TQL ANALYZE FORMAT JSON (0, 10, '5s') test;
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
-- SQLNESS REPLACE (Duration.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE FORMAT JSON (0, 10, '5s') test;
|
||||
|
||||
+-+-+-+
|
||||
|
||||
@@ -44,6 +44,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|
||||
-- SQLNESS REPLACE (Duration.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE (0, 10, '5s') test;
|
||||
|
||||
DROP TABLE test;
|
||||
@@ -89,6 +90,7 @@ TQL ANALYZE FORMAT JSON (0, 10, '5s') test;
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
-- SQLNESS REPLACE (Duration.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (flat_format.*) REDACTED
|
||||
TQL ANALYZE VERBOSE FORMAT JSON (0, 10, '5s') test;
|
||||
|
||||
-- analyze with TEXT format (should be same as default)
|
||||
|
||||
@@ -1,91 +0,0 @@
|
||||
CREATE TABLE promql_column_pruning (
|
||||
ts TIMESTAMP(3) TIME INDEX,
|
||||
job STRING,
|
||||
instance STRING,
|
||||
region STRING,
|
||||
greptime_value DOUBLE,
|
||||
PRIMARY KEY(job, instance, region),
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
INSERT INTO promql_column_pruning VALUES
|
||||
(0, 'job1', 'instance1', 'region1', 1),
|
||||
(0, 'job1', 'instance2', 'region1', 2),
|
||||
(0, 'job2', 'instance1', 'region1', 3),
|
||||
(5000, 'job1', 'instance1', 'region1', 4),
|
||||
(5000, 'job1', 'instance2', 'region1', 5),
|
||||
(5000, 'job2', 'instance1', 'region1', 6),
|
||||
(10000, 'job1', 'instance1', 'region1', 7),
|
||||
(10000, 'job1', 'instance2', 'region1', 8),
|
||||
(10000, 'job2', 'instance1', 'region1', 9);
|
||||
|
||||
Affected Rows: 9
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
|
||||
-- SQLNESS REPLACE (-+) -
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
TQL ANALYZE (0, 10, '5s') sum(promql_column_pruning);
|
||||
|
||||
+-+-+-+
|
||||
| stage | node | plan_|
|
||||
+-+-+-+
|
||||
| 0_| 0_|_CooperativeExec REDACTED
|
||||
|_|_|_MergeScanExec: REDACTED
|
||||
|_|_|_|
|
||||
| 1_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED
|
||||
|_|_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|
||||
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(promql_column_pruning.greptime_value)] REDACTED
|
||||
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|
||||
|_|_|_RepartitionExec: partitioning=REDACTED
|
||||
|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(promql_column_pruning.greptime_value)] REDACTED
|
||||
|_|_|_ProjectionExec: expr=[ts@0 as ts, greptime_value@4 as greptime_value] REDACTED
|
||||
|_|_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[ts] REDACTED
|
||||
|_|_|_PromSeriesDivideExec: tags=["job", "instance", "region"] REDACTED
|
||||
|_|_|_CooperativeExec REDACTED
|
||||
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|
||||
|_|_|_|
|
||||
|_|_| Total rows: 3_|
|
||||
+-+-+-+
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
|
||||
-- SQLNESS REPLACE (-+) -
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
TQL ANALYZE (0, 10, '5s') sum(rate(promql_column_pruning[5s]));
|
||||
|
||||
+-+-+-+
|
||||
| stage | node | plan_|
|
||||
+-+-+-+
|
||||
| 0_| 0_|_CooperativeExec REDACTED
|
||||
|_|_|_MergeScanExec: REDACTED
|
||||
|_|_|_|
|
||||
| 1_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED
|
||||
|_|_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|
||||
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(prom_rate(ts_range,greptime_value,ts,Int64(5000)))] REDACTED
|
||||
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|
||||
|_|_|_RepartitionExec: partitioning=REDACTED
|
||||
|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(prom_rate(ts_range,greptime_value,ts,Int64(5000)))] REDACTED
|
||||
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|
||||
|_|_|_FilterExec: prom_rate(ts_range,greptime_value,ts,Int64(5000))@1 IS NOT NULL REDACTED
|
||||
|_|_|_ProjectionExec: expr=[ts@0 as ts, prom_rate(ts_range@5, greptime_value@4, ts@0, 5000) as prom_rate(ts_range,greptime_value,ts,Int64(5000))] REDACTED
|
||||
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[5000], eval range=[5000], time index=[ts] REDACTED
|
||||
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[ts], filter NaN: [true] REDACTED
|
||||
|_|_|_PromSeriesDivideExec: tags=["job", "instance", "region"] REDACTED
|
||||
|_|_|_CooperativeExec REDACTED
|
||||
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|
||||
|_|_|_|
|
||||
|_|_| Total rows: 2_|
|
||||
+-+-+-+
|
||||
|
||||
DROP TABLE promql_column_pruning;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
@@ -1,39 +0,0 @@
|
||||
CREATE TABLE promql_column_pruning (
|
||||
ts TIMESTAMP(3) TIME INDEX,
|
||||
job STRING,
|
||||
instance STRING,
|
||||
region STRING,
|
||||
greptime_value DOUBLE,
|
||||
PRIMARY KEY(job, instance, region),
|
||||
);
|
||||
|
||||
INSERT INTO promql_column_pruning VALUES
|
||||
(0, 'job1', 'instance1', 'region1', 1),
|
||||
(0, 'job1', 'instance2', 'region1', 2),
|
||||
(0, 'job2', 'instance1', 'region1', 3),
|
||||
(5000, 'job1', 'instance1', 'region1', 4),
|
||||
(5000, 'job1', 'instance2', 'region1', 5),
|
||||
(5000, 'job2', 'instance1', 'region1', 6),
|
||||
(10000, 'job1', 'instance1', 'region1', 7),
|
||||
(10000, 'job1', 'instance2', 'region1', 8),
|
||||
(10000, 'job2', 'instance1', 'region1', 9);
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
|
||||
-- SQLNESS REPLACE (-+) -
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
TQL ANALYZE (0, 10, '5s') sum(promql_column_pruning);
|
||||
|
||||
-- SQLNESS REPLACE (metrics.*) REDACTED
|
||||
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
|
||||
-- SQLNESS REPLACE (-+) -
|
||||
-- SQLNESS REPLACE (\s\s+) _
|
||||
-- SQLNESS REPLACE (peers.*) REDACTED
|
||||
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
|
||||
-- SQLNESS REPLACE (Hash.*) REDACTED
|
||||
TQL ANALYZE (0, 10, '5s') sum(rate(promql_column_pruning[5s]));
|
||||
|
||||
DROP TABLE promql_column_pruning;
|
||||
Reference in New Issue
Block a user