Compare commits

..

4 Commits

Author SHA1 Message Date
Ruihang Xia
c036eff794 don't drop every columns
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2026-01-13 23:10:17 +08:00
Ruihang Xia
20d7c10c61 fix scalar manipulate and histogram fold for missing some columns
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2026-01-13 21:35:45 +08:00
Ruihang Xia
e252ed35c6 apply to other plans and add sqlness case
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2026-01-13 19:58:00 +08:00
Ruihang Xia
f711384d7c feat: enable pruning for manipulate plans
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2026-01-13 15:46:34 +08:00
92 changed files with 1975 additions and 4187 deletions

3
Cargo.lock generated
View File

@@ -5467,7 +5467,7 @@ dependencies = [
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=58aeee49267fb1eafa6f9123f9d0c47dd0f62722#58aeee49267fb1eafa6f9123f9d0c47dd0f62722"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=0e316b86d765e4718d6f0ca77b1ad179f222b822#0e316b86d765e4718d6f0ca77b1ad179f222b822"
dependencies = [
"prost 0.13.5",
"prost-types 0.13.5",
@@ -12512,7 +12512,6 @@ dependencies = [
"servers",
"snafu 0.8.6",
"store-api",
"table",
"tokio",
]

View File

@@ -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 = "58aeee49267fb1eafa6f9123f9d0c47dd0f62722" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "0e316b86d765e4718d6f0ca77b1ad179f222b822" }
hex = "0.4"
http = "1"
humantime = "2.1"

View File

@@ -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,13 +509,8 @@ impl StartCommand {
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
};
let ddl_manager = DdlManager::try_new(
ddl_context,
procedure_manager.clone(),
Arc::new(StandaloneRepartitionProcedureFactory),
true,
)
.context(error::InitDdlManagerSnafu)?;
let ddl_manager = DdlManager::try_new(ddl_context, procedure_manager.clone(), true)
.context(error::InitDdlManagerSnafu)?;
let ddl_manager = if let Some(configurator) =
plugins.get::<DdlManagerConfiguratorRef<DdlManagerConfigureContext>>()

View File

@@ -34,7 +34,7 @@ use table::requests::{
};
use crate::error::{
self, ColumnNotFoundSnafu, InvalidColumnDefSnafu, InvalidIndexOptionSnafu,
ColumnNotFoundSnafu, InvalidColumnDefSnafu, InvalidIndexOptionSnafu,
InvalidSetFulltextOptionRequestSnafu, InvalidSetSkippingIndexOptionRequestSnafu,
InvalidSetTableOptionRequestSnafu, InvalidUnsetTableOptionRequestSnafu,
MissingAlterIndexOptionSnafu, MissingFieldSnafu, MissingTableMetaSnafu,
@@ -251,10 +251,6 @@ 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 {

View File

@@ -161,13 +161,6 @@ 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>;
@@ -195,7 +188,6 @@ impl ErrorExt for Error {
Error::ColumnNotFound { .. } => StatusCode::TableColumnNotFound,
Error::SqlCommon { source, .. } => source.status_code(),
Error::MissingTableMeta { .. } => StatusCode::Unexpected,
Error::Unexpected { .. } => StatusCode::Unexpected,
}
}

View File

@@ -22,7 +22,7 @@ use snafu::OptionExt;
use table::metadata::RawTableInfo;
use crate::ddl::alter_table::AlterTableProcedure;
use crate::error::{self, InvalidProtoMsgSnafu, Result};
use crate::error::{InvalidProtoMsgSnafu, Result};
impl AlterTableProcedure {
/// Makes alter kind proto that all regions can reuse.
@@ -112,10 +112,6 @@ 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()?,
}
}

View File

@@ -161,18 +161,7 @@ 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()
}
}

View File

@@ -14,19 +14,15 @@
use std::sync::Arc;
use api::v1::Repartition;
use api::v1::alter_table_expr::Kind;
use common_error::ext::BoxedError;
use common_procedure::{
BoxedProcedure, BoxedProcedureLoader, Output, ProcedureId, ProcedureManagerRef,
ProcedureWithId, watcher,
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;
@@ -44,8 +40,7 @@ use crate::ddl::drop_view::DropViewProcedure;
use crate::ddl::truncate_table::TruncateTableProcedure;
use crate::ddl::{DdlContext, utils};
use crate::error::{
CreateRepartitionProcedureSnafu, EmptyDdlTasksSnafu, ProcedureOutputSnafu,
RegisterProcedureLoaderSnafu, RegisterRepartitionProcedureLoaderSnafu, Result,
EmptyDdlTasksSnafu, ProcedureOutputSnafu, RegisterProcedureLoaderSnafu, Result,
SubmitProcedureSnafu, TableInfoNotFoundSnafu, TableNotFoundSnafu, TableRouteNotFoundSnafu,
UnexpectedLogicalRouteTableSnafu, WaitProcedureSnafu,
};
@@ -95,7 +90,6 @@ 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>,
}
@@ -149,37 +143,16 @@ 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,
};
@@ -231,63 +204,9 @@ 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(
@@ -295,28 +214,13 @@ impl DdlManager {
table_id: TableId,
alter_table_task: AlterTableTask,
) -> Result<(ProcedureId, Option<Output>)> {
// 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.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a create table task.
@@ -331,7 +235,7 @@ impl DdlManager {
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a `[CreateViewTask]`.
@@ -346,7 +250,7 @@ impl DdlManager {
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a create multiple logical table tasks.
@@ -363,7 +267,7 @@ impl DdlManager {
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes alter multiple table tasks.
@@ -380,7 +284,7 @@ impl DdlManager {
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a drop table task.
@@ -395,7 +299,7 @@ impl DdlManager {
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a create database task.
@@ -414,7 +318,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.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a drop table task.
@@ -431,7 +335,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.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
pub async fn submit_alter_database(
@@ -442,7 +346,7 @@ impl DdlManager {
let procedure = AlterDatabaseProcedure::new(alter_database_task, context)?;
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a create flow task.
@@ -456,7 +360,7 @@ impl DdlManager {
let procedure = CreateFlowProcedure::new(create_flow, query_context, context);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a drop flow task.
@@ -469,7 +373,7 @@ impl DdlManager {
let procedure = DropFlowProcedure::new(drop_flow, context);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a drop view task.
@@ -482,7 +386,7 @@ impl DdlManager {
let procedure = DropViewProcedure::new(drop_view, context);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a truncate table task.
@@ -503,7 +407,7 @@ impl DdlManager {
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Submits and executes a comment on task.
@@ -516,11 +420,10 @@ impl DdlManager {
let procedure = CommentOnProcedure::new(comment_on_task, context);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
self.execute_procedure_and_wait(procedure_with_id).await
self.submit_procedure(procedure_with_id).await
}
/// Executes a procedure and waits for the result.
async fn execute_procedure_and_wait(
async fn submit_procedure(
&self,
procedure_with_id: ProcedureWithId,
) -> Result<(ProcedureId, Option<Output>)> {
@@ -539,18 +442,6 @@ 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,
@@ -1056,12 +947,8 @@ 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;
@@ -1072,7 +959,6 @@ 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;
@@ -1101,30 +987,6 @@ 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());
@@ -1161,7 +1023,6 @@ mod tests {
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
},
procedure_manager.clone(),
Arc::new(DummyRepartitionProcedureFactory),
true,
);

View File

@@ -104,20 +104,6 @@ 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)]
@@ -1184,8 +1170,6 @@ 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 { .. }

View File

@@ -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::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 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>>,
/// 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>>,
}
impl Display for GetFileRefs {

View File

@@ -1280,11 +1280,6 @@ 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,
&region_info.region_wal_options,
new_region_wal_options,
)?;
let update_datanode_table_txn = self.datanode_table_manager().build_update_txn(
table_id,
region_info,
@@ -1296,16 +1291,13 @@ 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,
update_topic_region_txn,
]);
let txn = Txn::merge_all(vec![update_datanode_table_txn, update_table_route_txn]);
let mut r = self.kv_backend.txn(txn).await?;
@@ -1490,7 +1482,6 @@ 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,
@@ -2271,218 +2262,6 @@ 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(),
},
&current_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(),
},
&current_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());

View File

@@ -243,7 +243,7 @@ impl TopicRegionManager {
let topic_region_mapping = self.get_topic_region_mapping(table_id, &region_wal_options);
let topic_region_keys = topic_region_mapping
.iter()
.map(|(region_id, topic)| TopicRegionKey::new(*region_id, topic))
.map(|(topic, region_id)| TopicRegionKey::new(*topic, region_id))
.collect::<Vec<_>>();
let operations = topic_region_keys
.into_iter()
@@ -252,55 +252,6 @@ 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);
@@ -480,420 +431,4 @@ 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, &region_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, &region_wal_options, &region_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");
}
}
}
}

View File

@@ -37,7 +37,6 @@ 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;
@@ -74,20 +73,10 @@ impl GrpcQueryHandler for Instance {
let output = match request {
Request::Inserts(requests) => self.handle_inserts(requests, ctx.clone()).await?,
Request::RowInserts(requests) => match ctx.extension(PHYSICAL_TABLE_PARAM) {
Some(physical_table) => {
self.handle_metric_row_inserts(
requests,
ctx.clone(),
physical_table.to_string(),
)
Request::RowInserts(requests) => {
self.handle_row_inserts(requests, ctx.clone(), false, false)
.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) => {

View File

@@ -1038,7 +1038,7 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to serialize partition expression"))]
#[snafu(display("Failed to serialize partition expression: {}", source))]
SerializePartitionExpr {
#[snafu(source)]
source: partition::error::Error,
@@ -1046,20 +1046,6 @@ 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,
@@ -1177,8 +1163,7 @@ impl ErrorExt for Error {
| Error::MailboxChannelClosed { .. }
| Error::IsNotLeader { .. } => StatusCode::IllegalState,
Error::RetryLaterWithSource { source, .. } => source.status_code(),
Error::SerializePartitionExpr { source, .. }
| Error::DeserializePartitionExpr { source, .. } => source.status_code(),
Error::SerializePartitionExpr { source, .. } => source.status_code(),
Error::Unsupported { .. } => StatusCode::Unsupported,
@@ -1204,8 +1189,7 @@ impl ErrorExt for Error {
| Error::RepartitionSourceRegionMissing { .. }
| Error::RepartitionTargetRegionMissing { .. }
| Error::PartitionExprMismatch { .. }
| Error::RepartitionSourceExprMismatch { .. }
| Error::EmptyPartitionExpr { .. } => StatusCode::InvalidArguments,
| Error::RepartitionSourceExprMismatch { .. } => StatusCode::InvalidArguments,
Error::LeaseKeyFromUtf8 { .. }
| Error::LeaseValueFromUtf8 { .. }
| Error::InvalidRegionKeyFromUtf8 { .. }

View File

@@ -12,6 +12,9 @@
// 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;
@@ -26,7 +29,6 @@ mod options;
mod procedure;
mod scheduler;
mod tracker;
mod util;
pub use options::GcSchedulerOptions;
pub use procedure::BatchGcProcedure;

View File

@@ -12,22 +12,29 @@
// 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::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;
use common_telemetry::{debug, error, warn};
use snafu::{OptionExt as _, ResultExt as _};
use store_api::storage::{GcReport, RegionId};
use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId};
use table::metadata::TableId;
use crate::cluster::MetaPeerClientRef;
use crate::error::{self, Result, TableMetadataManagerSnafu};
use crate::gc::procedure::BatchGcProcedure;
use crate::service::mailbox::MailboxRef;
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};
#[async_trait::async_trait]
pub(crate) trait SchedulerCtx: Send + Sync {
@@ -38,9 +45,19 @@ 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>;
@@ -83,7 +100,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 stats = stats.stats;
let mut stats = stats.stats;
let Some(latest_stat) = stats.iter().max_by_key(|s| s.timestamp_millis).cloned() else {
continue;
@@ -112,34 +129,142 @@ 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(region_ids, full_file_listing, timeout)
.await
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, &regions, 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,
})
}
}
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 for {} regions (full_file_listing: {})",
"Sending GC instruction to datanode {} for {} regions (full_file_listing: {})",
peer,
region_ids.len(),
full_file_listing
);
let procedure = BatchGcProcedure::new(
self.mailbox.clone(),
self.table_metadata_manager.clone(),
self.server_addr.clone(),
region_ids.to_vec(),
let gc_regions = GcRegions {
regions: region_ids.to_vec(),
file_refs_manifest: file_refs_manifest.clone(),
full_file_listing,
};
let procedure = GcRegionProcedure::new(
self.mailbox.clone(),
self.server_addr.clone(),
peer,
gc_regions,
format!("GC for {} regions", region_ids.len()),
timeout,
);
let procedure_with_id = ProcedureWithId::with_random_id(Box::new(procedure));
@@ -160,8 +285,96 @@ impl DefaultGcSchedulerCtx {
),
})?;
let gc_report = BatchGcProcedure::cast_result(res)?;
let gc_report = GcRegionProcedure::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)
}
}
}
}

View File

@@ -15,17 +15,24 @@
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::{GcReport, RegionId};
use store_api::storage::{FileRefsManifest, 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
@@ -54,8 +61,6 @@ 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());
@@ -78,6 +83,17 @@ 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,
@@ -194,11 +210,28 @@ impl GcScheduler {
let all_region_ids: Vec<RegionId> = candidates.iter().map(|(_, c)| c.region_id).collect();
// Step 2: Run GC for all regions on this datanode in a single batch
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,
&region_to_peer,
self.config.mailbox_timeout,
)
.await?;
// Step 2: Create a single GcRegionProcedure for all regions on this datanode
let (gc_report, fully_listed_regions) = {
// Partition regions into full listing and fast listing in a single pass
let batch_full_listing_decisions =
let mut batch_full_listing_decisions =
self.batch_should_use_full_listing(&all_region_ids).await;
let need_full_list_regions = batch_full_listing_decisions
@@ -209,7 +242,7 @@ impl GcScheduler {
},
)
.collect_vec();
let fast_list_regions = batch_full_listing_decisions
let mut fast_list_regions = batch_full_listing_decisions
.iter()
.filter_map(
|(&region_id, &need_full)| {
@@ -224,7 +257,13 @@ impl GcScheduler {
if !fast_list_regions.is_empty() {
match self
.ctx
.gc_regions(&fast_list_regions, false, self.config.mailbox_timeout)
.gc_regions(
peer.clone(),
&fast_list_regions,
&file_refs_manifest,
false,
self.config.mailbox_timeout,
)
.await
{
Ok(report) => combined_report.merge(report),
@@ -245,7 +284,13 @@ impl GcScheduler {
if !need_full_list_regions.is_empty() {
match self
.ctx
.gc_regions(&need_full_list_regions, true, self.config.mailbox_timeout)
.gc_regions(
peer.clone(),
&need_full_list_regions,
&file_refs_manifest,
true,
self.config.mailbox_timeout,
)
.await
{
Ok(report) => combined_report.merge(report),
@@ -285,6 +330,98 @@ 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 &region_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) = &region_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],

View File

@@ -36,9 +36,10 @@ use store_api::storage::{FileRefsManifest, GcReport, RegionId};
use table::metadata::TableId;
use tokio::sync::mpsc::Sender;
use crate::error::Result;
use crate::error::{Result, UnexpectedSnafu};
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};
@@ -66,10 +67,12 @@ 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>>>,
@@ -116,12 +119,57 @@ 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(&region_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) {
@@ -165,9 +213,39 @@ 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> {

View File

@@ -152,6 +152,7 @@ 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;

View File

@@ -64,7 +64,6 @@ 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(
@@ -122,6 +121,10 @@ 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"
@@ -203,6 +206,13 @@ async fn test_get_file_references_failure() {
datanode_report.deleted_files[&region_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]
@@ -245,22 +255,42 @@ async fn test_get_table_route_failure() {
last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())),
};
// 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;
// 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();
// The table route failure should be propagated as an error
assert!(
result.is_err(),
"Expected table route failure to propagate as error"
// 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)"
);
// 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
report.failed_datanodes.contains_key(&1),
"Failed datanodes should contain the datanode with route error"
);
}

View File

@@ -123,6 +123,11 @@ 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,

View File

@@ -19,8 +19,6 @@ 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;
@@ -28,16 +26,14 @@ use common_procedure::{
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
Result as ProcedureResult, Status,
};
use common_telemetry::{error, info, warn};
use common_telemetry::{debug, 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, KvBackendSnafu, Result, SerializeToJsonSnafu, TableMetadataManagerSnafu};
use crate::gc::util::table_route_to_region;
use crate::gc::{Peer2Regions, Region2Peers};
use crate::error::{self, Result, SerializeToJsonSnafu};
use crate::gc::Region2Peers;
use crate::handler::HeartbeatMailbox;
use crate::service::mailbox::{Channel, MailboxRef};
@@ -150,73 +146,56 @@ async fn send_gc_regions(
}
}
/// 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 {
/// TODO(discord9): another procedure which do both get file refs and gc regions.
pub struct GcRegionProcedure {
mailbox: MailboxRef,
table_metadata_manager: TableMetadataManagerRef,
data: BatchGcData,
data: GcRegionData,
}
#[derive(Serialize, Deserialize)]
pub struct BatchGcData {
state: State,
/// Meta server address
pub struct GcRegionData {
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 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
peer: Peer,
gc_regions: GcRegions,
description: String,
timeout: Duration,
gc_report: Option<GcReport>,
}
#[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";
impl GcRegionProcedure {
pub const TYPE_NAME: &'static str = "metasrv-procedure::GcRegionProcedure";
pub fn new(
mailbox: MailboxRef,
table_metadata_manager: TableMetadataManagerRef,
server_addr: String,
regions: Vec<RegionId>,
full_file_listing: bool,
peer: Peer,
gc_regions: GcRegions,
description: String,
timeout: Duration,
) -> Self {
Self {
mailbox,
table_metadata_manager,
data: BatchGcData {
state: State::Start,
data: GcRegionData {
peer,
server_addr,
regions,
full_file_listing,
gc_regions,
description,
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 {
@@ -228,129 +207,111 @@ impl BatchGcProcedure {
.build()
})
}
}
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)
#[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()
.await
.context(TableMetadataManagerSnafu)
.map_err(ProcedureError::external)?;
Ok(Status::done_with_output(reply))
}
/// 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)
fn dump(&self) -> ProcedureResult<String> {
serde_json::to_string(&self.data).context(ToJsonSnafu)
}
/// 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(())
/// 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)
}
}
/// 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();
/// 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,
}
// 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);
#[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,
},
}
// 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(&regions_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(&mut self) -> Result<FileRefsManifest> {
self.set_routes_and_related_regions().await?;
async fn get_file_references(&self) -> Result<FileRefsManifest> {
use std::collections::{HashMap, HashSet};
let query_regions = &self.data.regions;
let related_regions = &self.data.related_regions;
@@ -383,25 +344,20 @@ impl BatchGcProcedure {
}
}
let mut datanode2related_regions: HashMap<Peer, HashMap<RegionId, HashSet<RegionId>>> =
let mut datanode2related_regions: HashMap<Peer, HashMap<RegionId, Vec<RegionId>>> =
HashMap::new();
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
}
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();
let mut all_cross_region_refs = HashMap::new();
for (peer, regions) in datanode2query_regions {
let related_regions_for_peer =
@@ -444,25 +400,17 @@ 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<GcReport> {
async fn send_gc_instructions(&self) -> Result<Vec<RegionId>> {
let regions = &self.data.regions;
let region_routes = &self.data.region_routes;
let file_refs = &self.data.file_refs;
@@ -470,7 +418,6 @@ 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) {
@@ -522,15 +469,10 @@ impl BatchGcProcedure {
peer, success, need_retry
);
}
all_need_retry.extend(report.need_retry_regions.clone());
all_report.merge(report);
all_need_retry.extend(report.need_retry_regions);
}
if !all_need_retry.is_empty() {
warn!("Regions need retry after batch GC: {:?}", all_need_retry);
}
Ok(all_report)
Ok(all_need_retry.into_iter().collect())
}
}
@@ -565,10 +507,12 @@ impl Procedure for BatchGcProcedure {
// Send GC instructions to all datanodes
// TODO(discord9): handle need-retry regions
match self.send_gc_instructions().await {
Ok(report) => {
self.data.state = State::UpdateRepartition;
self.data.gc_report = Some(report);
Ok(Status::executing(false))
Ok(_) => {
info!(
"Batch GC completed successfully for regions {:?}",
self.data.regions
);
Ok(Status::done())
}
Err(e) => {
error!("Failed to send GC instructions: {}", e);
@@ -576,29 +520,6 @@ 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))
}
},
}
}

View File

@@ -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::sync::Arc;
use std::time::Instant;
@@ -38,6 +38,29 @@ 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.
///

View File

@@ -30,6 +30,15 @@ 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>;
@@ -37,7 +46,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 last_cleanup = *self.last_tracker_cleanup.lock().await;
let mut last_cleanup = *self.last_tracker_cleanup.lock().await;
let now = Instant::now();
// Check if enough time has passed since last cleanup
@@ -76,6 +85,25 @@ 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(&region_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,

View File

@@ -1,55 +0,0 @@
// 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 &region_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) = &region_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,
);
}
}
}

View File

@@ -70,7 +70,6 @@ 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;
@@ -401,17 +400,8 @@ impl MetasrvBuilder {
region_failure_detector_controller,
};
let procedure_manager_c = procedure_manager.clone();
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 = DdlManager::try_new(ddl_context, procedure_manager_c, true)
.context(error::InitDdlManagerSnafu)?;
let ddl_manager = if let Some(configurator) = plugins
.as_ref()

View File

@@ -166,8 +166,6 @@ 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.
@@ -398,7 +396,6 @@ 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?;
@@ -408,7 +405,6 @@ impl RegionMigrationManager {
leader_changed,
peer_conflict,
table_not_found,
region_not_found,
migrating: migrating_region_ids,
submitted: vec![],
procedure_id: None,
@@ -449,7 +445,6 @@ impl RegionMigrationManager {
leader_changed,
peer_conflict,
table_not_found,
region_not_found,
migrating: migrating_region_ids,
submitted: vec![],
procedure_id: None,
@@ -465,7 +460,6 @@ 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),

View File

@@ -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::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, HashSet<RegionId>> {
pub(crate) fn table_regions(&self) -> HashMap<TableId, Vec<RegionId>> {
let mut table_regions = HashMap::new();
for region_id in &self.region_ids {
table_regions
.entry(region_id.table_id())
.or_insert_with(HashSet::new)
.insert(*region_id);
.or_insert_with(Vec::new)
.push(*region_id);
}
table_regions
}
@@ -105,8 +105,6 @@ 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>,
}
@@ -211,12 +209,6 @@ 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))
@@ -228,12 +220,6 @@ 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)
@@ -474,8 +460,6 @@ 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),
@@ -493,7 +477,6 @@ 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)],
}
);

View File

@@ -23,43 +23,28 @@ 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::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 common_procedure::{Context as ProcedureContext, Status};
use serde::{Deserialize, Serialize};
use snafu::{OptionExt, ResultExt};
use store_api::storage::{RegionNumber, TableId};
use table::table_name::TableName;
use store_api::storage::TableId;
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;
@@ -75,35 +60,6 @@ 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,
@@ -118,26 +74,6 @@ 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:
@@ -199,7 +135,6 @@ 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() {
@@ -223,16 +158,6 @@ 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,
@@ -240,7 +165,7 @@ impl Context {
current_table_route_value,
new_region_routes,
region_options,
&validated_region_wal_options,
region_wal_options,
)
.await
.context(error::TableMetadataManagerSnafu)
@@ -282,202 +207,3 @@ 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(())
}
}

View File

@@ -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, wal_options)
ctx.update_table_route(&table_route_value, new_region_routes)
.await?;
ctx.invalidate_table_cache().await?;

View File

@@ -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, HashMap::new())
ctx.update_table_route(&table_route_value, new_region_routes)
.await?;
ctx.invalidate_table_cache().await?;

View File

@@ -27,7 +27,6 @@ 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;
@@ -35,7 +34,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::{FromJsonSnafu, ToJsonSnafu};
use common_procedure::error::ToJsonSnafu;
use common_procedure::{
Context as ProcedureContext, Error as ProcedureError, LockKey, Procedure,
Result as ProcedureResult, Status, StringKey, UserMetadata,
@@ -61,20 +60,8 @@ 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 {
pub(crate) const TYPE_NAME: &'static str = "metasrv-procedure::RepartitionGroup";
const TYPE_NAME: &'static str = "metasrv-procedure::RepartitionGroup";
pub fn new(persistent_context: PersistentContext, context: &repartition::Context) -> Self {
let state = Box::new(RepartitionStart);
@@ -90,19 +77,12 @@ impl RepartitionGroupProcedure {
},
}
}
}
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 })
}
#[derive(Debug, Serialize)]
pub struct RepartitionGroupData<'a> {
persistent_ctx: &'a PersistentContext,
state: &'a dyn State,
}
#[async_trait::async_trait]
@@ -169,23 +149,6 @@ 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 {

View File

@@ -31,7 +31,7 @@ use store_api::storage::RegionId;
use crate::error::{self, Error, Result};
use crate::handler::HeartbeatMailbox;
use crate::procedure::repartition::group::repartition_end::RepartitionEnd;
use crate::procedure::repartition::group::update_metadata::UpdateMetadata;
use crate::procedure::repartition::group::utils::{
HandleMultipleResult, group_region_routes_by_peer, handle_multiple_results,
};
@@ -52,7 +52,10 @@ impl State for ApplyStagingManifest {
) -> Result<(Box<dyn State>, Status)> {
self.apply_staging_manifests(ctx).await?;
Ok((Box::new(RepartitionEnd), Status::executing(true)))
Ok((
Box::new(UpdateMetadata::ApplyStaging),
Status::executing(true),
))
}
fn as_any(&self) -> &dyn Any {
@@ -60,18 +63,13 @@ 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<ApplyStagingManifestInstructions> {
) -> Result<HashMap<Peer, Vec<common_meta::instruction::ApplyStagingManifest>>> {
let target_partition_expr_by_region = targets
.iter()
.map(|target| {
@@ -88,25 +86,7 @@ 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());
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(&central_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[&central_region_id].clone(),
central_region_id,
manifest_path: staging_manifest_paths[&central_region_id].clone(),
},
));
if region_ids.is_empty() {
continue;
}
}
for (peer, region_ids) in target_region_routes_by_peer {
let apply_staging_manifests = region_ids
.into_iter()
.map(|region_id| common_meta::instruction::ApplyStagingManifest {
@@ -119,10 +99,7 @@ impl ApplyStagingManifest {
instructions.insert(peer.clone(), apply_staging_manifests);
}
Ok(ApplyStagingManifestInstructions {
instructions,
central_region_instruction,
})
Ok(instructions)
}
#[allow(dead_code)]
@@ -135,10 +112,7 @@ impl ApplyStagingManifest {
let targets = &ctx.persistent_ctx.targets;
let target_routes = &prepare_result.target_routes;
let central_region_id = prepare_result.central_region;
let ApplyStagingManifestInstructions {
instructions,
central_region_instruction,
} = Self::build_apply_staging_manifest_instructions(
let instructions = Self::build_apply_staging_manifest_instructions(
staging_manifest_paths,
target_routes,
targets,
@@ -181,10 +155,8 @@ impl ApplyStagingManifest {
let results = join_all(tasks).await;
let result = handle_multiple_results(&results);
match result {
HandleMultipleResult::AllSuccessful => {
// Coninute
}
HandleMultipleResult::AllRetryable(retryable_errors) => return error::RetryLaterSnafu {
HandleMultipleResult::AllSuccessful => Ok(()),
HandleMultipleResult::AllRetryable(retryable_errors) => error::RetryLaterSnafu {
reason: format!(
"All retryable errors during applying staging manifests for repartition table {}, group id {}: {:?}",
table_id, group_id,
@@ -196,7 +168,7 @@ impl ApplyStagingManifest {
),
}
.fail(),
HandleMultipleResult::AllNonRetryable(non_retryable_errors) => return error::UnexpectedSnafu {
HandleMultipleResult::AllNonRetryable(non_retryable_errors) => error::UnexpectedSnafu {
violated: format!(
"All non retryable errors during applying staging manifests for repartition table {}, group id {}: {:?}",
table_id, group_id,
@@ -211,7 +183,7 @@ impl ApplyStagingManifest {
HandleMultipleResult::PartialRetryable {
retryable_errors,
non_retryable_errors,
} => return error::UnexpectedSnafu {
} => error::UnexpectedSnafu {
violated: format!(
"Partial retryable errors during applying staging manifests for repartition table {}, group id {}: {:?}, non retryable errors: {:?}",
table_id, group_id,
@@ -229,20 +201,6 @@ 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(

View File

@@ -128,6 +128,14 @@ 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]
@@ -149,10 +157,7 @@ impl State for RepartitionStart {
_procedure_ctx: &ProcedureContext,
) -> Result<(Box<dyn State>, Status)> {
if ctx.persistent_ctx.group_prepare_result.is_some() {
return Ok((
Box::new(UpdateMetadata::ApplyStaging),
Status::executing(true),
));
return Ok(Self::next_state());
}
let table_id = ctx.persistent_ctx.table_id;
let group_id = ctx.persistent_ctx.group_id;
@@ -172,10 +177,7 @@ impl State for RepartitionStart {
ctx.persistent_ctx.targets.len()
);
Ok((
Box::new(UpdateMetadata::ApplyStaging),
Status::executing(true),
))
Ok(Self::next_state())
}
fn as_any(&self) -> &dyn Any {

View File

@@ -12,14 +12,11 @@
// 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 common_meta::rpc::router::RegionRoute;
use snafu::{OptionExt, ResultExt, ensure};
use store_api::storage::{RegionId, RegionNumber, TableId};
use snafu::{OptionExt, ResultExt};
use store_api::storage::TableId;
use crate::error::{self, Result};
@@ -50,207 +47,3 @@ 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()));
}
}

View File

@@ -744,18 +744,6 @@ 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
@@ -1234,34 +1222,6 @@ 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();

View File

@@ -42,8 +42,6 @@ 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;
@@ -249,16 +247,6 @@ 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> {
@@ -315,9 +303,6 @@ 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.
@@ -350,10 +335,7 @@ 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, metrics)
.await
&& let Some(metadata) = write_cache.file_cache().get_parquet_meta_data(key).await
{
metrics.file_cache_hit += 1;
let metadata = Arc::new(metadata);
@@ -452,11 +434,6 @@ 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());
}
@@ -509,11 +486,6 @@ 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()
}
@@ -674,9 +646,6 @@ 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 =
@@ -703,8 +672,6 @@ 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,

View File

@@ -43,7 +43,6 @@ 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.
///
@@ -567,11 +566,7 @@ 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,
cache_metrics: &mut MetadataCacheMetrics,
) -> Option<ParquetMetaData> {
pub(crate) async fn get_parquet_meta_data(&self, key: IndexKey) -> 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
@@ -580,7 +575,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(cache_metrics).await {
match metadata_loader.load().await {
Ok(metadata) => {
CACHE_HIT
.with_label_values(&[key.file_type.metric_label()])

View File

@@ -15,8 +15,6 @@
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;

View File

@@ -1,137 +0,0 @@
// 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);
}
}

View File

@@ -81,7 +81,7 @@ mod apply_staging_manifest_test;
mod puffin_index;
use std::any::Any;
use std::collections::{HashMap, HashSet};
use std::collections::HashMap;
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>,
related_regions: HashMap<RegionId, HashSet<RegionId>>,
manifest_regions: HashMap<RegionId, Vec<RegionId>>,
) -> Result<FileRefsManifest> {
let file_ref_mgr = self.file_ref_manager();
@@ -315,30 +315,15 @@ impl MitoEngine {
.filter_map(|region_id| self.find_region(region_id))
.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
};
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();
file_ref_mgr
.get_snapshot_of_file_refs(query_regions, dst_region_to_src_regions)
.get_snapshot_of_file_refs(query_regions, related_regions)
.await
}

View File

@@ -652,14 +652,6 @@ 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,
@@ -1332,8 +1324,6 @@ 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, .. }

View File

@@ -134,7 +134,6 @@ 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();
@@ -233,7 +232,6 @@ 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();
@@ -315,7 +313,6 @@ 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;
@@ -402,7 +399,6 @@ 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;

View File

@@ -1058,6 +1058,10 @@ 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`.
@@ -1066,6 +1070,10 @@ 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)]

View File

@@ -70,15 +70,11 @@ 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 {
@@ -502,16 +498,6 @@ 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 {
@@ -537,10 +523,6 @@ 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 {
@@ -685,31 +667,6 @@ 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`.
@@ -751,12 +708,6 @@ 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.
@@ -796,10 +747,6 @@ 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,
@@ -906,25 +853,6 @@ 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 {
@@ -1060,7 +988,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 reader = self
let res = self
.access_layer
.read_sst(file.clone())
.predicate(predicate)
@@ -1068,15 +996,7 @@ 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());
#[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
.fulltext_index_appliers(self.fulltext_index_appliers.clone())
.expected_metadata(Some(self.mapper.metadata().clone()))
.flat_format(self.flat_format)
.compaction(self.compaction)
@@ -1498,7 +1418,6 @@ 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)?;

View File

@@ -115,8 +115,6 @@ 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.
@@ -153,8 +151,6 @@ 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.
@@ -163,22 +159,8 @@ 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.
@@ -201,8 +183,6 @@ 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,
@@ -263,7 +243,6 @@ impl fmt::Debug for ScanMetricsSet {
build_reader_cost,
scan_cost,
yield_cost,
convert_cost,
total_cost,
num_rows,
num_batches,
@@ -276,19 +255,11 @@ 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,
@@ -299,7 +270,6 @@ impl fmt::Debug for ScanMetricsSet {
num_distributor_batches,
distributor_scan_cost,
distributor_yield_cost,
distributor_divider_cost,
merge_metrics,
dedup_metrics,
stream_eof,
@@ -337,12 +307,6 @@ 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}")?;
@@ -356,9 +320,6 @@ 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}")?;
}
@@ -368,42 +329,9 @@ 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 {
@@ -430,12 +358,6 @@ 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 {
@@ -544,25 +466,27 @@ 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.
@@ -576,19 +500,11 @@ 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,
@@ -609,22 +525,13 @@ 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;
@@ -675,7 +582,6 @@ impl ScanMetricsSet {
num_batches,
scan_cost,
yield_cost,
divider_cost,
} = distributor_metrics;
self.num_series_send_timeout += *num_series_send_timeout;
@@ -684,7 +590,6 @@ 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.
@@ -701,11 +606,6 @@ 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());
@@ -731,10 +631,6 @@ 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"])
@@ -751,10 +647,6 @@ 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);
}
}
@@ -830,19 +722,21 @@ impl Drop for PartitionMetricsInner {
if self.explain_verbose {
common_telemetry::info!(
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}",
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}, convert_batch_costs: {}",
self.scanner_type,
self.region_id,
self.partition,
metrics,
self.convert_cost,
);
} else {
common_telemetry::debug!(
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}",
"{} finished, region_id: {}, partition: {}, scan_metrics: {:?}, convert_batch_costs: {}",
self.scanner_type,
self.region_id,
self.partition,
metrics,
self.convert_cost,
);
}
}
@@ -889,10 +783,7 @@ 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 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 metrics = ScanMetricsSet::default().with_prepare_scan_cost(query_start.elapsed());
let inner = PartitionMetricsInner {
region_id,
partition,
@@ -907,7 +798,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,
convert_cost: MetricBuilder::new(metrics_set).subset_time("convert_cost", partition),
elapsed_compute: MetricBuilder::new(metrics_set).elapsed_compute(partition),
};
Self(Arc::new(inner))
@@ -959,6 +850,9 @@ 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);
@@ -1038,8 +932,6 @@ 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`.

View File

@@ -410,9 +410,6 @@ 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(),
@@ -434,6 +431,8 @@ 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?;
@@ -442,12 +441,6 @@ 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;
@@ -455,7 +448,6 @@ impl SeqScan {
debug_assert!(!batch.is_empty());
if batch.is_empty() {
fetch_start = Instant::now();
continue;
}
@@ -484,7 +476,6 @@ impl SeqScan {
}
metrics.scan_cost += fetch_start.elapsed();
fetch_start = Instant::now();
part_metrics.merge_metrics(&metrics);
}
@@ -525,9 +516,6 @@ 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(),
@@ -546,16 +534,12 @@ 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;
@@ -563,7 +547,6 @@ impl SeqScan {
debug_assert!(record_batch.num_rows() > 0);
if record_batch.num_rows() == 0 {
fetch_start = Instant::now();
continue;
}
@@ -575,7 +558,6 @@ impl SeqScan {
}
metrics.scan_cost += fetch_start.elapsed();
fetch_start = Instant::now();
part_metrics.merge_metrics(&metrics);
}

View File

@@ -446,9 +446,6 @@ 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(),
@@ -481,6 +478,7 @@ 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? {
@@ -495,10 +493,7 @@ impl SeriesDistributor {
}
// Use divider to split series
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 {
if let Some(series_batch) = divider.push(record_batch) {
let yield_start = Instant::now();
self.senders
.send_batch(SeriesBatch::Flat(series_batch))
@@ -509,10 +504,7 @@ impl SeriesDistributor {
}
// Send any remaining batch in the divider
let divider_start = Instant::now();
let series_batch = divider.finish();
metrics.divider_cost += divider_start.elapsed();
if let Some(series_batch) = series_batch {
if let Some(series_batch) = divider.finish() {
let yield_start = Instant::now();
self.senders
.send_batch(SeriesBatch::Flat(series_batch))
@@ -544,9 +536,6 @@ 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(),
@@ -579,6 +568,7 @@ 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? {

View File

@@ -344,12 +344,6 @@ 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
}

View File

@@ -87,7 +87,7 @@ impl FileReferenceManager {
pub(crate) async fn get_snapshot_of_file_refs(
&self,
query_regions_for_mem: Vec<MitoRegionRef>,
dst_region_to_src_regions: Vec<(MitoRegionRef, HashSet<RegionId>)>,
related_regions_in_manifest: Vec<(MitoRegionRef, Vec<RegionId>)>,
) -> Result<FileRefsManifest> {
let mut ref_files = HashMap::new();
// get from in memory file handles
@@ -99,17 +99,12 @@ impl FileReferenceManager {
let mut manifest_version = HashMap::new();
let mut cross_region_refs = HashMap::new();
// get file refs from related regions' manifests
for (dst_region, src_regions) in &dst_region_to_src_regions {
let manifest = dst_region.manifest_ctx.manifest().await;
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 meta in manifest.files.values() {
if src_regions.contains(&meta.region_id) {
cross_region_refs
.entry(meta.region_id)
.or_insert_with(HashSet::new)
.insert(dst_region.region_id());
if queries.contains(&meta.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.
@@ -124,7 +119,7 @@ impl FileReferenceManager {
}
}
// not sure if related region's manifest version is needed, but record it for now.
manifest_version.insert(dst_region.region_id(), manifest.manifest_version);
manifest_version.insert(related_region.region_id(), manifest.manifest_version);
}
for r in &query_regions_for_mem {
@@ -143,7 +138,6 @@ impl FileReferenceManager {
Ok(FileRefsManifest {
file_refs: ref_files,
manifest_version,
cross_region_refs,
})
}

View File

@@ -1,650 +0,0 @@
// 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());
}
}

View File

@@ -44,9 +44,6 @@ 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};
@@ -326,7 +323,6 @@ 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;
};
@@ -565,12 +561,36 @@ 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.
// 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;
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!(
@@ -581,24 +601,7 @@ impl VectorIndexer {
}
.build()
})?;
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);
blob_data.extend_from_slice(&bitmap_len.to_le_bytes());
// Write NULL bitmap
blob_data.extend_from_slice(&null_bitmap_bytes);
// Write vector index
@@ -683,78 +686,7 @@ 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() {
@@ -905,24 +837,23 @@ mod tests {
let mut index_bytes = vec![0u8; index_size];
creator.save_to_buffer(&mut index_bytes).unwrap();
let total_size =
VECTOR_INDEX_BLOB_HEADER_SIZE + null_bitmap_bytes.len() + index_bytes.len();
// Header: 33 bytes
let header_size = 33;
let total_size = 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();
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(&bitmap_len.to_le_bytes());
blob_data.extend_from_slice(&null_bitmap_bytes);
blob_data.extend_from_slice(&index_bytes);
@@ -930,62 +861,60 @@ mod tests {
assert_eq!(blob_data.len(), total_size);
// Parse header and verify values
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!(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];
assert_eq!(
decoded.metric,
datatypes::schema::VectorDistanceMetric::L2sq
metric,
datatypes::schema::VectorDistanceMetric::L2sq.as_u8()
);
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());
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());
// Verify null bitmap can be deserialized
let null_bitmap_data =
&blob_data[header_size..header_size + decoded.null_bitmap_len as usize];
let null_bitmap_data = &blob_data[header_size..header_size + 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(),
&region_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));
}
}
}

View File

@@ -1,324 +0,0 @@
// 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);
}
}

View File

@@ -14,10 +14,8 @@
//! 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.

View File

@@ -13,7 +13,6 @@
// limitations under the License.
use std::result::Result as StdResult;
use std::sync::atomic::{AtomicU64, AtomicUsize, Ordering};
use bytes::Bytes;
use futures::FutureExt;
@@ -25,7 +24,6 @@ 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;
@@ -67,33 +65,24 @@ impl<'a> MetadataLoader<'a> {
Ok(file_size)
}
pub async fn load(&self, cache_metrics: &mut MetadataCacheMetrics) -> Result<ParquetMetaData> {
pub async fn load(&self) -> 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,
};
let metadata = reader
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)
})
}
}
@@ -111,13 +100,10 @@ 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
@@ -125,8 +111,6 @@ 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()

View File

@@ -14,8 +14,6 @@
//! Parquet reader.
#[cfg(feature = "vector_index")]
use std::collections::BTreeSet;
use std::collections::VecDeque;
use std::sync::Arc;
use std::time::{Duration, Instant};
@@ -43,8 +41,6 @@ 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,
@@ -65,8 +61,6 @@ 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,
};
@@ -80,7 +74,6 @@ 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) => {
@@ -124,12 +117,6 @@ 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.
@@ -163,10 +150,6 @@ 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,
@@ -228,19 +211,6 @@ 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 {
@@ -498,7 +468,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(cache_metrics).await?;
let metadata = metadata_loader.load().await?;
let metadata = Arc::new(metadata);
// Cache the metadata.
@@ -602,19 +572,6 @@ 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
}
@@ -649,13 +606,11 @@ 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(
@@ -723,13 +678,11 @@ 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(
@@ -793,13 +746,11 @@ 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)| {
(
@@ -848,48 +799,6 @@ 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,
@@ -920,13 +829,11 @@ 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)| {
(
@@ -1076,29 +983,6 @@ 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,
@@ -1124,8 +1008,6 @@ 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,
@@ -1135,24 +1017,9 @@ 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.
@@ -1169,22 +1036,13 @@ 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
@@ -1220,9 +1078,6 @@ 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"])
@@ -1239,9 +1094,6 @@ 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) {
@@ -1258,67 +1110,11 @@ 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 {
@@ -1330,10 +1126,6 @@ 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 {
@@ -1343,8 +1135,6 @@ impl std::fmt::Debug for MetadataCacheMetrics {
file_cache_hit,
cache_miss,
metadata_load_cost,
num_reads,
bytes_read,
} = self;
if self.is_empty() {
@@ -1363,12 +1153,6 @@ 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, "}}")
}
@@ -1386,8 +1170,6 @@ 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;
}
}

View File

@@ -520,10 +520,9 @@ 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, &mut cache_metrics)
.get_parquet_meta_data(index_key)
.await;
listener.on_file_cache_filled(index_key.file_id);

View File

@@ -736,14 +736,7 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to serialize partition expression"))]
SerializePartitionExpr {
#[snafu(implicit)]
location: Location,
source: partition::error::Error,
},
#[snafu(display("Failed to deserialize partition expression"))]
#[snafu(display("Failed to deserialize partition expression: {}", source))]
DeserializePartitionExpr {
#[snafu(source)]
source: partition::error::Error,
@@ -990,8 +983,7 @@ impl ErrorExt for Error {
| Error::MissingInsertBody { .. } => StatusCode::Internal,
Error::ExecuteAdminFunction { .. }
| Error::EncodeJson { .. }
| Error::DeserializePartitionExpr { .. }
| Error::SerializePartitionExpr { .. } => StatusCode::Unexpected,
| Error::DeserializePartitionExpr { .. } => StatusCode::Unexpected,
Error::ViewNotFound { .. }
| Error::ViewInfoNotFound { .. }
| Error::TableNotFound { .. } => StatusCode::TableNotFound,

View File

@@ -16,11 +16,9 @@ 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,
Repartition, column_def,
AlterDatabaseExpr, AlterTableExpr, CreateFlowExpr, CreateTableExpr, CreateViewExpr, column_def,
};
#[cfg(feature = "enterprise")]
use api::v1::{
@@ -90,9 +88,8 @@ use crate::error::{
FlowNotFoundSnafu, InvalidPartitionRuleSnafu, InvalidPartitionSnafu, InvalidSqlSnafu,
InvalidTableNameSnafu, InvalidViewNameSnafu, InvalidViewStmtSnafu, NotSupportedSnafu,
PartitionExprToPbSnafu, Result, SchemaInUseSnafu, SchemaNotFoundSnafu, SchemaReadOnlySnafu,
SerializePartitionExprSnafu, SubstraitCodecSnafu, TableAlreadyExistsSnafu,
TableMetadataManagerSnafu, TableNotFoundSnafu, UnrecognizedTableOptionSnafu,
ViewAlreadyExistsSnafu,
SubstraitCodecSnafu, TableAlreadyExistsSnafu, TableMetadataManagerSnafu, TableNotFoundSnafu,
UnrecognizedTableOptionSnafu, ViewAlreadyExistsSnafu,
};
use crate::expr_helper::{self, RepartitionRequest};
use crate::statement::StatementExecutor;
@@ -1411,56 +1408,20 @@ impl StatementExecutor {
.context(InvalidPartitionSnafu)?;
info!(
"Submitting repartition task for table {} (table_id={}), from {} to {} partitions",
"Repartition table {} (table_id={}) from {:?} to {:?}, new partition count: {}",
table_ref,
table_id,
from_partition_exprs.len(),
from_partition_exprs,
into_partition_exprs,
new_partition_exprs_len
);
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))
// 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()
}
#[tracing::instrument(skip_all)]
@@ -2208,7 +2169,6 @@ 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;
@@ -2226,39 +2186,6 @@ 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() {

View File

@@ -185,19 +185,6 @@ 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 {
@@ -221,32 +208,6 @@ 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]
@@ -393,7 +354,7 @@ impl PartitionExpr {
let bound: PartitionBound = serde_json::from_str(s).context(error::DeserializeJsonSnafu)?;
match bound {
PartitionBound::Expr(expr) => Ok(Some(expr.canonicalize())),
PartitionBound::Expr(expr) => Ok(Some(expr)),
_ => Ok(None),
}
}
@@ -533,7 +494,7 @@ mod tests {
.try_as_logical_expr()
.unwrap()
.to_string(),
"a > Int64(10) OR a IS NULL"
"Int64(10) < a OR a IS NULL"
);
// Test Gt with column on LHS
@@ -558,7 +519,7 @@ mod tests {
.try_as_logical_expr()
.unwrap()
.to_string(),
"a < Int64(10) OR a IS NULL"
"Int64(10) > a OR a IS NULL"
);
// Test GtEq with column on LHS

View File

@@ -34,7 +34,7 @@ use datafusion::physical_plan::{
RecordBatchStream, SendableRecordBatchStream,
};
use datafusion_common::DFSchema;
use datafusion_expr::EmptyRelation;
use datafusion_expr::{EmptyRelation, col};
use datatypes::arrow;
use datatypes::arrow::array::{ArrayRef, Float64Array, TimestampMillisecondArray};
use datatypes::arrow::datatypes::{DataType, Field, SchemaRef, TimeUnit};
@@ -107,7 +107,21 @@ impl UserDefinedLogicalNodeCore for Absent {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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]])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {

View File

@@ -40,6 +40,7 @@ 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};
@@ -88,7 +89,45 @@ impl UserDefinedLogicalNodeCore for HistogramFold {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
@@ -998,11 +1037,26 @@ 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),
@@ -1190,6 +1244,100 @@ 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![

View File

@@ -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,7 +84,37 @@ impl UserDefinedLogicalNodeCore for InstantManipulate {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
@@ -440,8 +470,6 @@ 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()
@@ -473,6 +501,14 @@ 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);
@@ -541,10 +577,104 @@ 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,
@@ -555,11 +685,13 @@ impl InstantManipulateStream {
assert_eq!(take_indices.len(), aligned_ts.len());
let indices_array = UInt64Array::from(take_indices);
let mut arrays = record_batch
.columns()
.iter()
.map(|array| compute::take(array, &indices_array, None))
.collect::<ArrowResult<Vec<_>>>()?;
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)?;
}
arrays[self.time_index] = Arc::new(TimestampMillisecondArray::from(aligned_ts));
let result = RecordBatch::try_new(record_batch.schema(), arrays)
@@ -570,6 +702,8 @@ impl InstantManipulateStream {
#[cfg(test)]
mod test {
use datafusion::common::ToDFSchema;
use datafusion::logical_expr::{EmptyRelation, LogicalPlan};
use datafusion::prelude::SessionContext;
use super::*;
@@ -611,6 +745,30 @@ 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(

View File

@@ -31,6 +31,7 @@ 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;
@@ -83,7 +84,38 @@ impl UserDefinedLogicalNodeCore for SeriesNormalize {
}
fn expressions(&self) -> Vec<datafusion::logical_expr::Expr> {
vec![]
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])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
@@ -429,8 +461,10 @@ 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;
@@ -461,6 +495,23 @@ 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());

View File

@@ -18,7 +18,7 @@ use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use common_telemetry::debug;
use common_telemetry::{debug, warn};
use datafusion::arrow::array::{Array, ArrayRef, Int64Array, TimestampMillisecondArray};
use datafusion::arrow::compute;
use datafusion::arrow::datatypes::{Field, SchemaRef};
@@ -38,6 +38,7 @@ 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;
@@ -288,7 +289,53 @@ impl UserDefinedLogicalNodeCore for RangeManipulate {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
@@ -734,16 +781,31 @@ 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),
@@ -844,6 +906,92 @@ 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(

View File

@@ -31,6 +31,7 @@ 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};
@@ -266,7 +267,36 @@ impl UserDefinedLogicalNodeCore for ScalarCalculate {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
@@ -275,15 +305,9 @@ 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();
@@ -624,6 +648,109 @@ 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),

View File

@@ -33,6 +33,7 @@ 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};
@@ -76,7 +77,38 @@ impl UserDefinedLogicalNodeCore for SeriesDivide {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
@@ -544,8 +576,10 @@ 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::*;
@@ -611,6 +645,26 @@ 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());

View File

@@ -32,6 +32,7 @@ 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};
@@ -145,7 +146,20 @@ impl UserDefinedLogicalNodeCore for UnionDistinctOn {
}
fn expressions(&self) -> Vec<Expr> {
vec![]
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<_>>(),
])
}
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
@@ -540,9 +554,43 @@ 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![

View File

@@ -25,7 +25,6 @@ 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 {
@@ -39,14 +38,13 @@ impl PuffinMetadataCache {
cache: moka::sync::CacheBuilder::new(capacity)
.name("puffin_metadata")
.weigher(|k: &String, v| puffin_metadata_weight(k, v))
.eviction_listener(move |k, v, _cause| {
.eviction_listener(|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(),
}
}
@@ -57,10 +55,6 @@ 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);
}

View File

@@ -37,5 +37,4 @@ serde.workspace = true
servers.workspace = true
snafu.workspace = true
store-api.workspace = true
table.workspace = true
tokio.workspace = true

View File

@@ -36,12 +36,6 @@ 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>;
@@ -51,7 +45,6 @@ impl ErrorExt for Error {
match self {
Error::OpenMetadataKvBackend { source, .. } => source.status_code(),
Error::External { source, .. } => source.status_code(),
Error::NoSupportRepartitionProcedure { .. } => StatusCode::Unsupported,
}
}

View File

@@ -20,4 +20,4 @@ pub mod procedure;
pub use information_extension::StandaloneInformationExtension;
pub use metadata::build_metadata_kvbackend;
pub use procedure::{StandaloneRepartitionProcedureFactory, build_procedure_manager};
pub use procedure::build_procedure_manager;

View File

@@ -14,19 +14,12 @@
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(
@@ -50,32 +43,3 @@ 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(())
}
}

View File

@@ -95,15 +95,8 @@ 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 its files
/// Manifest version when this manifest is read for it's 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)]
@@ -186,8 +179,6 @@ 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();

View File

@@ -6,7 +6,6 @@ license.workspace = true
[features]
dashboard = []
vector_index = []
[lints]
workspace = true

View File

@@ -52,7 +52,6 @@ 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};
@@ -216,7 +215,6 @@ impl GreptimeDbStandaloneBuilder {
flow_id_sequence,
));
let repartition_procedure_factory = Arc::new(StandaloneRepartitionProcedureFactory);
let ddl_manager = Arc::new(
DdlManager::try_new(
DdlContext {
@@ -231,7 +229,6 @@ impl GreptimeDbStandaloneBuilder {
region_failure_detector_controller: Arc::new(NoopRegionFailureDetectorControl),
},
procedure_manager.clone(),
repartition_procedure_factory,
register_procedure_loaders,
)
.unwrap(),

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashSet;
use std::collections::{HashMap, HashSet};
use std::time::Duration;
use common_meta::key::TableMetadataManagerRef;
@@ -105,10 +105,8 @@ 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
@@ -192,16 +190,17 @@ 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
false, // full_file_listing
region_routes,
HashMap::new(), // related_regions (empty for this simple test)
Duration::from_secs(10), // timeout
);

View File

@@ -1379,19 +1379,6 @@ 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
@@ -1558,7 +1545,14 @@ create_on_flush = "auto"
create_on_compaction = "auto"
apply_on_query = "auto"
mem_threshold_on_create = "auto"
{vector_index_config}[region_engine.mito.memtable]
[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]
type = "time_series"
[region_engine.mito.gc]

View File

@@ -544,8 +544,7 @@ 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 |
| | 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=[ |
| | 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 |
| | ]] |

View File

@@ -1,101 +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
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

View File

@@ -90,7 +90,6 @@ 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;
+-+-+-+

View File

@@ -44,7 +44,6 @@ 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;

View File

@@ -0,0 +1,66 @@
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

View File

@@ -10,6 +10,7 @@ CREATE TABLE alter_repartition_table(
device_id >= 200
);
-- valid grammar, currently not implemented
ALTER TABLE alter_repartition_table REPARTITION (
device_id < 100
) INTO (
@@ -17,17 +18,19 @@ ALTER TABLE alter_repartition_table REPARTITION (
device_id < 100 AND area >= 'South'
);
SHOW CREATE TABLE alter_repartition_table;
ALTER TABLE alter_repartition_table MERGE PARTITION (
-- 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'
);
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.
-- valid grammar, currently not implemented
ALTER TABLE alter_repartition_table MERGE PARTITION (
device_id < 100,
device_id >= 100 AND device_id < 200
);
-- invalid: empty source clause
ALTER TABLE alter_repartition_table REPARTITION () INTO (

View File

@@ -63,7 +63,6 @@ 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=~".*"};
+-+-+-+
@@ -87,7 +86,6 @@ 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=~".+"};
+-+-+-+
@@ -111,7 +109,6 @@ 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!~".*"};
+-+-+-+
@@ -137,7 +134,6 @@ 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!~".+"};
+-+-+-+

View File

@@ -26,7 +26,6 @@ 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
@@ -36,7 +35,6 @@ 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
@@ -46,7 +44,6 @@ 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
@@ -56,7 +53,6 @@ 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;

View File

@@ -24,40 +24,46 @@ VALUES
Affected Rows: 10
-- should not fail with mismatch timezone
SELECT count(*) FROM (
-- SQLNESS REPLACE \d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d+Z NOW
SELECT
now()
FROM
ngx_access_log);
ngx_access_log;
+----------+
| count(*) |
+----------+
| 10 |
+----------+
+--------------------------------+
| now() |
+--------------------------------+
| NOW |
| NOW |
| NOW |
| NOW |
| NOW |
| NOW |
| NOW |
| NOW |
| NOW |
| NOW |
+--------------------------------+
-- SQLNESS REPLACE TimestampNanosecond\(\d+ TimestampNanosecond(NOW
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE (partitioning.*) REDACTED
EXPLAIN SELECT count(*) FROM (
SELECT
EXPLAIN SELECT
now()
FROM
ngx_access_log);
ngx_access_log;
+---------------+-----------------------------------------------------------------------------------+
| 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 |
+---------------+-------------------------------------------------------------------------------+
| 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 |
| | MergeScanExec: REDACTED
| | |
+---------------+-----------------------------------------------------------------------------------+
| | |
+---------------+-------------------------------------------------------------------------------+
DROP TABLE ngx_access_log;

View File

@@ -21,19 +21,18 @@ VALUES
("client10", "KR", "2022-01-01 00:00:09");
-- should not fail with mismatch timezone
SELECT count(*) FROM (
-- SQLNESS REPLACE \d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d+Z NOW
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 count(*) FROM (
SELECT
EXPLAIN SELECT
now()
FROM
ngx_access_log);
ngx_access_log;
DROP TABLE ngx_access_log;

View File

@@ -90,7 +90,6 @@ 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;
+-+-+-+
@@ -209,7 +208,6 @@ 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;
+-+-+-+

View File

@@ -44,7 +44,6 @@ 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;
@@ -90,7 +89,6 @@ 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)

View File

@@ -0,0 +1,91 @@
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

View File

@@ -0,0 +1,39 @@
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;