From b57dfc18dc74a0ea6a0af2436c90f061d6987ead Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Fri, 27 Mar 2026 10:19:00 +0800 Subject: [PATCH 01/20] feat: pending rows batching for metrics (#7831) * feat: metric batch 2s PoC Signed-off-by: jeremyhi * chore: max_concurrent_flushes Signed-off-by: jeremyhi * chore: work channel size Signed-off-by: jeremyhi * feat(servers): add metrics and logs for pending rows batch flush Add the `FLUSH_ELAPSED` histogram metric to track the duration of pending rows batch flushes in the Prometheus store protocol handler. This provides better observability into the performance and latency of the batcher. Also update telemetry by: - Recording elapsed time for both successful and failed flush operations. - Adding an informational log upon successful flush including row count and duration. - Including elapsed time in error logs when a flush fails. Signed-off-by: Lei, HUANG * feat(servers): implement columnar batching for pending rows Refactor PendingRowsBatcher to use columnar batching for the metrics store. Incoming RowInsertRequests are now converted to RecordBatches, partitioned, and flushed via BulkInsert requests to datanodes. - Enhance MultiDimPartitionRule to handle scalar boolean predicates. - Add metrics for tracking flush failures and dropped rows. - Update dependencies to support columnar batching in servers. Signed-off-by: Lei, HUANG * feat(servers): add backpressure for pending rows Implement backpressure in PendingRowsBatcher by limiting in-flight requests with a semaphore and making the submission wait for the flush result. This ensures Prometheus write requests are throttled and only return once the data has been successfully flushed to datanodes. - Add max_inflight_requests to PromStoreOptions. - Use oneshot channels to notify submitters of flush completion. - Limit concurrent requests using a new inflight_semaphore. - Update PendingRowsBatcher::submit to wait for the flush outcome. Signed-off-by: Lei, HUANG * feat: add stage-level metrics for bulk ingestion Introduce histograms to track the elapsed time of various stages in the metric engine bulk insert path and the server's pending rows batcher. This provides better observability into the performance bottlenecks of the ingestion pipeline. Signed-off-by: Lei, HUANG * - `src/metric-engine/src/engine/bulk_insert.rs`: Removed the fallback mechanism that converted record batches to rows when bulk inserts were unsupported, along with related helper functions and unused imports. - `src/operator/src/insert.rs`: Removed an unused import (`common_time::TimeToLive::Instant`). Signed-off-by: Lei, HUANG * feat(servers): columnar Prom remote write Optimize the Prometheus remote write path by allowing direct conversion from decoded Prometheus samples to Arrow RecordBatches. This bypasses intermediate row-based representations when `PendingRowsBatcher` is active and no pipeline is used, improving ingestion efficiency. - Implement `as_record_batch_groups` in `TablesBuilder` and `PromWriteRequest`. - Add `submit_prom_record_batch_groups` to `PendingRowsBatcher`. - Introduce `DecodedPromWriteRequest` in `prom_store`. - Implement row-to-RecordBatch conversion logic in `prom_row_builder`. Signed-off-by: Lei, HUANG * Revert "feat(servers): columnar Prom remote write" This reverts commit efbb63c12a3e7fcec03858ea0351efd94fec8242. * refactor(servers): improve row to RecordBatch conversion - Use `snafu::ensure` for row validation in `rows_to_record_batch`. - Add explicit type hint for `MutableVector` to improve clarity. - Reorganize and clean up imports in `pending_rows_batcher.rs`. Signed-off-by: Lei, HUANG * perf(servers): use arrow builders for row conversion This commit optimizes the conversion from `api::v1::Rows` to `RecordBatch` by using Arrow builders directly. This avoids the overhead of `MutableVector` and `common_recordbatch`, leading to better performance in the `pending_rows_batcher`. Additionally, the `#[allow(dead_code)]` attribute is removed from `modify_batch_sparse` in the metric engine as it is now utilized. Signed-off-by: Lei, HUANG * perf(metric-engine): optimize batch modification Optimize `modify_batch_sparse` by reusing buffers, using Arrow builders, and employing fast-path encoding methods. This reduces allocations and avoids redundant downcasting and serializer overhead. Signed-off-by: Lei, HUANG * feat/metric-engine-support-bulk: **Add Environment Variable for Batch Sync Control** - `pending_rows_batcher.rs`: Introduced an environment variable `PENDING_ROWS_BATCH_SYNC` to control the synchronization behavior of batch processing. If set to true, the function will wait for the flush result; otherwise, it will return immediatel with the total rows count. Signed-off-by: Lei, HUANG * wip Signed-off-by: Lei, HUANG * chore: update and fix clippy Signed-off-by: Lei, HUANG * fix: failing test Signed-off-by: Lei, HUANG * picking-pending-rows-batcher: ### Commit Message Remove Unused Code and Simplify Error Handling - **`src/error.rs`**: Removed the `BatcherQueueFull` error variant and its associated logic, simplifying the error handling by removing unused code. - **`src/http/prom_store.rs`**: Eliminated the `try_decompress` function, streamlining the decompression logic by directly using `snappy_decompress` in `decode_remote_read_request`. Signed-off-by: Lei, HUANG * chore: parse PENDING_ROWS_BATCH_SYNC once Signed-off-by: Lei, HUANG * chore: revert unrelated changes Signed-off-by: Lei, HUANG * **Refactor Prometheus Write Handling** - **`prom_store.rs`**: Introduced `pre_write` method in `PromStoreProtocolHandler` to handle pre-write checks for Prometheus remote write requests. Updated `write` method to utilize `pre_write`. - **`server.rs`**: Modified `PendingRowsBatcher` initialization to conditionally create a batcher based on `with_metric_engine` flag. - **`http/prom_store.rs`**: Integrated `pre_write` checks before submitting requests to `PendingRowsBatcher`. - **`query_handler.rs`**: Added `pre_write` method to `PromStoreProtocolHandler` trait for pre-write operations. Signed-off-by: Lei, HUANG * picking-pending-rows-batcher: - **Fix Label Typo**: Corrected a typo in the label value from `"flush_wn ite_region"` to `"flush_write_region"` in `pending_rows_batcher.rs`. - **Refactor Array Building Logic**: Introduced a macro `build_array!` to streamline the construction of `ArrayRef` for different data types, reducing code duplication in `pending_rows_batcher.rs`. Signed-off-by: Lei, HUANG * format toml Signed-off-by: Lei, HUANG * picking-pending-rows-batcher: ### Update PromStore and PendingRowsBatcher Configuration - **`prom_store.rs`**: Set `pending_rows_flush_interval` to `Duration::ZERO` to disable automatic flushing. - **`pending_rows_batcher.rs`**: Enhance validation to disable the batcher when `flush_interval` is zero or configuration values like `max_batch_rows`, `max_concurrent_flushes`, `worker_channel_capacity`, or `max_inflight_requests` are zero, preventing potential panics or deadlocks. Signed-off-by: Lei, HUANG * picking-pending-rows-batcher: ### Update `pending_rows_flush_interval` to Zero - **Files Modified**: - `src/frontend/src/service_config/prom_store.rs` - `tests-integration/tests/http.rs` - **Key Changes**: - Updated `pending_rows_flush_interval` from `Duration::from_secs(2)` to `Duration::ZERO` in `prom_store.rs`. - Changed `pending_rows_flush_interval` configuration from `"2s"` to `"0s"` in `http.rs`. These changes set the flush interval to zero, potentially affecting how frequently pending rows are flushed. Signed-off-by: Lei, HUANG * picking-pending-rows-batcher: **Add Worker Management Enhancements** - **`metrics.rs`**: Introduced `PENDING_WORKERS` gauge to track active pending rows batch workers. - **`pending_rows_batcher.rs`**: - Added worker idle timeout logic with `WORKER_IDLE_TIMEOUT_MULTIPLIER`. - Implemented worker management functions: `spawn_worker`, `remove_worker_if_same_channel`, and `should_close_worker_on_idle_timeout`. - Enhanced worker lifecycle management to handle idle workers and ensure proper cleanup. - **Tests**: Added unit tests for worker removal and idle timeout logic. Signed-off-by: Lei, HUANG * fix: clippy Signed-off-by: Lei, HUANG --------- Signed-off-by: jeremyhi Signed-off-by: Lei, HUANG Co-authored-by: jeremyhi --- Cargo.lock | 2 + src/frontend/src/instance/prom_store.rs | 19 +- src/frontend/src/server.rs | 16 + src/frontend/src/service_config/prom_store.rs | 55 +- src/metric-engine/src/batch_modifier.rs | 17 +- src/partition/Cargo.toml | 1 + src/servers/Cargo.toml | 1 + src/servers/src/error.rs | 4 + src/servers/src/http.rs | 5 +- src/servers/src/http/prom_store.rs | 24 +- src/servers/src/lib.rs | 1 + src/servers/src/metrics.rs | 51 +- src/servers/src/pending_rows_batcher.rs | 1253 +++++++++++++++++ src/servers/src/query_handler.rs | 5 + src/servers/tests/http/prom_store_test.rs | 2 +- tests-integration/src/test_util.rs | 1 + tests-integration/tests/http.rs | 5 + 17 files changed, 1437 insertions(+), 25 deletions(-) create mode 100644 src/servers/src/pending_rows_batcher.rs diff --git a/Cargo.lock b/Cargo.lock index d8f6241136..676eaf0822 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9384,6 +9384,7 @@ dependencies = [ "common-macro", "common-meta", "common-query", + "common-telemetry", "criterion 0.7.0", "datafusion-common", "datafusion-expr", @@ -12067,6 +12068,7 @@ dependencies = [ "operator", "otel-arrow-rust", "parking_lot 0.12.4", + "partition", "permutation", "pg_interval_2", "pgwire", diff --git a/src/frontend/src/instance/prom_store.rs b/src/frontend/src/instance/prom_store.rs index 9a323eb989..c8f76753af 100644 --- a/src/frontend/src/instance/prom_store.rs +++ b/src/frontend/src/instance/prom_store.rs @@ -161,12 +161,11 @@ impl Instance { #[async_trait] impl PromStoreProtocolHandler for Instance { - async fn write( + async fn pre_write( &self, - request: RowInsertRequests, + request: &RowInsertRequests, ctx: QueryContextRef, - with_metric_engine: bool, - ) -> ServerResult { + ) -> ServerResult<()> { self.plugins .get::() .as_ref() @@ -175,7 +174,17 @@ impl PromStoreProtocolHandler for Instance { let interceptor_ref = self .plugins .get::>(); - interceptor_ref.pre_write(&request, ctx.clone())?; + interceptor_ref.pre_write(request, ctx)?; + Ok(()) + } + + async fn write( + &self, + request: RowInsertRequests, + ctx: QueryContextRef, + with_metric_engine: bool, + ) -> ServerResult { + self.pre_write(&request, ctx.clone()).await?; let output = if with_metric_engine { let physical_table = ctx diff --git a/src/frontend/src/server.rs b/src/frontend/src/server.rs index 4b51efbd33..4d0db700d1 100644 --- a/src/frontend/src/server.rs +++ b/src/frontend/src/server.rs @@ -37,6 +37,7 @@ use servers::interceptor::LogIngestInterceptorRef; use servers::metrics_handler::MetricsHandler; use servers::mysql::server::{MysqlServer, MysqlSpawnConfig, MysqlSpawnRef}; use servers::otel_arrow::OtelArrowServiceHandler; +use servers::pending_rows_batcher::PendingRowsBatcher; use servers::postgres::PostgresServer; use servers::request_memory_limiter::ServerMemoryLimiter; use servers::server::{Server, ServerHandlers}; @@ -124,12 +125,27 @@ where } if opts.prom_store.enable { + let pending_rows_batcher = if opts.prom_store.with_metric_engine { + PendingRowsBatcher::try_new( + self.instance.partition_manager().clone(), + self.instance.node_manager().clone(), + self.instance.catalog_manager().clone(), + opts.prom_store.pending_rows_flush_interval, + opts.prom_store.max_batch_rows, + opts.prom_store.max_concurrent_flushes, + opts.prom_store.worker_channel_capacity, + opts.prom_store.max_inflight_requests, + ) + } else { + None + }; builder = builder .with_prom_handler( self.instance.clone(), Some(self.instance.clone()), opts.prom_store.with_metric_engine, opts.http.prom_validation_mode, + pending_rows_batcher, ) .with_prometheus_handler(self.instance.clone()); } diff --git a/src/frontend/src/service_config/prom_store.rs b/src/frontend/src/service_config/prom_store.rs index b3adf889d2..99f1eada6d 100644 --- a/src/frontend/src/service_config/prom_store.rs +++ b/src/frontend/src/service_config/prom_store.rs @@ -12,12 +12,40 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::time::Duration; + use serde::{Deserialize, Serialize}; #[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq)] pub struct PromStoreOptions { pub enable: bool, pub with_metric_engine: bool, + #[serde(default, with = "humantime_serde")] + pub pending_rows_flush_interval: Duration, + #[serde(default = "default_max_batch_rows")] + pub max_batch_rows: usize, + #[serde(default = "default_max_concurrent_flushes")] + pub max_concurrent_flushes: usize, + #[serde(default = "default_worker_channel_capacity")] + pub worker_channel_capacity: usize, + #[serde(default = "default_max_inflight_requests")] + pub max_inflight_requests: usize, +} + +fn default_max_batch_rows() -> usize { + 100_000 +} + +fn default_max_concurrent_flushes() -> usize { + 256 +} + +fn default_worker_channel_capacity() -> usize { + 65526 +} + +fn default_max_inflight_requests() -> usize { + 3000 } impl Default for PromStoreOptions { @@ -25,18 +53,43 @@ impl Default for PromStoreOptions { Self { enable: true, with_metric_engine: true, + pending_rows_flush_interval: Duration::ZERO, + max_batch_rows: default_max_batch_rows(), + max_concurrent_flushes: default_max_concurrent_flushes(), + worker_channel_capacity: default_worker_channel_capacity(), + max_inflight_requests: default_max_inflight_requests(), } } } #[cfg(test)] mod tests { + use std::time::Duration; + use super::PromStoreOptions; + use crate::service_config::prom_store::{ + default_max_batch_rows, default_max_concurrent_flushes, default_max_inflight_requests, + default_worker_channel_capacity, + }; #[test] fn test_prom_store_options() { let default = PromStoreOptions::default(); assert!(default.enable); - assert!(default.with_metric_engine) + assert!(default.with_metric_engine); + assert_eq!(default.pending_rows_flush_interval, Duration::ZERO); + assert_eq!(default.max_batch_rows, default_max_batch_rows()); + assert_eq!( + default.max_concurrent_flushes, + default_max_concurrent_flushes() + ); + assert_eq!( + default.worker_channel_capacity, + default_worker_channel_capacity() + ); + assert_eq!( + default.max_inflight_requests, + default_max_inflight_requests() + ); } } diff --git a/src/metric-engine/src/batch_modifier.rs b/src/metric-engine/src/batch_modifier.rs index 8a5774889b..d06eaa976b 100644 --- a/src/metric-engine/src/batch_modifier.rs +++ b/src/metric-engine/src/batch_modifier.rs @@ -18,12 +18,11 @@ use std::sync::Arc; use datatypes::arrow::array::{Array, BinaryBuilder, StringArray, UInt64Array}; use datatypes::arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; use datatypes::arrow::record_batch::RecordBatch; -use datatypes::value::ValueRef; use fxhash::FxHasher; use mito_codec::row_converter::SparsePrimaryKeyCodec; use snafu::ResultExt; use store_api::storage::ColumnId; -use store_api::storage::consts::{PRIMARY_KEY_COLUMN_NAME, ReservedColumnId}; +use store_api::storage::consts::PRIMARY_KEY_COLUMN_NAME; use crate::error::{EncodePrimaryKeySnafu, Result, UnexpectedRequestSnafu}; @@ -112,7 +111,6 @@ fn build_tag_arrays<'a>( } /// Modifies a RecordBatch for sparse primary key encoding. -#[allow(dead_code)] pub(crate) fn modify_batch_sparse( batch: RecordBatch, table_id: u32, @@ -128,24 +126,17 @@ pub(crate) fn modify_batch_sparse( let mut buffer = Vec::new(); for row in 0..num_rows { buffer.clear(); - let internal = [ - (ReservedColumnId::table_id(), ValueRef::UInt32(table_id)), - ( - ReservedColumnId::tsid(), - ValueRef::UInt64(tsid_array.value(row)), - ), - ]; codec - .encode_to_vec(internal.into_iter(), &mut buffer) + .encode_internal(table_id, tsid_array.value(row), &mut buffer) .context(EncodePrimaryKeySnafu)?; let tags = sorted_tag_columns .iter() .zip(tag_arrays.iter()) .filter(|(_, arr)| !arr.is_null(row)) - .map(|(tc, arr)| (tc.column_id, ValueRef::String(arr.value(row)))); + .map(|(tc, arr)| (tc.column_id, arr.value(row).as_bytes())); codec - .encode_to_vec(tags, &mut buffer) + .encode_raw_tag_value(tags, &mut buffer) .context(EncodePrimaryKeySnafu)?; pk_builder.append_value(&buffer); diff --git a/src/partition/Cargo.toml b/src/partition/Cargo.toml index d498ed8c13..a8e3a8ae11 100644 --- a/src/partition/Cargo.toml +++ b/src/partition/Cargo.toml @@ -15,6 +15,7 @@ common-error.workspace = true common-macro.workspace = true common-meta.workspace = true common-query.workspace = true +common-telemetry.workspace = true datafusion-common.workspace = true datafusion-expr.workspace = true datafusion-physical-expr.workspace = true diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 8e84ef77d6..6531390ca3 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -88,6 +88,7 @@ opentelemetry-proto.workspace = true operator.workspace = true otel-arrow-rust.workspace = true parking_lot.workspace = true +partition.workspace = true pg_interval = { version = "0.5.2", package = "pg_interval_2" } pgwire = { version = "0.38.2", default-features = false, features = [ "server-api-ring", diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index 18ac964f05..5fae7a82db 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -56,6 +56,9 @@ pub enum Error { #[snafu(display("Internal error: {}", err_msg))] Internal { err_msg: String }, + #[snafu(display("Pending rows batcher channel closed"))] + BatcherChannelClosed, + #[snafu(display("Unsupported data type: {}, reason: {}", data_type, reason))] UnsupportedDataType { data_type: ConcreteDataType, @@ -684,6 +687,7 @@ impl ErrorExt for Error { use Error::*; match self { Internal { .. } + | BatcherChannelClosed | InternalIo { .. } | TokioIo { .. } | StartHttp { .. } diff --git a/src/servers/src/http.rs b/src/servers/src/http.rs index 506a240cac..eb2086726a 100644 --- a/src/servers/src/http.rs +++ b/src/servers/src/http.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use std::convert::Infallible; use std::fmt::Display; use std::net::SocketAddr; -use std::sync::Mutex as StdMutex; +use std::sync::{Arc, Mutex as StdMutex}; use std::time::Duration; use async_trait::async_trait; @@ -75,6 +75,7 @@ use crate::http::result::null_result::NullResponse; use crate::interceptor::LogIngestInterceptorRef; use crate::metrics::http_metrics_layer; use crate::metrics_handler::MetricsHandler; +use crate::pending_rows_batcher::PendingRowsBatcher; use crate::prometheus_handler::PrometheusHandlerRef; use crate::query_handler::sql::ServerSqlQueryHandlerRef; use crate::query_handler::{ @@ -585,12 +586,14 @@ impl HttpServerBuilder { pipeline_handler: Option, prom_store_with_metric_engine: bool, prom_validation_mode: PromValidationMode, + pending_rows_batcher: Option>, ) -> Self { let state = PromStoreState { prom_store_handler: handler, pipeline_handler, prom_store_with_metric_engine, prom_validation_mode, + pending_rows_batcher, }; Self { diff --git a/src/servers/src/http/prom_store.rs b/src/servers/src/http/prom_store.rs index 58c6e0eddd..bfc072e84e 100644 --- a/src/servers/src/http/prom_store.rs +++ b/src/servers/src/http/prom_store.rs @@ -35,6 +35,7 @@ use snafu::prelude::*; use crate::error::{self, InternalSnafu, PipelineSnafu, Result}; use crate::http::extractor::PipelineInfo; use crate::http::header::{GREPTIME_DB_HEADER_METRICS, write_cost_header_map}; +use crate::pending_rows_batcher::PendingRowsBatcher; use crate::prom_remote_write::decode::PromSeriesProcessor; use crate::prom_remote_write::decode_remote_write_request; use crate::prom_remote_write::validation::PromValidationMode; @@ -52,6 +53,7 @@ pub struct PromStoreState { pub pipeline_handler: Option, pub prom_store_with_metric_engine: bool, pub prom_validation_mode: PromValidationMode, + pub pending_rows_batcher: Option>, } #[derive(Debug, Serialize, Deserialize)] @@ -92,6 +94,7 @@ pub async fn remote_write( pipeline_handler, prom_store_with_metric_engine, prom_validation_mode, + pending_rows_batcher, } = state; if let Some(_vm_handshake) = params.get_vm_proto_version { @@ -100,9 +103,11 @@ pub async fn remote_write( let db = params.db.clone().unwrap_or_default(); query_ctx.set_channel(Channel::Prometheus); - if let Some(physical_table) = params.physical_table { - query_ctx.set_extension(PHYSICAL_TABLE_PARAM, physical_table); - } + let physical_table = params + .physical_table + .clone() + .unwrap_or_else(|| GREPTIME_PHYSICAL_TABLE.to_string()); + query_ctx.set_extension(PHYSICAL_TABLE_PARAM, physical_table.clone()); let query_ctx = Arc::new(query_ctx); let _timer = crate::metrics::METRIC_HTTP_PROM_STORE_WRITE_ELAPSED .with_label_values(&[db.as_str()]) @@ -135,6 +140,19 @@ pub async fn remote_write( req.as_insert_requests() }; + if prom_store_with_metric_engine && let Some(batcher) = pending_rows_batcher { + for (temp_ctx, reqs) in req.as_req_iter(query_ctx) { + prom_store_handler + .pre_write(&reqs, temp_ctx.clone()) + .await?; + let rows = batcher.submit(reqs, temp_ctx).await?; + crate::metrics::PROM_STORE_REMOTE_WRITE_SAMPLES + .with_label_values(&[db.as_str()]) + .inc_by(rows); + } + return Ok((StatusCode::NO_CONTENT, write_cost_header_map(0)).into_response()); + } + let mut cost = 0; for (temp_ctx, reqs) in req.as_req_iter(query_ctx) { let cnt: u64 = reqs diff --git a/src/servers/src/lib.rs b/src/servers/src/lib.rs index 9ee7395691..c44c674b9e 100644 --- a/src/servers/src/lib.rs +++ b/src/servers/src/lib.rs @@ -41,6 +41,7 @@ pub mod mysql; pub mod opentsdb; pub mod otel_arrow; pub mod otlp; +pub mod pending_rows_batcher; mod pipeline; pub mod postgres; pub mod prom_remote_write; diff --git a/src/servers/src/metrics.rs b/src/servers/src/metrics.rs index 25a900ed3d..37f923b73d 100644 --- a/src/servers/src/metrics.rs +++ b/src/servers/src/metrics.rs @@ -121,13 +121,62 @@ lazy_static! { /// Duration to convert prometheus write request to gRPC request. pub static ref METRIC_HTTP_PROM_STORE_CONVERT_ELAPSED: Histogram = METRIC_HTTP_PROM_STORE_CODEC_ELAPSED .with_label_values(&["convert"]); - /// The samples count of Prometheus remote write. + /// The samples count of Prometheus remote write. pub static ref PROM_STORE_REMOTE_WRITE_SAMPLES: IntCounterVec = register_int_counter_vec!( "greptime_servers_prometheus_remote_write_samples", "frontend prometheus remote write samples", &[METRIC_DB_LABEL] ) .unwrap(); + pub static ref PENDING_BATCHES: IntGauge = register_int_gauge!( + "greptime_prom_store_pending_batches", + "Number of pending batches waiting to be flushed" + ) + .unwrap(); + pub static ref PENDING_ROWS: IntGauge = register_int_gauge!( + "greptime_prom_store_pending_rows", + "Number of pending rows waiting to be flushed" + ) + .unwrap(); + pub static ref PENDING_WORKERS: IntGauge = register_int_gauge!( + "greptime_prom_store_pending_workers", + "Number of active pending rows batch workers" + ) + .unwrap(); + pub static ref FLUSH_TOTAL: IntCounter = register_int_counter!( + "greptime_prom_store_flush_total", + "Total number of batch flushes" + ) + .unwrap(); + pub static ref FLUSH_ROWS: Histogram = register_histogram!( + "greptime_prom_store_flush_rows", + "Number of rows per flush", + vec![100.0, 1000.0, 10000.0, 50000.0, 100000.0, 500000.0] + ) + .unwrap(); + pub static ref FLUSH_ELAPSED: Histogram = register_histogram!( + "greptime_prom_store_flush_elapsed", + "Elapsed time of pending rows batch flush in seconds", + vec![0.005, 0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 60.0, 300.0] + ) + .unwrap(); + pub static ref FLUSH_DROPPED_ROWS: IntCounter = register_int_counter!( + "greptime_pending_rows_flush_dropped_rows", + "Total rows dropped due to pending rows flush failures" + ) + .unwrap(); + pub static ref FLUSH_FAILURES: IntCounter = register_int_counter!( + "greptime_pending_rows_flush_failures", + "Total pending rows flush failures" + ) + .unwrap(); + pub static ref PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED: HistogramVec = register_histogram_vec!( + "greptime_prom_store_pending_rows_batch_ingest_stage_elapsed", + "Elapsed time of pending rows batch ingestion stages in seconds", + &["stage"], + vec![0.0005, 0.001, 0.005, 0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 60.0] + ) + .unwrap(); /// Http prometheus read duration per database. pub static ref METRIC_HTTP_PROM_STORE_READ_ELAPSED: HistogramVec = register_histogram_vec!( "greptime_servers_http_prometheus_read_elapsed", diff --git a/src/servers/src/pending_rows_batcher.rs b/src/servers/src/pending_rows_batcher.rs new file mode 100644 index 0000000000..f8486e3636 --- /dev/null +++ b/src/servers/src/pending_rows_batcher.rs @@ -0,0 +1,1253 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::Arc; +use std::time::{Duration, Instant}; + +use api::helper::ColumnDataTypeWrapper; +use api::v1::region::{ + BulkInsertRequest, RegionRequest, RegionRequestHeader, bulk_insert_request, region_request, +}; +use api::v1::value::ValueData; +use api::v1::{ArrowIpc, RowInsertRequests, Rows}; +use arrow::array::{ + ArrayRef, Float64Builder, StringBuilder, TimestampMicrosecondBuilder, + TimestampMillisecondBuilder, TimestampNanosecondBuilder, TimestampSecondBuilder, + new_null_array, +}; +use arrow::compute::{cast, concat_batches, filter_record_batch}; +use arrow::datatypes::{Field, Schema as ArrowSchema}; +use arrow::record_batch::RecordBatch; +use arrow_schema::TimeUnit; +use bytes::Bytes; +use catalog::CatalogManagerRef; +use common_grpc::flight::{FlightEncoder, FlightMessage}; +use common_meta::node_manager::NodeManagerRef; +use common_query::prelude::GREPTIME_PHYSICAL_TABLE; +use common_telemetry::tracing_context::TracingContext; +use common_telemetry::{debug, error, info, warn}; +use dashmap::DashMap; +use dashmap::mapref::entry::Entry; +use datatypes::data_type::DataType; +use datatypes::prelude::ConcreteDataType; +use partition::manager::PartitionRuleManagerRef; +use session::context::QueryContextRef; +use snafu::{ResultExt, ensure}; +use store_api::storage::RegionId; +use tokio::sync::{OwnedSemaphorePermit, Semaphore, broadcast, mpsc, oneshot}; + +use crate::error; +use crate::error::{Error, Result}; +use crate::metrics::{ + FLUSH_DROPPED_ROWS, FLUSH_ELAPSED, FLUSH_FAILURES, FLUSH_ROWS, FLUSH_TOTAL, PENDING_BATCHES, + PENDING_ROWS, PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED, PENDING_WORKERS, +}; + +const PHYSICAL_TABLE_KEY: &str = "physical_table"; +/// Whether wait for ingestion result before reply to client. +const PENDING_ROWS_BATCH_SYNC_ENV: &str = "PENDING_ROWS_BATCH_SYNC"; +const WORKER_IDLE_TIMEOUT_MULTIPLIER: u32 = 3; + +#[derive(Debug, Clone, Hash, Eq, PartialEq)] +struct BatchKey { + catalog: String, + schema: String, + physical_table: String, +} + +#[derive(Debug)] +struct TableBatch { + table_name: String, + batches: Vec, + row_count: usize, +} + +struct PendingBatch { + tables: HashMap, + created_at: Option, + total_row_count: usize, + ctx: Option, + waiters: Vec, +} + +struct FlushWaiter { + response_tx: oneshot::Sender>, + _permit: OwnedSemaphorePermit, +} + +struct FlushBatch { + table_batches: Vec, + total_row_count: usize, + ctx: QueryContextRef, + waiters: Vec, +} + +#[derive(Clone)] +struct PendingWorker { + tx: mpsc::Sender, +} + +enum WorkerCommand { + Submit { + table_batches: Vec<(String, RecordBatch)>, + total_rows: usize, + ctx: QueryContextRef, + response_tx: oneshot::Sender>, + _permit: OwnedSemaphorePermit, + }, +} + +// Batch key is derived from QueryContext; it assumes catalog/schema/physical_table fully +// define the write target and must remain consistent across the batch. +fn batch_key_from_ctx(ctx: &QueryContextRef) -> BatchKey { + let physical_table = ctx + .extension(PHYSICAL_TABLE_KEY) + .unwrap_or(GREPTIME_PHYSICAL_TABLE) + .to_string(); + BatchKey { + catalog: ctx.current_catalog().to_string(), + schema: ctx.current_schema(), + physical_table, + } +} + +/// Prometheus remote write pending rows batcher. +pub struct PendingRowsBatcher { + workers: Arc>, + flush_interval: Duration, + max_batch_rows: usize, + partition_manager: PartitionRuleManagerRef, + node_manager: NodeManagerRef, + catalog_manager: CatalogManagerRef, + flush_semaphore: Arc, + inflight_semaphore: Arc, + worker_channel_capacity: usize, + pending_rows_batch_sync: bool, + shutdown: broadcast::Sender<()>, +} + +impl PendingRowsBatcher { + #[allow(clippy::too_many_arguments)] + pub fn try_new( + partition_manager: PartitionRuleManagerRef, + node_manager: NodeManagerRef, + catalog_manager: CatalogManagerRef, + flush_interval: Duration, + max_batch_rows: usize, + max_concurrent_flushes: usize, + worker_channel_capacity: usize, + max_inflight_requests: usize, + ) -> Option> { + // Disable the batcher if flush is disabled or configuration is invalid. + // Zero values for these knobs either cause panics (e.g., zero-capacity channels) + // or deadlocks (e.g., semaphores with no permits). + if flush_interval.is_zero() + || max_batch_rows == 0 + || max_concurrent_flushes == 0 + || worker_channel_capacity == 0 + || max_inflight_requests == 0 + { + return None; + } + + let (shutdown, _) = broadcast::channel(1); + let pending_rows_batch_sync = std::env::var(PENDING_ROWS_BATCH_SYNC_ENV) + .ok() + .as_deref() + .and_then(|v| v.parse::().ok()) + .unwrap_or(true); + let workers = Arc::new(DashMap::new()); + PENDING_WORKERS.set(workers.len() as i64); + + Some(Arc::new(Self { + workers, + flush_interval, + max_batch_rows, + partition_manager, + node_manager, + catalog_manager, + flush_semaphore: Arc::new(Semaphore::new(max_concurrent_flushes)), + inflight_semaphore: Arc::new(Semaphore::new(max_inflight_requests)), + worker_channel_capacity, + pending_rows_batch_sync, + shutdown, + })) + } + + pub async fn submit(&self, requests: RowInsertRequests, ctx: QueryContextRef) -> Result { + let (table_batches, total_rows) = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["submit_build_table_batches"]) + .start_timer(); + build_table_batches(requests)? + }; + if total_rows == 0 { + return Ok(0); + } + let table_batches = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["submit_align_region_schema"]) + .start_timer(); + self.align_table_batches_to_region_schema(table_batches, &ctx) + .await? + }; + + let permit = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["submit_acquire_inflight_permit"]) + .start_timer(); + self.inflight_semaphore + .clone() + .acquire_owned() + .await + .map_err(|_| Error::BatcherChannelClosed)? + }; + + let (response_tx, response_rx) = oneshot::channel(); + + let batch_key = batch_key_from_ctx(&ctx); + let mut cmd = Some(WorkerCommand::Submit { + table_batches, + total_rows, + ctx, + response_tx, + _permit: permit, + }); + + { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["submit_send_to_worker"]) + .start_timer(); + + for _ in 0..2 { + let worker = self.get_or_spawn_worker(batch_key.clone()); + let Some(worker_cmd) = cmd.take() else { + break; + }; + + match worker.tx.send(worker_cmd).await { + Ok(()) => break, + Err(err) => { + cmd = Some(err.0); + remove_worker_if_same_channel( + self.workers.as_ref(), + &batch_key, + &worker.tx, + ); + } + } + } + + if cmd.is_some() { + return Err(Error::BatcherChannelClosed); + } + } + + if self.pending_rows_batch_sync { + let result = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["submit_wait_flush_result"]) + .start_timer(); + response_rx.await.map_err(|_| Error::BatcherChannelClosed)? + }; + result.map(|()| total_rows as u64) + } else { + Ok(total_rows as u64) + } + } + + async fn align_table_batches_to_region_schema( + &self, + table_batches: Vec<(String, RecordBatch)>, + ctx: &QueryContextRef, + ) -> Result> { + let catalog = ctx.current_catalog().to_string(); + let schema = ctx.current_schema(); + let mut region_schemas: HashMap> = HashMap::new(); + let mut aligned_batches = Vec::with_capacity(table_batches.len()); + + for (table_name, record_batch) in table_batches { + let region_schema = if let Some(region_schema) = region_schemas.get(&table_name) { + region_schema.clone() + } else { + let table = self + .catalog_manager + .table(&catalog, &schema, &table_name, Some(ctx.as_ref())) + .await + .map_err(|err| Error::Internal { + err_msg: format!( + "Failed to resolve table {} for pending batch alignment: {}", + table_name, err + ), + })? + .ok_or_else(|| Error::Internal { + err_msg: format!( + "Table not found during pending batch alignment: {}", + table_name + ), + })?; + let region_schema = table.table_info().meta.schema.arrow_schema().clone(); + region_schemas.insert(table_name.clone(), region_schema.clone()); + region_schema + }; + + let record_batch = align_record_batch_to_schema(record_batch, region_schema.as_ref())?; + aligned_batches.push((table_name, record_batch)); + } + + Ok(aligned_batches) + } + + fn get_or_spawn_worker(&self, key: BatchKey) -> PendingWorker { + if let Some(worker) = self.workers.get(&key) + && !worker.tx.is_closed() + { + return worker.clone(); + } + + let entry = self.workers.entry(key.clone()); + match entry { + Entry::Occupied(mut worker) => { + if worker.get().tx.is_closed() { + let new_worker = self.spawn_worker(key); + worker.insert(new_worker.clone()); + PENDING_WORKERS.set(self.workers.len() as i64); + new_worker + } else { + worker.get().clone() + } + } + Entry::Vacant(vacant) => { + let worker = self.spawn_worker(key); + + vacant.insert(worker.clone()); + PENDING_WORKERS.set(self.workers.len() as i64); + worker + } + } + } + + fn spawn_worker(&self, key: BatchKey) -> PendingWorker { + let (tx, rx) = mpsc::channel(self.worker_channel_capacity); + let worker = PendingWorker { tx: tx.clone() }; + let worker_idle_timeout = self + .flush_interval + .checked_mul(WORKER_IDLE_TIMEOUT_MULTIPLIER) + .unwrap_or(self.flush_interval); + + start_worker( + key, + worker.tx.clone(), + self.workers.clone(), + rx, + self.shutdown.clone(), + self.partition_manager.clone(), + self.node_manager.clone(), + self.catalog_manager.clone(), + self.flush_interval, + worker_idle_timeout, + self.max_batch_rows, + self.flush_semaphore.clone(), + ); + + worker + } +} + +impl Drop for PendingRowsBatcher { + fn drop(&mut self) { + let _ = self.shutdown.send(()); + } +} + +impl PendingBatch { + fn new() -> Self { + Self { + tables: HashMap::new(), + created_at: None, + total_row_count: 0, + ctx: None, + waiters: Vec::new(), + } + } +} + +#[allow(clippy::too_many_arguments)] +fn start_worker( + key: BatchKey, + worker_tx: mpsc::Sender, + workers: Arc>, + mut rx: mpsc::Receiver, + shutdown: broadcast::Sender<()>, + partition_manager: PartitionRuleManagerRef, + node_manager: NodeManagerRef, + catalog_manager: CatalogManagerRef, + flush_interval: Duration, + worker_idle_timeout: Duration, + max_batch_rows: usize, + flush_semaphore: Arc, +) { + tokio::spawn(async move { + let mut batch = PendingBatch::new(); + let mut interval = tokio::time::interval(flush_interval); + let mut shutdown_rx = shutdown.subscribe(); + let idle_deadline = tokio::time::Instant::now() + worker_idle_timeout; + let idle_timer = tokio::time::sleep_until(idle_deadline); + tokio::pin!(idle_timer); + + loop { + tokio::select! { + cmd = rx.recv() => { + match cmd { + Some(WorkerCommand::Submit { table_batches, total_rows, ctx, response_tx, _permit }) => { + idle_timer.as_mut().reset(tokio::time::Instant::now() + worker_idle_timeout); + + if batch.total_row_count == 0 { + batch.created_at = Some(Instant::now()); + batch.ctx = Some(ctx); + PENDING_BATCHES.inc(); + } + + batch.waiters.push(FlushWaiter { response_tx, _permit }); + + for (table_name, record_batch) in table_batches { + let entry = batch.tables.entry(table_name.clone()).or_insert_with(|| TableBatch { + table_name, + batches: Vec::new(), + row_count: 0, + }); + entry.row_count += record_batch.num_rows(); + entry.batches.push(record_batch); + } + + batch.total_row_count += total_rows; + PENDING_ROWS.add(total_rows as i64); + + if batch.total_row_count >= max_batch_rows + && let Some(flush) = drain_batch(&mut batch) { + spawn_flush( + flush, + partition_manager.clone(), + node_manager.clone(), + catalog_manager.clone(), + flush_semaphore.clone(), + ).await; + } + } + None => { + if let Some(flush) = drain_batch(&mut batch) { + flush_batch( + flush, + partition_manager.clone(), + node_manager.clone(), + catalog_manager.clone(), + ).await; + } + break; + } + } + } + _ = &mut idle_timer => { + if !should_close_worker_on_idle_timeout(batch.total_row_count, rx.len()) { + idle_timer + .as_mut() + .reset(tokio::time::Instant::now() + worker_idle_timeout); + continue; + } + + debug!( + "Closing idle pending rows worker due to timeout: catalog={}, schema={}, physical_table={}", + key.catalog, + key.schema, + key.physical_table + ); + break; + } + _ = interval.tick() => { + if let Some(created_at) = batch.created_at + && batch.total_row_count > 0 + && created_at.elapsed() >= flush_interval + && let Some(flush) = drain_batch(&mut batch) { + spawn_flush( + flush, + partition_manager.clone(), + node_manager.clone(), + catalog_manager.clone(), + flush_semaphore.clone(), + ).await; + } + } + _ = shutdown_rx.recv() => { + if let Some(flush) = drain_batch(&mut batch) { + flush_batch( + flush, + partition_manager.clone(), + node_manager.clone(), + catalog_manager.clone(), + ).await; + } + break; + } + } + } + + remove_worker_if_same_channel(workers.as_ref(), &key, &worker_tx); + }); +} + +fn remove_worker_if_same_channel( + workers: &DashMap, + key: &BatchKey, + worker_tx: &mpsc::Sender, +) -> bool { + if let Some(worker) = workers.get(key) + && worker.tx.same_channel(worker_tx) + { + drop(worker); + workers.remove(key); + PENDING_WORKERS.set(workers.len() as i64); + return true; + } + + false +} + +fn should_close_worker_on_idle_timeout(total_row_count: usize, queued_requests: usize) -> bool { + total_row_count == 0 && queued_requests == 0 +} + +fn drain_batch(batch: &mut PendingBatch) -> Option { + if batch.total_row_count == 0 { + return None; + } + + let ctx = match batch.ctx.take() { + Some(ctx) => ctx, + None => { + flush_with_error(batch, "Pending batch missing context"); + return None; + } + }; + + let total_row_count = batch.total_row_count; + let table_batches = std::mem::take(&mut batch.tables).into_values().collect(); + let waiters = std::mem::take(&mut batch.waiters); + batch.total_row_count = 0; + batch.created_at = None; + + PENDING_ROWS.sub(total_row_count as i64); + PENDING_BATCHES.dec(); + + Some(FlushBatch { + table_batches, + total_row_count, + ctx, + waiters, + }) +} + +async fn spawn_flush( + flush: FlushBatch, + partition_manager: PartitionRuleManagerRef, + node_manager: NodeManagerRef, + catalog_manager: CatalogManagerRef, + semaphore: Arc, +) { + match semaphore.acquire_owned().await { + Ok(permit) => { + tokio::spawn(async move { + let _permit = permit; + flush_batch(flush, partition_manager, node_manager, catalog_manager).await; + }); + } + Err(err) => { + warn!(err; "Flush semaphore closed, flushing inline"); + flush_batch(flush, partition_manager, node_manager, catalog_manager).await; + } + } +} + +async fn flush_batch( + flush: FlushBatch, + partition_manager: PartitionRuleManagerRef, + node_manager: NodeManagerRef, + catalog_manager: CatalogManagerRef, +) { + let FlushBatch { + table_batches, + total_row_count, + ctx, + waiters, + } = flush; + let start = Instant::now(); + let mut first_error: Option = None; + + let catalog = ctx.current_catalog().to_string(); + let schema = ctx.current_schema(); + + macro_rules! record_failure { + ($row_count:expr, $msg:expr) => {{ + let msg = $msg; + if first_error.is_none() { + first_error = Some(msg.clone()); + } + mark_flush_failure($row_count, &msg); + }}; + } + + for table_batch in table_batches { + let Some(first_batch) = table_batch.batches.first() else { + continue; + }; + + let schema_ref = first_batch.schema(); + let record_batch = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_concat_table_batches"]) + .start_timer(); + match concat_batches(&schema_ref, &table_batch.batches) { + Ok(batch) => batch, + Err(err) => { + record_failure!( + table_batch.row_count, + format!( + "Failed to concat table batch {}: {:?}", + table_batch.table_name, err + ) + ); + continue; + } + } + }; + + let table = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_resolve_table"]) + .start_timer(); + match catalog_manager + .table( + &catalog, + &schema, + &table_batch.table_name, + Some(ctx.as_ref()), + ) + .await + { + Ok(Some(table)) => table, + Ok(None) => { + record_failure!( + table_batch.row_count, + format!( + "Table not found during pending flush: {}", + table_batch.table_name + ) + ); + continue; + } + Err(err) => { + record_failure!( + table_batch.row_count, + format!( + "Failed to resolve table {} for pending flush: {:?}", + table_batch.table_name, err + ) + ); + continue; + } + } + }; + let table_info = table.table_info(); + + let partition_rule = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_fetch_partition_rule"]) + .start_timer(); + match partition_manager + .find_table_partition_rule(&table_info) + .await + { + Ok(rule) => rule, + Err(err) => { + record_failure!( + table_batch.row_count, + format!( + "Failed to fetch partition rule for table {}: {:?}", + table_batch.table_name, err + ) + ); + continue; + } + } + }; + + let region_masks = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_split_record_batch"]) + .start_timer(); + match partition_rule.0.split_record_batch(&record_batch) { + Ok(masks) => masks, + Err(err) => { + record_failure!( + table_batch.row_count, + format!( + "Failed to split record batch for table {}: {:?}", + table_batch.table_name, err + ) + ); + continue; + } + } + }; + + for (region_number, mask) in region_masks { + if mask.select_none() { + continue; + } + + let region_batch = if mask.select_all() { + record_batch.clone() + } else { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_filter_record_batch"]) + .start_timer(); + match filter_record_batch(&record_batch, mask.array()) { + Ok(batch) => batch, + Err(err) => { + record_failure!( + table_batch.row_count, + format!( + "Failed to filter record batch for table {}: {:?}", + table_batch.table_name, err + ) + ); + continue; + } + } + }; + + let row_count = region_batch.num_rows(); + if row_count == 0 { + continue; + } + + let region_id = RegionId::new(table_info.table_id(), region_number); + let datanode = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_resolve_region_leader"]) + .start_timer(); + match partition_manager.find_region_leader(region_id).await { + Ok(peer) => peer, + Err(err) => { + record_failure!( + row_count, + format!("Failed to resolve region leader {}: {:?}", region_id, err) + ); + continue; + } + } + }; + + let (schema_bytes, data_header, payload) = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_encode_ipc"]) + .start_timer(); + match record_batch_to_ipc(region_batch) { + Ok(encoded) => encoded, + Err(err) => { + record_failure!( + row_count, + format!( + "Failed to encode Arrow IPC for region {}: {:?}", + region_id, err + ) + ); + continue; + } + } + }; + + let request = RegionRequest { + header: Some(RegionRequestHeader { + tracing_context: TracingContext::from_current_span().to_w3c(), + ..Default::default() + }), + body: Some(region_request::Body::BulkInsert(BulkInsertRequest { + region_id: region_id.as_u64(), + partition_expr_version: None, + body: Some(bulk_insert_request::Body::ArrowIpc(ArrowIpc { + schema: schema_bytes, + data_header, + payload, + })), + })), + }; + + let datanode = node_manager.datanode(&datanode).await; + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["flush_write_region"]) + .start_timer(); + match datanode.handle(request).await { + Ok(_) => { + FLUSH_TOTAL.inc(); + FLUSH_ROWS.observe(row_count as f64); + } + Err(err) => { + record_failure!( + row_count, + format!( + "Bulk insert flush failed for region {}: {:?}", + region_id, err + ) + ); + } + } + } + } + + let elapsed = start.elapsed().as_secs_f64(); + FLUSH_ELAPSED.observe(elapsed); + info!( + "Pending rows batch flushed, total rows: {}, elapsed time: {}s", + total_row_count, elapsed + ); + + notify_waiters(waiters, &first_error); +} + +fn notify_waiters(waiters: Vec, first_error: &Option) { + for waiter in waiters { + let result = match first_error { + Some(err_msg) => Err(Error::Internal { + err_msg: err_msg.clone(), + }), + None => Ok(()), + }; + let _ = waiter.response_tx.send(result); + // waiter._permit is dropped here, releasing the inflight semaphore slot + } +} + +fn mark_flush_failure(row_count: usize, message: &str) { + error!("Pending rows batch flush failed, message: {}", message); + FLUSH_FAILURES.inc(); + FLUSH_DROPPED_ROWS.inc_by(row_count as u64); +} + +fn flush_with_error(batch: &mut PendingBatch, message: &str) { + if batch.total_row_count == 0 { + return; + } + + let row_count = batch.total_row_count; + let waiters = std::mem::take(&mut batch.waiters); + batch.tables.clear(); + batch.total_row_count = 0; + batch.created_at = None; + batch.ctx = None; + + PENDING_ROWS.sub(row_count as i64); + PENDING_BATCHES.dec(); + + let err_msg = Some(message.to_string()); + notify_waiters(waiters, &err_msg); + mark_flush_failure(row_count, message); +} + +fn build_table_batches(requests: RowInsertRequests) -> Result<(Vec<(String, RecordBatch)>, usize)> { + let mut table_batches = Vec::with_capacity(requests.inserts.len()); + let mut total_rows = 0; + + for request in requests.inserts { + let Some(rows) = request.rows else { + continue; + }; + if rows.rows.is_empty() { + continue; + } + + let record_batch = rows_to_record_batch(&rows)?; + total_rows += record_batch.num_rows(); + table_batches.push((request.table_name, record_batch)); + } + + Ok((table_batches, total_rows)) +} + +fn align_record_batch_to_schema( + record_batch: RecordBatch, + target_schema: &ArrowSchema, +) -> Result { + let source_schema = record_batch.schema(); + if source_schema.as_ref() == target_schema { + return Ok(record_batch); + } + + for source_field in source_schema.fields() { + if target_schema + .column_with_name(source_field.name()) + .is_none() + { + return Err(Error::Internal { + err_msg: format!( + "Failed to align record batch schema, column '{}' not found in target schema", + source_field.name() + ), + }); + } + } + + let row_count = record_batch.num_rows(); + let mut columns = Vec::with_capacity(target_schema.fields().len()); + for target_field in target_schema.fields() { + let column = if let Some((index, source_field)) = + source_schema.column_with_name(target_field.name()) + { + let source_column = record_batch.column(index).clone(); + if source_field.data_type() == target_field.data_type() { + source_column + } else { + cast(source_column.as_ref(), target_field.data_type()).map_err(|err| { + Error::Internal { + err_msg: format!( + "Failed to cast column '{}' to target type {:?}: {}", + target_field.name(), + target_field.data_type(), + err + ), + } + })? + } + } else { + new_null_array(target_field.data_type(), row_count) + }; + columns.push(column); + } + + RecordBatch::try_new(Arc::new(target_schema.clone()), columns).map_err(|err| Error::Internal { + err_msg: format!("Failed to build aligned record batch: {}", err), + }) +} + +fn rows_to_record_batch(rows: &Rows) -> Result { + let row_count = rows.rows.len(); + let column_count = rows.schema.len(); + + for (idx, row) in rows.rows.iter().enumerate() { + ensure!( + row.values.len() == column_count, + error::InternalSnafu { + err_msg: format!( + "Column count mismatch in row {}, expected {}, got {}", + idx, + column_count, + row.values.len() + ) + } + ); + } + + let mut fields = Vec::with_capacity(column_count); + let mut columns = Vec::with_capacity(column_count); + + for (idx, column_schema) in rows.schema.iter().enumerate() { + let datatype_wrapper = ColumnDataTypeWrapper::try_new( + column_schema.datatype, + column_schema.datatype_extension.clone(), + )?; + let data_type = ConcreteDataType::from(datatype_wrapper); + fields.push(Field::new( + column_schema.column_name.clone(), + data_type.as_arrow_type(), + true, + )); + columns.push(build_arrow_array( + rows, + idx, + &column_schema.column_name, + data_type.as_arrow_type(), + row_count, + )?); + } + + RecordBatch::try_new(Arc::new(ArrowSchema::new(fields)), columns).context(error::ArrowSnafu) +} + +fn build_arrow_array( + rows: &Rows, + col_idx: usize, + column_name: &String, + column_data_type: arrow::datatypes::DataType, + row_count: usize, +) -> Result { + macro_rules! build_array { + ($builder:expr, $( $pattern:pat => $value:expr ),+ $(,)?) => {{ + let mut builder = $builder; + for row in &rows.rows { + match row.values[col_idx].value_data.as_ref() { + $(Some($pattern) => builder.append_value($value),)+ + Some(v) => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!("Unexpected value: {:?}", v), + } + .fail(); + } + None => builder.append_null(), + } + } + Arc::new(builder.finish()) as ArrayRef + }}; + } + + let array: ArrayRef = match column_data_type { + arrow::datatypes::DataType::Float64 => { + build_array!(Float64Builder::with_capacity(row_count), ValueData::F64Value(v) => *v) + } + arrow::datatypes::DataType::Utf8 => build_array!( + StringBuilder::with_capacity(row_count, 0), + ValueData::StringValue(v) => v + ), + arrow::datatypes::DataType::Timestamp(u, _) => match u { + TimeUnit::Second => build_array!( + TimestampSecondBuilder::with_capacity(row_count), + ValueData::TimestampSecondValue(v) => *v + ), + TimeUnit::Millisecond => build_array!( + TimestampMillisecondBuilder::with_capacity(row_count), + ValueData::TimestampMillisecondValue(v) => *v + ), + TimeUnit::Microsecond => build_array!( + TimestampMicrosecondBuilder::with_capacity(row_count), + ValueData::DatetimeValue(v) => *v, + ValueData::TimestampMicrosecondValue(v) => *v + ), + TimeUnit::Nanosecond => build_array!( + TimestampNanosecondBuilder::with_capacity(row_count), + ValueData::TimestampNanosecondValue(v) => *v + ), + }, + ty => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Unexpected column type {:?}, column name: {}", + ty, column_name + ), + } + .fail(); + } + }; + + Ok(array) +} + +fn record_batch_to_ipc(record_batch: RecordBatch) -> Result<(Bytes, Bytes, Bytes)> { + let mut encoder = FlightEncoder::default(); + let schema = encoder.encode_schema(record_batch.schema().as_ref()); + let mut iter = encoder + .encode(FlightMessage::RecordBatch(record_batch)) + .into_iter(); + let Some(flight_data) = iter.next() else { + return Err(Error::Internal { + err_msg: "Failed to encode empty flight data".to_string(), + }); + }; + if iter.next().is_some() { + return Err(Error::NotSupported { + feat: "bulk insert RecordBatch with dictionary arrays".to_string(), + }); + } + + Ok(( + schema.data_header, + flight_data.data_header, + flight_data.data_body, + )) +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use api::v1::value::ValueData; + use api::v1::{ColumnDataType, ColumnSchema, Row, Rows, SemanticType, Value}; + use arrow::array::{Array, Float64Array, Int32Array, Int64Array, StringArray}; + use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; + use arrow::record_batch::RecordBatch; + use dashmap::DashMap; + use tokio::sync::mpsc; + + use super::{ + BatchKey, PendingWorker, WorkerCommand, align_record_batch_to_schema, + remove_worker_if_same_channel, rows_to_record_batch, should_close_worker_on_idle_timeout, + }; + + #[test] + fn test_rows_to_record_batch() { + let rows = Rows { + schema: vec![ + ColumnSchema { + column_name: "ts".to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "value".to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "host".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ], + rows: vec![ + Row { + values: vec![ + Value { + value_data: Some(ValueData::TimestampMillisecondValue(1000)), + }, + Value { + value_data: Some(ValueData::F64Value(42.0)), + }, + Value { + value_data: Some(ValueData::StringValue("h1".to_string())), + }, + ], + }, + Row { + values: vec![ + Value { + value_data: Some(ValueData::TimestampMillisecondValue(2000)), + }, + Value { value_data: None }, + Value { + value_data: Some(ValueData::StringValue("h2".to_string())), + }, + ], + }, + ], + }; + + let rb = rows_to_record_batch(&rows).unwrap(); + assert_eq!(2, rb.num_rows()); + assert_eq!(3, rb.num_columns()); + } + + #[test] + fn test_align_record_batch_to_schema_reorder_and_fill_missing() { + let source_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("host", DataType::Utf8, true), + Field::new("value", DataType::Float64, true), + ])); + let source = RecordBatch::try_new( + source_schema, + vec![ + Arc::new(StringArray::from(vec!["h1"])), + Arc::new(Float64Array::from(vec![42.0])), + ], + ) + .unwrap(); + + let target = ArrowSchema::new(vec![ + Field::new("ts", DataType::Int64, true), + Field::new("host", DataType::Utf8, true), + Field::new("value", DataType::Float64, true), + ]); + + let aligned = align_record_batch_to_schema(source, &target).unwrap(); + assert_eq!(aligned.schema().as_ref(), &target); + assert_eq!(1, aligned.num_rows()); + assert_eq!(3, aligned.num_columns()); + let ts = aligned + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(ts.is_null(0)); + } + + #[test] + fn test_align_record_batch_to_schema_cast_column_type() { + let source_schema = Arc::new(ArrowSchema::new(vec![Field::new( + "value", + DataType::Int32, + true, + )])); + let source = RecordBatch::try_new( + source_schema, + vec![Arc::new(Int32Array::from(vec![Some(7), None]))], + ) + .unwrap(); + + let target = ArrowSchema::new(vec![Field::new("value", DataType::Int64, true)]); + let aligned = align_record_batch_to_schema(source, &target).unwrap(); + let value = aligned + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(Some(7), value.iter().next().flatten()); + assert!(value.is_null(1)); + } + + #[test] + fn test_remove_worker_if_same_channel_removes_matching_entry() { + let workers = DashMap::new(); + let key = BatchKey { + catalog: "greptime".to_string(), + schema: "public".to_string(), + physical_table: "phy".to_string(), + }; + + let (tx, _rx) = mpsc::channel::(1); + workers.insert(key.clone(), PendingWorker { tx: tx.clone() }); + + assert!(remove_worker_if_same_channel(&workers, &key, &tx)); + assert!(!workers.contains_key(&key)); + } + + #[test] + fn test_remove_worker_if_same_channel_keeps_newer_entry() { + let workers = DashMap::new(); + let key = BatchKey { + catalog: "greptime".to_string(), + schema: "public".to_string(), + physical_table: "phy".to_string(), + }; + + let (stale_tx, _stale_rx) = mpsc::channel::(1); + let (fresh_tx, _fresh_rx) = mpsc::channel::(1); + workers.insert( + key.clone(), + PendingWorker { + tx: fresh_tx.clone(), + }, + ); + + assert!(!remove_worker_if_same_channel(&workers, &key, &stale_tx)); + assert!(workers.contains_key(&key)); + assert!(workers.get(&key).unwrap().tx.same_channel(&fresh_tx)); + } + + #[test] + fn test_worker_idle_timeout_close_decision() { + assert!(should_close_worker_on_idle_timeout(0, 0)); + assert!(!should_close_worker_on_idle_timeout(1, 0)); + assert!(!should_close_worker_on_idle_timeout(0, 1)); + } +} diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index 21c7646560..b55502e742 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -86,6 +86,11 @@ pub struct PromStoreResponse { #[async_trait] pub trait PromStoreProtocolHandler { + /// Runs pre-write checks/hooks for prometheus remote write requests. + async fn pre_write(&self, _request: &RowInsertRequests, _ctx: QueryContextRef) -> Result<()> { + Ok(()) + } + /// Handling prometheus remote write requests async fn write( &self, diff --git a/src/servers/tests/http/prom_store_test.rs b/src/servers/tests/http/prom_store_test.rs index b1e974d3d3..c5d5207486 100644 --- a/src/servers/tests/http/prom_store_test.rs +++ b/src/servers/tests/http/prom_store_test.rs @@ -120,7 +120,7 @@ fn make_test_app(tx: mpsc::Sender<(String, Vec)>) -> Router { let instance = Arc::new(DummyInstance { tx }); let server = HttpServerBuilder::new(http_opts) .with_sql_handler(instance.clone()) - .with_prom_handler(instance, None, true, PromValidationMode::Unchecked) + .with_prom_handler(instance, None, true, PromValidationMode::Unchecked, None) .build(); server.build(server.make_app()).unwrap() } diff --git a/tests-integration/src/test_util.rs b/tests-integration/src/test_util.rs index 2bf6e812c7..8e7c3ce8a6 100644 --- a/tests-integration/src/test_util.rs +++ b/tests-integration/src/test_util.rs @@ -623,6 +623,7 @@ pub async fn setup_test_prom_app_with_frontend( Some(frontend_ref.clone()), true, PromValidationMode::Strict, + None, ) .with_prometheus_handler(frontend_ref) .with_greptime_config_options(instance.opts.datanode_options().to_toml().unwrap()) diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 05a34eb5b7..933fcadf6b 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -1483,6 +1483,11 @@ enable = true [prom_store] enable = true with_metric_engine = true +pending_rows_flush_interval = "0s" +max_batch_rows = 100000 +max_concurrent_flushes = 256 +worker_channel_capacity = 65526 +max_inflight_requests = 3000 [wal] provider = "raft_engine" From 6f2ec120598f5ed5eca436d6497968ef7a49453c Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Fri, 27 Mar 2026 15:22:02 +0800 Subject: [PATCH 02/20] feat(partition): add expression split utility (#7822) * feat(partition): add expression split utility Implement MVP split logic with checker-safe degrade paths and move module under utils/split with aligned split naming and tests. Signed-off-by: WenyXu * refactor: minor Signed-off-by: WenyXu * chore: header Signed-off-by: WenyXu * chore: styling Signed-off-by: WenyXu * fix(partition): degrade split when branch becomes unsatisfiable Detect empty conjunction branches after split and return EmptyBranch instead of silently succeeding. This keeps split behavior aligned with expected partition semantics and adds regression tests for contradictory cuts. Signed-off-by: WenyXu * fix(partition): tighten empty-branch split detection Handle Eq/NotEq contradictions and discrete-gap unsatisfiable ranges in split empty-branch checks. Add regression tests for equality conflicts and impossible int/date intervals. Signed-off-by: WenyXu * fix(partition): degrade singleton and uint impossible split branches Signed-off-by: WenyXu * fix(partition): enforce finite float bounds in split degradation Signed-off-by: WenyXu * fix(partition): drop date and timestamp support from expr split Signed-off-by: WenyXu * fix(partition): reject nan and infinity in expr split Signed-off-by: WenyXu * refactor(partition): reuse conjunction bound collection in expr split Signed-off-by: WenyXu * chore: fmt Signed-off-by: WenyXu * chore: add comments Signed-off-by: WenyXu * fix(partition): respect null-first semantics in empty branch checks Signed-off-by: WenyXu * refactor(partition): restrict expr split to range-only shapes Signed-off-by: WenyXu * docs(partition): clarify split helper scope and test names Signed-off-by: WenyXu * chore: add comments Signed-off-by: WenyXu --------- Signed-off-by: WenyXu --- src/partition/src/lib.rs | 1 + src/partition/src/utils.rs | 15 + src/partition/src/utils/split.rs | 1263 ++++++++++++++++++++++++++++++ 3 files changed, 1279 insertions(+) create mode 100644 src/partition/src/utils.rs create mode 100644 src/partition/src/utils/split.rs diff --git a/src/partition/src/lib.rs b/src/partition/src/lib.rs index c9257e8ee5..647210d1d5 100644 --- a/src/partition/src/lib.rs +++ b/src/partition/src/lib.rs @@ -27,5 +27,6 @@ pub mod partition; pub mod simplify; pub mod splitter; pub mod subtask; +pub mod utils; pub use crate::partition::{PartitionRule, PartitionRuleRef}; diff --git a/src/partition/src/utils.rs b/src/partition/src/utils.rs new file mode 100644 index 0000000000..de212b5589 --- /dev/null +++ b/src/partition/src/utils.rs @@ -0,0 +1,15 @@ +// 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. + +pub mod split; diff --git a/src/partition/src/utils/split.rs b/src/partition/src/utils/split.rs new file mode 100644 index 0000000000..4b1980e34e --- /dev/null +++ b/src/partition/src/utils/split.rs @@ -0,0 +1,1263 @@ +// 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. + +//! Expression split utilities for partition rules. +//! +//! This module provides a conservative way to split one partition expression `R` +//! by a split expression `S` into: +//! - `left = R AND S` +//! - `right = R AND NOT(S)` +//! +//! The implementation intentionally reuses existing partition components +//! (`Collider`, `simplify`, `PartitionChecker`) and degrades to no-split when an +//! unsupported shape/type is encountered. + +use std::collections::{BTreeMap, HashSet}; + +use datatypes::value::Value; +use snafu::ensure; + +use crate::collider::Collider; +use crate::error::{self, Result}; +use crate::expr::{Operand, PartitionExpr, RestrictedOp}; + +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum ExprSplitDegradeReason { + UnsupportedType, + UnsupportedNotExpansion, + ColliderRejected, + EmptyBranch, +} + +/// Splits one partition expression with a split predicate. +/// +/// Returns `(left, right)` on success, where: +/// - `left = R AND S` +/// - `right = R AND NOT(S)` +/// +/// Supported shape: +/// - `split_expr` must be a single atomic range predicate (`<`, `<=`, `>`, `>=`). +/// - `base_expr` must be a pure `AND` tree of atomic range predicates, possibly +/// across unrelated columns. +/// +/// Returns [`ExprSplitDegradeReason`] when this cannot safely process the shape/type. +pub fn split_partition_expr( + base_expr: PartitionExpr, + split_expr: PartitionExpr, +) -> std::result::Result<(PartitionExpr, PartitionExpr), ExprSplitDegradeReason> { + let base = base_expr.canonicalize(); + let split = split_expr.canonicalize(); + + if validate_supported_expr(&base).is_err() || validate_supported_expr(&split).is_err() { + return Err(ExprSplitDegradeReason::UnsupportedType); + } + + if !validate_base_expr_shape(&base) || !validate_split_expr_shape(&split) { + return Err(ExprSplitDegradeReason::UnsupportedType); + } + + let not_split = match negate_split_expr(&split) { + Ok(expr) => expr, + Err(_) => { + return Err(ExprSplitDegradeReason::UnsupportedNotExpansion); + } + }; + + let left_raw = base.clone().and(split); + let right_raw = base.clone().and(not_split); + + if Collider::new(std::slice::from_ref(&left_raw)).is_err() + || Collider::new(std::slice::from_ref(&right_raw)).is_err() + { + return Err(ExprSplitDegradeReason::ColliderRejected); + } + + let left_expr = simplify_and_bounds(left_raw); + let right_expr = simplify_and_bounds(right_raw); + + if is_empty_and_conjunction(&left_expr) || is_empty_and_conjunction(&right_expr) { + return Err(ExprSplitDegradeReason::EmptyBranch); + } + + Ok((left_expr, right_expr)) +} + +/// Detects whether a pure conjunction expression is definitely unsatisfiable. +/// +/// Scope and intent: +/// - This checker is intentionally conservative. +/// - It only analyzes expressions that can be flattened into: +/// `atom1 AND atom2 AND ...` +/// - If any `OR` is present, it returns `false` (unknown / not handled here). +/// +/// Strategy: +/// - For each column, keep only the tightest lower bound (`>` / `>=`) and +/// tightest upper bound (`<` / `<=`). +/// - `=` is treated as both lower and upper bound at the same value. +/// - `!=` is tracked per column to catch direct conflicts with `=`. +/// - After bounds are collected, the conjunction is empty iff for any column: +/// - lower value is greater than upper value, or +/// - lower value equals upper value but at least one bound is exclusive. +/// - For discrete domains (`Int*`, `UInt*`), adjacent open bounds with no +/// representable value in between are also treated as empty. +/// +/// Notes: +/// - This is still a conservative fast path focused on conjunction emptiness +/// detection for split degradation. +/// - `split_partition_expr` currently restricts its main path to range-only +/// conjunctions, but this helper remains slightly more general so shared +/// bound collection and direct conflict checks stay reusable. +fn is_empty_and_conjunction(expr: &PartitionExpr) -> bool { + let Some(collected) = collect_conjunction_bounds(expr) else { + return false; + }; + + if collected.has_conflict { + return true; + } + + let CollectedConjunction { + lowers, + uppers, + not_equals, + passthrough: _, + has_conflict: _, + } = collected; + + if lowers + .iter() + .any(|(col, lower)| !uppers.contains_key(col) && is_strictly_greater_than_domain_max(lower)) + { + return true; + } + + // Check for contradiction between collected lower/upper bounds per column. + lowers.into_iter().any(|(col, lower)| { + let Some(upper) = uppers.get(&col) else { + return false; + }; + + match lower.value.partial_cmp(&upper.value) { + Some(std::cmp::Ordering::Greater) => true, + Some(std::cmp::Ordering::Equal) => { + if !lower.inclusive || !upper.inclusive { + true + } else { + not_equals + .get(&col) + .is_some_and(|excluded| excluded.contains(&lower.value)) + } + } + Some(std::cmp::Ordering::Less) => { + match ( + discrete_value_index(&lower.value), + discrete_value_index(&upper.value), + ) { + (Some(lower_idx), Some(upper_idx)) => { + let min_candidate = if lower.inclusive { + Some(lower_idx) + } else { + lower_idx.checked_add(1) + }; + let max_candidate = if upper.inclusive { + Some(upper_idx) + } else { + upper_idx.checked_sub(1) + }; + match (min_candidate, max_candidate) { + (Some(min_val), Some(max_val)) => min_val > max_val, + _ => true, + } + } + _ => false, + } + } + _ => false, + } + }) +} + +fn discrete_value_index(v: &Value) -> Option { + match v { + Value::Int8(x) => Some(*x as i128), + Value::Int16(x) => Some(*x as i128), + Value::Int32(x) => Some(*x as i128), + Value::Int64(x) => Some(*x as i128), + Value::UInt8(x) => Some(*x as i128), + Value::UInt16(x) => Some(*x as i128), + Value::UInt32(x) => Some(*x as i128), + Value::UInt64(x) => Some(*x as i128), + _ => None, + } +} + +fn is_strictly_greater_than_domain_max(bound: &LowerBound) -> bool { + if bound.inclusive { + return false; + } + + is_domain_max_value(&bound.value) +} + +fn is_domain_max_value(v: &Value) -> bool { + match v { + Value::Float32(v) => v.0 == f32::MAX, + Value::Float64(v) => v.0 == f64::MAX, + Value::UInt8(v) => *v == u8::MAX, + Value::UInt16(v) => *v == u16::MAX, + Value::UInt32(v) => *v == u32::MAX, + Value::UInt64(v) => *v == u64::MAX, + Value::Int8(v) => *v == i8::MAX, + Value::Int16(v) => *v == i16::MAX, + Value::Int32(v) => *v == i32::MAX, + Value::Int64(v) => *v == i64::MAX, + _ => false, + } +} + +/// Rewrites `NOT(expr)` into an equivalent `PartitionExpr` without introducing a unary NOT node. +/// +/// Why this function exists: +/// - `PartitionExpr` only models binary operators. +/// - Cut logic needs `R AND NOT(S)`. +/// - We therefore rewrite `NOT(S)` into an equivalent binary-expression tree. +/// +/// Rewrite rules: +/// - Atomic comparisons: +/// - `=` <-> `!=` +/// - `<` <-> `>=` +/// - `<=` <-> `>` +/// - `>` <-> `<=` +/// - `>=` <-> `<` +/// - Boolean composition: +/// - `NOT(A AND B)` => `NOT(A) OR NOT(B)` +/// - `NOT(A OR B)` => `NOT(A) AND NOT(B)` +/// +/// Failure behavior: +/// - For `AND/OR`, both sides must be `Operand::Expr`; otherwise returns `NoExprOperand`. +/// - Any unsupported shape bubbles up as an error and the caller degrades to no-split. +pub fn negate_split_expr(expr: &PartitionExpr) -> Result { + match expr.op() { + RestrictedOp::Eq + | RestrictedOp::NotEq + | RestrictedOp::Lt + | RestrictedOp::LtEq + | RestrictedOp::Gt + | RestrictedOp::GtEq => { + // Atomic negate by operator inversion. + let op = match expr.op() { + RestrictedOp::Eq => RestrictedOp::NotEq, + RestrictedOp::NotEq => RestrictedOp::Eq, + RestrictedOp::Lt => RestrictedOp::GtEq, + RestrictedOp::LtEq => RestrictedOp::Gt, + RestrictedOp::Gt => RestrictedOp::LtEq, + RestrictedOp::GtEq => RestrictedOp::Lt, + RestrictedOp::And | RestrictedOp::Or => unreachable!(), + }; + Ok(PartitionExpr::new( + expr.lhs().clone(), + op, + expr.rhs().clone(), + )) + } + RestrictedOp::And | RestrictedOp::Or => { + // De Morgan transform on recursive sub-expressions. + let lhs = match expr.lhs() { + Operand::Expr(lhs) => lhs, + other => { + return error::NoExprOperandSnafu { + operand: other.clone(), + } + .fail(); + } + }; + let rhs = match expr.rhs() { + Operand::Expr(rhs) => rhs, + other => { + return error::NoExprOperandSnafu { + operand: other.clone(), + } + .fail(); + } + }; + let not_lhs = negate_split_expr(lhs)?; + let not_rhs = negate_split_expr(rhs)?; + let op = match expr.op() { + // NOT(A AND B) => NOT(A) OR NOT(B) + RestrictedOp::And => RestrictedOp::Or, + // NOT(A OR B) => NOT(A) AND NOT(B) + RestrictedOp::Or => RestrictedOp::And, + _ => unreachable!(), + }; + Ok(PartitionExpr::new( + Operand::Expr(not_lhs), + op, + Operand::Expr(not_rhs), + )) + } + } +} + +pub fn validate_supported_expr(expr: &PartitionExpr) -> Result<()> { + match expr.op() { + RestrictedOp::And | RestrictedOp::Or => { + let lhs = match expr.lhs() { + Operand::Expr(lhs) => lhs, + other => { + return error::NoExprOperandSnafu { + operand: other.clone(), + } + .fail(); + } + }; + let rhs = match expr.rhs() { + Operand::Expr(rhs) => rhs, + other => { + return error::NoExprOperandSnafu { + operand: other.clone(), + } + .fail(); + } + }; + validate_supported_expr(lhs)?; + validate_supported_expr(rhs)?; + Ok(()) + } + _ => validate_atomic(expr), + } +} + +fn validate_atomic(expr: &PartitionExpr) -> Result<()> { + let (lhs, rhs) = (expr.lhs(), expr.rhs()); + match (lhs, rhs) { + (Operand::Column(_), Operand::Value(v)) | (Operand::Value(v), Operand::Column(_)) => { + ensure!( + is_supported_value(v), + error::InvalidExprSnafu { expr: expr.clone() } + ); + if is_nan_value(v) || is_infinite_value(v) { + return error::InvalidExprSnafu { expr: expr.clone() }.fail(); + } + Ok(()) + } + _ => error::InvalidExprSnafu { expr: expr.clone() }.fail(), + } +} + +/// Validates that `base_expr` stays within the range-only split contract. +/// +/// Scope and intent: +/// - The split utility only handles interval-style partition predicates. +/// - `base_expr` may mention multiple columns, but it must remain a pure `AND` +/// tree of atomic range predicates. +fn validate_base_expr_shape(expr: &PartitionExpr) -> bool { + let mut atoms = Vec::new(); + if !collect_and_atoms(expr, &mut atoms) { + return false; + } + + atoms + .into_iter() + .all(|atom| is_atomic_range_expr(&atom.canonicalize())) +} + +/// Validates that `split_expr` is a single atomic range predicate. +/// +/// This restriction keeps `NOT(split_expr)` in the same range-only subset so the +/// resulting left/right branches stay within the supported contract. +fn validate_split_expr_shape(expr: &PartitionExpr) -> bool { + is_atomic_range_expr(expr) +} + +/// Returns whether `expr` is an atomic `column op value` range predicate. +/// +/// Supported operators are limited to `<`, `<=`, `>`, and `>=`. +fn is_atomic_range_expr(expr: &PartitionExpr) -> bool { + atom_col_op_val(expr).is_some_and(|(_, op, _)| { + matches!( + op, + RestrictedOp::Lt | RestrictedOp::LtEq | RestrictedOp::Gt | RestrictedOp::GtEq + ) + }) +} + +fn is_supported_value(v: &Value) -> bool { + matches!( + v, + Value::Int8(_) + | Value::Int16(_) + | Value::Int32(_) + | Value::Int64(_) + | Value::UInt8(_) + | Value::UInt16(_) + | Value::UInt32(_) + | Value::UInt64(_) + | Value::Float32(_) + | Value::Float64(_) + | Value::String(_) + ) +} + +fn is_nan_value(v: &Value) -> bool { + match v { + Value::Float32(x) => x.0.is_nan(), + Value::Float64(x) => x.0.is_nan(), + _ => false, + } +} + +fn is_infinite_value(v: &Value) -> bool { + match v { + Value::Float32(x) => x.0.is_infinite(), + Value::Float64(x) => x.0.is_infinite(), + _ => false, + } +} + +#[derive(Debug, Clone)] +struct LowerBound { + value: Value, + inclusive: bool, +} + +#[derive(Debug, Clone)] +struct UpperBound { + value: Value, + inclusive: bool, +} + +struct CollectedConjunction { + lowers: BTreeMap, + uppers: BTreeMap, + not_equals: BTreeMap>, + passthrough: Vec, + has_conflict: bool, +} + +/// Simplifies conjunction-only range predicates by keeping the tightest bounds per column. +/// +/// This pass is intentionally conservative and only runs when the whole expression +/// can be flattened into `atom1 AND atom2 AND ...` without any `OR` node. +/// +/// Behavior: +/// - For each column, collect all lower-bound predicates (`>` / `>=`) and keep the +/// tightest one. +/// - For each column, collect all upper-bound predicates (`<` / `<=`) and keep the +/// tightest one. +/// - Non-range predicates (for example `=` / `!=`) are preserved as-is. +/// - If the expression contains `OR`, this function returns the original expression. +/// +/// Tightness rules: +/// - Upper bound: smaller value is tighter; if equal value, exclusive (`<`) is tighter. +/// - Lower bound: larger value is tighter; if equal value, exclusive (`>`) is tighter. +/// +/// Examples: +/// - `a <= 10 AND a < 10` => `a < 10` +/// - `a >= 10 AND a > 10` => `a > 10` +/// - `a < 10 AND a < 5` => `a < 5` +fn simplify_and_bounds(expr: PartitionExpr) -> PartitionExpr { + let Some(collected) = collect_conjunction_bounds(&expr) else { + return expr; + }; + + let CollectedConjunction { + lowers, + uppers, + not_equals: _, + passthrough, + has_conflict: _, + } = collected; + + let mut out = passthrough; + out.extend(lowers.into_iter().map(|(col, lower)| { + PartitionExpr::new( + Operand::Column(col), + if lower.inclusive { + RestrictedOp::GtEq + } else { + RestrictedOp::Gt + }, + Operand::Value(lower.value), + ) + })); + out.extend(uppers.into_iter().map(|(col, upper)| { + PartitionExpr::new( + Operand::Column(col), + if upper.inclusive { + RestrictedOp::LtEq + } else { + RestrictedOp::Lt + }, + Operand::Value(upper.value), + ) + })); + + fold_and_exprs(out).unwrap_or(expr) +} + +/// Flattens an expression into atomic terms when it is a pure conjunction tree. +/// +/// Returns `false` if any `OR` is encountered, signaling caller to skip this +/// simplification path. +fn collect_and_atoms(expr: &PartitionExpr, out: &mut Vec) -> bool { + match expr.op() { + RestrictedOp::And => { + let lhs = match expr.lhs() { + Operand::Expr(lhs) => lhs, + _ => return false, + }; + let rhs = match expr.rhs() { + Operand::Expr(rhs) => rhs, + _ => return false, + }; + collect_and_atoms(lhs, out) && collect_and_atoms(rhs, out) + } + RestrictedOp::Or => false, + _ => { + out.push(expr.clone()); + true + } + } +} + +/// Extracts `(column, op, value)` from a canonicalized atomic expression. +fn atom_col_op_val(expr: &PartitionExpr) -> Option<(String, RestrictedOp, Value)> { + let lhs = expr.lhs(); + let rhs = expr.rhs(); + match (lhs, rhs) { + (Operand::Column(col), Operand::Value(v)) => { + Some((col.clone(), expr.op().clone(), v.clone())) + } + _ => None, + } +} + +/// Collects per-column bounds and passthrough atoms from a pure `AND` tree. +/// +/// Scope and intent: +/// - This helper is shared by [`is_empty_and_conjunction`] and +/// [`simplify_and_bounds`] so both paths interpret conjunction atoms the same +/// way. +/// - It only handles conjunction-only expressions. If any `OR` is present, it +/// returns `None` and lets callers keep their conservative fallback behavior. +/// +/// Behavior: +/// - Tightest lower/upper bounds are recorded per column. +/// - `=` contributes both a lower and an upper bound at the same value. +/// - `!=` and non-range atoms are preserved in `passthrough` for callers that +/// need to rebuild the conjunction. +/// - `has_conflict` is set when atomic constraints already contradict each +/// other (for example `a = 1 AND a <> 1`). +/// +/// Notes: +/// - This helper is intentionally a bit more general than the current +/// `split_partition_expr` contract, which now only feeds range-only +/// conjunctions into the main split path. +fn collect_conjunction_bounds(expr: &PartitionExpr) -> Option { + let mut atoms = Vec::new(); + if !collect_and_atoms(expr, &mut atoms) { + return None; + } + + let mut lowers = BTreeMap::new(); + let mut uppers = BTreeMap::new(); + let mut equals = BTreeMap::new(); + let mut not_equals: BTreeMap> = BTreeMap::new(); + let mut passthrough = Vec::new(); + let mut seen = HashSet::new(); + let mut has_conflict = false; + + for atom in atoms { + let atom = atom.canonicalize(); + let Some((col, op, val)) = atom_col_op_val(&atom) else { + push_unique_expr(&mut passthrough, &mut seen, atom); + continue; + }; + + match op { + RestrictedOp::Lt | RestrictedOp::LtEq => update_upper_bound( + &mut uppers, + col, + UpperBound { + value: val, + inclusive: matches!(op, RestrictedOp::LtEq), + }, + ), + RestrictedOp::Gt | RestrictedOp::GtEq => update_lower_bound( + &mut lowers, + col, + LowerBound { + value: val, + inclusive: matches!(op, RestrictedOp::GtEq), + }, + ), + RestrictedOp::Eq => { + if let Some(existing) = equals.get(&col) + && existing != &val + { + has_conflict = true; + } + if not_equals + .get(&col) + .is_some_and(|excluded| excluded.contains(&val)) + { + has_conflict = true; + } + equals.insert(col.clone(), val.clone()); + update_lower_bound( + &mut lowers, + col.clone(), + LowerBound { + value: val.clone(), + inclusive: true, + }, + ); + update_upper_bound( + &mut uppers, + col, + UpperBound { + value: val, + inclusive: true, + }, + ); + push_unique_expr(&mut passthrough, &mut seen, atom); + } + RestrictedOp::NotEq => { + if equals.get(&col).is_some_and(|eq| eq == &val) { + has_conflict = true; + } + not_equals.entry(col).or_default().insert(val); + push_unique_expr(&mut passthrough, &mut seen, atom); + } + RestrictedOp::And | RestrictedOp::Or => { + push_unique_expr(&mut passthrough, &mut seen, atom); + } + } + } + + Some(CollectedConjunction { + lowers, + uppers, + not_equals, + passthrough, + has_conflict, + }) +} + +fn push_unique_expr(out: &mut Vec, seen: &mut HashSet, expr: PartitionExpr) { + let key = expr.to_string(); + if seen.insert(key) { + out.push(expr); + } +} + +fn update_upper_bound( + uppers: &mut BTreeMap, + col: String, + candidate: UpperBound, +) { + match uppers.get_mut(&col) { + Some(current) => { + if prefer_upper(&candidate, current) { + *current = candidate; + } + } + None => { + uppers.insert(col, candidate); + } + } +} + +fn update_lower_bound( + lowers: &mut BTreeMap, + col: String, + candidate: LowerBound, +) { + match lowers.get_mut(&col) { + Some(current) => { + if prefer_lower(&candidate, current) { + *current = candidate; + } + } + None => { + lowers.insert(col, candidate); + } + } +} + +fn prefer_upper(candidate: &UpperBound, current: &UpperBound) -> bool { + // "Smaller" upper bound is tighter. For equal value, exclusive is tighter. + match candidate.value.partial_cmp(¤t.value) { + Some(std::cmp::Ordering::Less) => true, + Some(std::cmp::Ordering::Equal) => !candidate.inclusive && current.inclusive, + _ => false, + } +} + +fn prefer_lower(candidate: &LowerBound, current: &LowerBound) -> bool { + // "Larger" lower bound is tighter. For equal value, exclusive is tighter. + match candidate.value.partial_cmp(¤t.value) { + Some(std::cmp::Ordering::Greater) => true, + Some(std::cmp::Ordering::Equal) => !candidate.inclusive && current.inclusive, + _ => false, + } +} + +/// Folds a list of expressions into a left-associated AND tree. +/// Returns `None` if the input list is empty. +fn fold_and_exprs(mut exprs: Vec) -> Option { + exprs.drain(..).reduce(|acc, next| acc.and(next)) +} + +#[cfg(test)] +mod tests { + use datatypes::value::{OrderedFloat, Value}; + use store_api::storage::RegionNumber; + + use super::*; + use crate::checker::PartitionChecker; + use crate::expr::col; + use crate::multi_dim::MultiDimPartitionRule; + + fn validate_cut_result_with_checker( + original_rule_exprs: &[PartitionExpr], + replaced_index: usize, + left: &Option, + right: &Option, + partition_columns: Vec, + regions: Vec, + ) -> Result<()> { + ensure!( + replaced_index < original_rule_exprs.len(), + error::UnexpectedSnafu { + err_msg: format!( + "replaced index out of bounds: {replaced_index} >= {}", + original_rule_exprs.len() + ) + } + ); + + let mut exprs = original_rule_exprs.to_vec(); + exprs.remove(replaced_index); + exprs.extend(left.iter().cloned()); + exprs.extend(right.iter().cloned()); + + ensure!( + !exprs.is_empty(), + error::UnexpectedSnafu { + err_msg: "empty rule exprs after split".to_string() + } + ); + + let final_regions = if regions.len() == exprs.len() { + regions + } else { + (0..exprs.len() as RegionNumber).collect() + }; + + let rule = MultiDimPartitionRule::try_new(partition_columns, final_regions, exprs, false)?; + let checker = PartitionChecker::try_new(&rule)?; + checker.check()?; + Ok(()) + } + + #[test] + fn test_split_simple_range() { + // R: a < 10 + let base = col("a").lt(Value::Int64(10)); + // S: a < 5 + let split = col("a").lt(Value::Int64(5)); + let (left, right) = split_partition_expr(base, split).unwrap(); + // left = R AND S = a < 5 + assert_eq!(left.to_string(), "a < 5"); + // right = R AND NOT(S) = a >= 5 AND a < 10 + assert_eq!(right.to_string(), "a >= 5 AND a < 10"); + } + + #[test] + fn test_split_string_interval() { + // R: v > 'm' AND v < 'n' + let base = col("v") + .gt(Value::String("m".into())) + .and(col("v").lt(Value::String("n".into()))); + // S: v < 'm~' + let split = col("v").lt(Value::String("m~".into())); + let (left, right) = split_partition_expr(base, split).unwrap(); + // left = (v > m AND v < n) AND (v < m~) -> v > m AND v < m~ + assert_eq!(left.to_string(), "v > m AND v < m~"); + // right = (v > m AND v < n) AND (v >= m~) -> v >= m~ AND v < n + assert_eq!(right.to_string(), "v >= m~ AND v < n"); + } + + #[test] + fn test_split_numeric_interval_mid_split() { + // R: a > 3 AND a < 10 + let base = col("a") + .gt(Value::Int64(3)) + .and(col("a").lt(Value::Int64(10))); + // S: a < 5 + let split = col("a").lt(Value::Int64(5)); + + let (left, right) = split_partition_expr(base, split).unwrap(); + + // left = (a > 3 AND a < 10) AND (a < 5) -> a > 3 AND a < 5 + assert_eq!(left.to_string(), "a > 3 AND a < 5"); + // right = (a > 3 AND a < 10) AND (a >= 5) -> a >= 5 AND a < 10 + assert_eq!(right.to_string(), "a >= 5 AND a < 10"); + } + + #[test] + fn test_split_base_expr_allows_unrelated_range_columns() { + // R: a > 20 AND b < 20 + let base = col("a") + .gt(Value::Int64(20)) + .and(col("b").lt(Value::Int64(20))); + // S: a < 30 + let split = col("a").lt(Value::Int64(30)); + + let (left, right) = split_partition_expr(base, split).unwrap(); + + // left keeps the unrelated `b < 20` bound while splitting column `a`. + assert_eq!(left.to_string(), "a > 20 AND a < 30 AND b < 20"); + // right also preserves the unrelated column bound. + assert_eq!(right.to_string(), "a >= 30 AND b < 20"); + } + + #[test] + fn test_split_degrade_on_unsupported_type() { + // intentionally excludes boolean from split-able value types. + let base = col("a").eq(Value::Boolean(true)); + let split = col("a").eq(Value::Boolean(true)); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_validate_cut_result_with_checker() { + // Original partition set: a < 10, a >= 10 + let original = vec![ + col("a").lt(Value::Int64(10)), + col("a").gt_eq(Value::Int64(10)), + ]; + let left = Some(col("a").lt(Value::Int64(5))); + let right = Some( + col("a") + .gt_eq(Value::Int64(5)) + .and(col("a").lt(Value::Int64(10))), + ); + + validate_cut_result_with_checker( + &original, + 0, + &left, + &right, + vec!["a".to_string()], + vec![1, 2, 3], + ) + .unwrap(); + } + + #[test] + fn test_split_degrade_on_empty_branch() { + // R: a < 10 + let base = col("a").lt(Value::Int64(10)); + // S: a < 20 + let split = col("a").lt(Value::Int64(20)); + + // right = (a < 10) AND (a >= 20) is unsatisfiable, should degrade. + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::EmptyBranch); + } + + #[test] + fn test_split_rejects_eq_in_base_expr() { + // R: a = 5 falls outside the range-only base_expr contract. + let base = col("a").eq(Value::Int64(5)); + // S: a < 6 remains a valid range split. + let split = col("a").lt(Value::Int64(6)); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_split_degrade_on_discrete_gap_int() { + // R: a < 5 + let base = col("a").lt(Value::Int64(5)); + // S: a <= 4 + let split = col("a").lt_eq(Value::Int64(4)); + + // right = (a < 5) AND (a > 4) has no integer solution, should degrade. + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::EmptyBranch); + } + + #[test] + fn test_split_degrade_on_unsupported_date_type() { + // Date is intentionally excluded from split-supported value types. + let base = col("d").lt(Value::Date(5.into())); + let split = col("d").lt_eq(Value::Date(4.into())); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_split_degrade_on_unsupported_timestamp_type() { + // Timestamp is intentionally excluded from split-supported value types. + let base = col("ts").lt(Value::Timestamp(0.into())); + let split = col("ts").lt_eq(Value::Timestamp(1.into())); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_split_rejects_not_eq_in_split_expr() { + // R: a >= 5 AND a <= 5 + let base = col("a") + .gt_eq(Value::Int64(5)) + .and(col("a").lt_eq(Value::Int64(5))); + // S: a <> 5 falls outside the range-only split_expr contract. + let split = col("a").not_eq(Value::Int64(5)); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_split_rejects_eq_in_split_expr() { + // R: a >= 5 AND a <= 5 + let base = col("a") + .gt_eq(Value::Int64(5)) + .and(col("a").lt_eq(Value::Int64(5))); + // S: a = 5 falls outside the range-only split_expr contract. + let split = col("a").eq(Value::Int64(5)); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_split_degrade_on_uint_one_sided_impossible_upper_bound() { + // R: a < 10 (UInt64 domain) + let base = col("a").lt(Value::UInt64(10)); + // S: a < 0 is still satisfiable by NULL under null-first partition semantics. + // The split keeps a nullable left branch instead of degrading it as empty. + let split = col("a").lt(Value::UInt64(0)); + + let (left, right) = split_partition_expr(base, split).unwrap(); + assert_eq!(left.to_string(), "a < 0"); + assert_eq!(right.to_string(), "a >= 0 AND a < 10"); + } + + #[test] + fn test_split_degrade_on_uint_one_sided_impossible_lower_bound() { + // R: a < 10 (UInt64 domain) + let base = col("a").lt(Value::UInt64(10)); + // S: a > u64::MAX (impossible on UInt64) + let split = col("a").gt(Value::UInt64(u64::MAX)); + + // left = (a < 10) AND (a > u64::MAX) is unsatisfiable on UInt64, should degrade. + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::EmptyBranch); + } + + #[test] + fn test_split_degrade_on_int_one_sided_impossible_upper_bound() { + // R: a < 10 (Int64 domain) + let base = col("a").lt(Value::Int64(10)); + // S: a < i64::MIN is still satisfiable by NULL under null-first partition semantics. + // The split keeps a nullable left branch instead of degrading it as empty. + let split = col("a").lt(Value::Int64(i64::MIN)); + + let (left, right) = split_partition_expr(base, split).unwrap(); + assert_eq!(left.to_string(), format!("a < {}", i64::MIN)); + assert_eq!(right.to_string(), format!("a >= {} AND a < 10", i64::MIN)); + } + + #[test] + fn test_split_degrade_on_int_one_sided_impossible_lower_bound() { + // R: a < 10 (Int64 domain) + let base = col("a").lt(Value::Int64(10)); + // S: a > i64::MAX (impossible on Int64) + let split = col("a").gt(Value::Int64(i64::MAX)); + + // left = (a < 10) AND (a > i64::MAX) is unsatisfiable on Int64, should degrade. + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::EmptyBranch); + } + + #[test] + fn test_split_degrade_on_string_one_sided_impossible_upper_bound() { + // R: s < "z" (String domain) + let base = col("s").lt(Value::String("z".into())); + // S: s < "" is still satisfiable by NULL under null-first partition semantics. + // The split keeps a nullable left branch instead of degrading it as empty. + let split = col("s").lt(Value::String("".into())); + + let (left, right) = split_partition_expr(base, split).unwrap(); + assert_eq!(left.to_string(), "s < "); + assert_eq!(right.to_string(), "s >= AND s < z"); + } + + #[test] + fn test_split_degrade_on_float64_one_sided_impossible_upper_bound() { + // R: a < 10.0 (Float64 domain) + let base = col("a").lt(Value::Float64(OrderedFloat(10.0))); + // S: a < f64::MIN is still satisfiable by NULL under null-first partition semantics. + // The split keeps a nullable left branch instead of degrading it as empty. + let split = col("a").lt(Value::Float64(OrderedFloat(f64::MIN))); + + let (left, right) = split_partition_expr(base, split).unwrap(); + assert_eq!(left.to_string(), format!("a < {}", f64::MIN)); + assert_eq!(right.to_string(), format!("a >= {} AND a < 10", f64::MIN)); + } + + #[test] + fn test_split_degrade_on_float64_one_sided_impossible_lower_bound() { + // R: a < 10.0 (Float64 domain) + let base = col("a").lt(Value::Float64(OrderedFloat(10.0))); + // S: a > f64::MAX (impossible with finite-only float policy) + let split = col("a").gt(Value::Float64(OrderedFloat(f64::MAX))); + + // left = (a < 10.0) AND (a > f64::MAX) is unsatisfiable, should degrade. + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::EmptyBranch); + } + + #[test] + fn test_split_degrade_on_float32_one_sided_impossible_upper_bound() { + // R: a < 10.0f32 (Float32 domain) + let base = col("a").lt(Value::Float32(OrderedFloat(10.0))); + // S: a < f32::MIN is still satisfiable by NULL under null-first partition semantics. + // The split keeps a nullable left branch instead of degrading it as empty. + let split = col("a").lt(Value::Float32(OrderedFloat(f32::MIN))); + + let (left, right) = split_partition_expr(base, split).unwrap(); + assert_eq!(left.to_string(), format!("a < {}", f32::MIN)); + assert_eq!(right.to_string(), format!("a >= {} AND a < 10", f32::MIN)); + } + + #[test] + fn test_split_degrade_on_float32_one_sided_impossible_lower_bound() { + // R: a < 10.0f32 (Float32 domain) + let base = col("a").lt(Value::Float32(OrderedFloat(10.0))); + // S: a > f32::MAX (impossible with finite-only float policy) + let split = col("a").gt(Value::Float32(OrderedFloat(f32::MAX))); + + // left = (a < 10.0f32) AND (a > f32::MAX) is unsatisfiable, should degrade. + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::EmptyBranch); + } + + #[test] + fn test_simplify_same_upper_bound_prefers_strict() { + // a <= 10 AND a < 10 => a < 10 + let expr = col("a") + .lt_eq(Value::Int64(10)) + .and(col("a").lt(Value::Int64(10))); + + let simplified = simplify_and_bounds(expr); + assert_eq!(simplified.to_string(), "a < 10"); + } + + #[test] + fn test_simplify_same_lower_bound_prefers_strict() { + // a >= 10 AND a > 10 => a > 10 + let expr = col("a") + .gt_eq(Value::Int64(10)) + .and(col("a").gt(Value::Int64(10))); + + let simplified = simplify_and_bounds(expr); + assert_eq!(simplified.to_string(), "a > 10"); + } + + #[test] + fn test_negate_split_expr_demorgan_and() { + // expr: (a < 10) AND (a >= 3) + let expr = col("a") + .lt(Value::Int64(10)) + .and(col("a").gt_eq(Value::Int64(3))); + let not_expr = negate_split_expr(&expr).unwrap(); + // NOT(expr) => (a >= 10) OR (a < 3) + assert_eq!(not_expr.to_string(), "a >= 10 OR a < 3"); + } + + #[test] + fn test_negate_split_expr_demorgan_or() { + // expr: (a = 1) OR (a <> 2) + let expr = PartitionExpr::new( + Operand::Expr(col("a").eq(Value::Int64(1))), + RestrictedOp::Or, + Operand::Expr(col("a").not_eq(Value::Int64(2))), + ); + let not_expr = negate_split_expr(&expr).unwrap(); + // NOT(expr) => (a <> 1) AND (a = 2) + assert_eq!(not_expr.to_string(), "a <> 1 AND a = 2"); + } + + #[test] + fn test_negate_split_expr_invalid_and_operand() { + // malformed AND: rhs is a scalar value, not an Expr subtree. + let malformed = PartitionExpr { + lhs: Box::new(Operand::Expr(col("a").lt(Value::Int64(10)))), + op: RestrictedOp::And, + rhs: Box::new(Operand::Value(Value::Int64(1))), + }; + assert!(negate_split_expr(&malformed).is_err()); + } + + #[test] + fn test_validate_supported_expr_value_column_allowed() { + // Canonicalization can flip to column-value; validator must accept value-column input. + let expr = PartitionExpr::new( + Operand::Value(Value::Int64(10)), + RestrictedOp::Lt, + Operand::Column("a".to_string()), + ); + assert!(validate_supported_expr(&expr).is_ok()); + } + + #[test] + fn test_validate_supported_expr_invalid_atomic_shape() { + // column-column atomic comparison is out of shape. + let expr = PartitionExpr::new( + Operand::Column("a".to_string()), + RestrictedOp::Eq, + Operand::Column("b".to_string()), + ); + assert!(validate_supported_expr(&expr).is_err()); + } + + #[test] + fn test_validate_supported_expr_nan_comparison_rejected() { + // NaN cannot be used in any supported comparison predicate. + let expr = col("a").lt(Value::Float64(OrderedFloat(f64::NAN))); + assert!(validate_supported_expr(&expr).is_err()); + } + + #[test] + fn test_validate_supported_expr_infinite_comparison_rejected() { + // Infinity cannot be used in any supported comparison predicate under + // finite-only float policy. + let pos_inf = col("a").gt(Value::Float64(OrderedFloat(f64::INFINITY))); + let neg_inf = col("a").lt(Value::Float32(OrderedFloat(f32::NEG_INFINITY))); + assert!(validate_supported_expr(&pos_inf).is_err()); + assert!(validate_supported_expr(&neg_inf).is_err()); + } + + #[test] + fn test_validate_supported_expr_nan_eq_rejected() { + let expr = col("a").eq(Value::Float64(OrderedFloat(f64::NAN))); + assert!(validate_supported_expr(&expr).is_err()); + } + + #[test] + fn test_validate_supported_expr_infinite_eq_rejected() { + let pos_inf = col("a").eq(Value::Float64(OrderedFloat(f64::INFINITY))); + let neg_inf = col("a").not_eq(Value::Float32(OrderedFloat(f32::NEG_INFINITY))); + assert!(validate_supported_expr(&pos_inf).is_err()); + assert!(validate_supported_expr(&neg_inf).is_err()); + } + + #[test] + fn test_simplify_and_bounds_or_keeps_original() { + // OR tree is intentionally not flattened by AND-only simplifier. + let expr = PartitionExpr::new( + Operand::Expr(col("a").lt(Value::Int64(10))), + RestrictedOp::Or, + Operand::Expr(col("a").gt_eq(Value::Int64(20))), + ); + let simplified = simplify_and_bounds(expr.clone()); + assert_eq!(simplified.to_string(), expr.to_string()); + } + + #[test] + fn test_simplify_and_bounds_keep_stronger_when_weaker_seen_later() { + // upper: stronger bound first, weaker later -> keep stronger (< 5). + let upper = col("a") + .lt(Value::Int64(5)) + .and(col("a").lt(Value::Int64(10))); + assert_eq!(simplify_and_bounds(upper).to_string(), "a < 5"); + + // lower: stronger bound first, weaker later -> keep stronger (> 10). + let lower = col("a") + .gt(Value::Int64(10)) + .and(col("a").gt(Value::Int64(5))); + assert_eq!(simplify_and_bounds(lower).to_string(), "a > 10"); + } + + #[test] + fn test_internal_helpers_uncovered_branches() { + // Empty AND fold should return None. + assert!(fold_and_exprs(vec![]).is_none()); + + // Any OR in tree disables AND-bound simplification path. + let mut out = Vec::new(); + let or_expr = PartitionExpr::new( + Operand::Expr(col("a").lt(Value::Int64(10))), + RestrictedOp::Or, + Operand::Expr(col("a").gt_eq(Value::Int64(20))), + ); + assert!(!collect_and_atoms(&or_expr, &mut out)); + + // value-value atom has no (column, op, value) projection. + let value_value = PartitionExpr::new( + Operand::Value(Value::Int64(1)), + RestrictedOp::Eq, + Operand::Value(Value::Int64(2)), + ); + assert!(atom_col_op_val(&value_value).is_none()); + } + + #[test] + fn test_split_rejects_or_in_base_expr() { + // R: (a < 10) OR (a >= 20 AND a < 30) falls outside the AND-only base_expr contract. + let base = PartitionExpr::new( + Operand::Expr(col("a").lt(Value::Int64(10))), + RestrictedOp::Or, + Operand::Expr( + col("a") + .gt_eq(Value::Int64(20)) + .and(col("a").lt(Value::Int64(30))), + ), + ); + // S: a < 5 + let split = col("a").lt(Value::Int64(5)); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } + + #[test] + fn test_split_rejects_or_in_split_expr() { + // R: a < 10 + let base = col("a").lt(Value::Int64(10)); + // S: (a < 5) OR (a >= 8 AND a < 9) falls outside the atomic split_expr contract. + let split = PartitionExpr::new( + Operand::Expr(col("a").lt(Value::Int64(5))), + RestrictedOp::Or, + Operand::Expr( + col("a") + .gt_eq(Value::Int64(8)) + .and(col("a").lt(Value::Int64(9))), + ), + ); + + let result = split_partition_expr(base, split); + assert_eq!(result.unwrap_err(), ExprSplitDegradeReason::UnsupportedType); + } +} From fe45ae446c035392fc61845e1e0d95c14bcd52f7 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Sat, 28 Mar 2026 07:36:13 +0800 Subject: [PATCH 03/20] perf: optimize promql range functions (#7878) * bench(promql): add range-function benchmark suite * perf(promql): use flat buffers in range function hot loops * perf(promql): reuse quantile scratch buffers --- Cargo.lock | 9 +- src/promql/Cargo.toml | 8 + src/promql/benches/bench_main.rs | 21 ++ src/promql/benches/bench_range_fn.rs | 355 +++++++++++++++++++++ src/promql/src/functions.rs | 94 +++++- src/promql/src/functions/idelta.rs | 65 ++-- src/promql/src/functions/predict_linear.rs | 139 +++++--- src/promql/src/functions/quantile.rs | 80 +++-- 8 files changed, 653 insertions(+), 118 deletions(-) create mode 100644 src/promql/benches/bench_main.rs create mode 100644 src/promql/benches/bench_range_fn.rs diff --git a/Cargo.lock b/Cargo.lock index 676eaf0822..0f3b58b373 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10215,6 +10215,7 @@ dependencies = [ "common-macro", "common-recordbatch", "common-telemetry", + "criterion 0.7.0", "datafusion", "datafusion-common", "datafusion-expr", @@ -11595,9 +11596,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.10" +version = "0.103.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df33b2b81ac578cabaf06b89b0631153a3f416b0a886e8a7a1707fb51abbd1ef" +checksum = "e4a72fe2bcf7a6ac6fd7d0b9e5cb68aeb7d4c0a0271730218b3e92d43b4eb435" dependencies = [ "ring", "rustls-pki-types", @@ -13365,9 +13366,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "tar" -version = "0.4.45" +version = "0.4.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22692a6476a21fa75fdfc11d452fda482af402c008cdbaf3476414e122040973" +checksum = "1d863878d212c87a19c1a610eb53bb01fe12951c0501cf5a0d65f724914a667a" dependencies = [ "filetime", "libc", diff --git a/src/promql/Cargo.toml b/src/promql/Cargo.toml index f93cb8beb9..306563d1ce 100644 --- a/src/promql/Cargo.toml +++ b/src/promql/Cargo.toml @@ -27,4 +27,12 @@ prost.workspace = true snafu.workspace = true [dev-dependencies] +criterion.workspace = true +datafusion-common.workspace = true +datafusion-expr.workspace = true +datatypes.workspace = true tokio.workspace = true + +[[bench]] +name = "bench_main" +harness = false diff --git a/src/promql/benches/bench_main.rs b/src/promql/benches/bench_main.rs new file mode 100644 index 0000000000..2d93887041 --- /dev/null +++ b/src/promql/benches/bench_main.rs @@ -0,0 +1,21 @@ +// 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 criterion::criterion_main; + +mod bench_range_fn; + +criterion_main! { + bench_range_fn::benches +} diff --git a/src/promql/benches/bench_range_fn.rs b/src/promql/benches/bench_range_fn.rs new file mode 100644 index 0000000000..840956b942 --- /dev/null +++ b/src/promql/benches/bench_range_fn.rs @@ -0,0 +1,355 @@ +// 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. + +//! Benchmarks for PromQL range functions. + +use std::sync::Arc; + +use criterion::{BenchmarkId, Criterion, criterion_group}; +use datafusion::arrow::array::{Float64Array, TimestampMillisecondArray}; +use datafusion::physical_plan::ColumnarValue; +use datafusion_common::ScalarValue; +use datafusion_common::config::ConfigOptions; +use datafusion_expr::ScalarFunctionArgs; +use datatypes::arrow::datatypes::{DataType, Field}; +use promql::functions::{Delta, IDelta, Increase, PredictLinear, QuantileOverTime, Rate}; +use promql::range_array::RangeArray; + +fn build_sliding_ranges( + num_points: usize, + window_size: u32, + values: Vec, + eval_offset_ms: i64, +) -> (RangeArray, RangeArray, Arc) { + let step_ms = 1000i64; + let timestamps: Vec = (0..num_points as i64).map(|i| (i + 1) * step_ms).collect(); + + let ts_array = Arc::new(TimestampMillisecondArray::from(timestamps.clone())); + let val_array = Arc::new(Float64Array::from(values)); + + let num_windows = if num_points >= window_size as usize { + num_points - window_size as usize + 1 + } else { + 0 + }; + + let ranges: Vec<(u32, u32)> = (0..num_windows).map(|i| (i as u32, window_size)).collect(); + + let eval_ts: Vec = (0..num_windows) + .map(|i| timestamps[i + window_size as usize - 1] + eval_offset_ms) + .collect(); + let eval_ts_array = Arc::new(TimestampMillisecondArray::from(eval_ts)); + + let ts_range = RangeArray::from_ranges(ts_array, ranges.clone()).unwrap(); + let val_range = RangeArray::from_ranges(val_array, ranges).unwrap(); + + (ts_range, val_range, eval_ts_array) +} + +fn build_monotonic_counter_values(num_points: usize) -> Vec { + let mut current = 0.0; + (0..num_points) + .map(|i| { + current += 1.0 + (i % 7) as f64 * 0.25; + current + }) + .collect() +} + +fn build_resetting_counter_values(num_points: usize) -> Vec { + let mut current = 0.0; + (0..num_points) + .map(|i| { + if i > 0 && i % 37 == 0 { + current = 1.0; + } else { + current += 1.0 + (i % 5) as f64 * 0.5; + } + current + }) + .collect() +} + +fn build_gauge_values(num_points: usize) -> Vec { + (0..num_points) + .map(|i| ((i % 29) as f64 - 14.0) * 1.25 + (i % 3) as f64 * 0.1) + .collect() +} + +fn build_default_values(num_points: usize) -> Vec { + (0..num_points).map(|i| i as f64 * 1.5 + 0.1).collect() +} + +fn make_extrapolated_rate_input( + num_points: usize, + window_size: u32, + values: Vec, + eval_offset_ms: i64, +) -> Vec { + let (ts_range, val_range, eval_ts) = + build_sliding_ranges(num_points, window_size, values, eval_offset_ms); + let range_length = window_size as i64 * 1000; + vec![ + ColumnarValue::Array(Arc::new(ts_range.into_dict())), + ColumnarValue::Array(Arc::new(val_range.into_dict())), + ColumnarValue::Array(eval_ts), + ColumnarValue::Scalar(ScalarValue::Int64(Some(range_length))), + ] +} + +fn make_idelta_input(num_points: usize, window_size: u32) -> Vec { + let (ts_range, val_range, _) = + build_sliding_ranges(num_points, window_size, build_default_values(num_points), 0); + vec![ + ColumnarValue::Array(Arc::new(ts_range.into_dict())), + ColumnarValue::Array(Arc::new(val_range.into_dict())), + ] +} + +fn make_quantile_input(num_points: usize, window_size: u32) -> Vec { + let (ts_range, val_range, _) = + build_sliding_ranges(num_points, window_size, build_default_values(num_points), 0); + vec![ + ColumnarValue::Array(Arc::new(ts_range.into_dict())), + ColumnarValue::Array(Arc::new(val_range.into_dict())), + ColumnarValue::Scalar(ScalarValue::Float64(Some(0.9))), + ] +} + +fn make_predict_linear_input(num_points: usize, window_size: u32) -> Vec { + let (ts_range, val_range, _) = + build_sliding_ranges(num_points, window_size, build_default_values(num_points), 0); + vec![ + ColumnarValue::Array(Arc::new(ts_range.into_dict())), + ColumnarValue::Array(Arc::new(val_range.into_dict())), + // predict 60s into the future + ColumnarValue::Scalar(ScalarValue::Int64(Some(60))), + ] +} + +struct PreparedUdfCall { + args: Vec, + arg_fields: Vec>, + number_rows: usize, + return_field: Arc, + config_options: Arc, +} + +impl PreparedUdfCall { + fn new(args: Vec) -> Self { + let arg_fields = args + .iter() + .enumerate() + .map(|(i, c)| Arc::new(Field::new(format!("c{i}"), c.data_type(), true))) + .collect(); + let number_rows = args + .iter() + .find_map(|c| match c { + ColumnarValue::Array(a) => Some(a.len()), + _ => None, + }) + .unwrap_or(1); + Self { + args, + arg_fields, + number_rows, + return_field: Arc::new(Field::new("out", DataType::Float64, true)), + config_options: Arc::new(ConfigOptions::default()), + } + } +} + +fn invoke_prepared(udf: &datafusion::logical_expr::ScalarUDF, prepared: &PreparedUdfCall) { + udf.invoke_with_args(ScalarFunctionArgs { + args: prepared.args.clone(), + arg_fields: prepared.arg_fields.clone(), + number_rows: prepared.number_rows, + return_field: prepared.return_field.clone(), + config_options: prepared.config_options.clone(), + }) + .unwrap(); +} + +fn bench_range_functions(c: &mut Criterion) { + let mut group = c.benchmark_group("range_fn"); + + // Benchmark parameters: (total_points, window_size) + let params: &[(usize, u32)] = &[ + (1_000, 10), // small series, small window + (10_000, 10), // large series, small window + (10_000, 60), // large series, typical 1-min window at 1s step + (10_000, 360), // large series, wide 6-min window + ]; + + // --- rate (monotonic counter) --- + let rate_udf = Rate::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_extrapolated_rate_input( + n, + w, + build_monotonic_counter_values(n), + 500, + )); + group.bench_with_input( + BenchmarkId::new("rate_counter", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&rate_udf, &prepared)), + ); + } + + // --- rate (periodic resets) --- + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_extrapolated_rate_input( + n, + w, + build_resetting_counter_values(n), + 500, + )); + group.bench_with_input( + BenchmarkId::new("rate_counter_reset", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&rate_udf, &prepared)), + ); + } + + // --- increase (monotonic counter) --- + let increase_udf = Increase::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_extrapolated_rate_input( + n, + w, + build_monotonic_counter_values(n), + 500, + )); + group.bench_with_input( + BenchmarkId::new("increase_counter", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&increase_udf, &prepared)), + ); + } + + // --- increase (periodic resets) --- + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_extrapolated_rate_input( + n, + w, + build_resetting_counter_values(n), + 500, + )); + group.bench_with_input( + BenchmarkId::new("increase_counter_reset", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&increase_udf, &prepared)), + ); + } + + // --- delta (gauge) --- + let delta_udf = Delta::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_extrapolated_rate_input( + n, + w, + build_gauge_values(n), + 500, + )); + group.bench_with_input( + BenchmarkId::new("delta_gauge", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&delta_udf, &prepared)), + ); + } + + // --- idelta --- + let idelta_udf = IDelta::::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_idelta_input(n, w)); + group.bench_with_input( + BenchmarkId::new("idelta", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&idelta_udf, &prepared)), + ); + } + + // --- irate --- + let irate_udf = IDelta::::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_idelta_input(n, w)); + group.bench_with_input( + BenchmarkId::new("irate", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&irate_udf, &prepared)), + ); + } + + // --- quantile_over_time --- + let quantile_udf = QuantileOverTime::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_quantile_input(n, w)); + group.bench_with_input( + BenchmarkId::new("quantile_over_time", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&quantile_udf, &prepared)), + ); + } + + // --- predict_linear --- + let predict_udf = PredictLinear::scalar_udf(); + for &(n, w) in params { + let prepared = PreparedUdfCall::new(make_predict_linear_input(n, w)); + group.bench_with_input( + BenchmarkId::new("predict_linear", format!("n{n}_w{w}")), + &(n, w), + |b, _| b.iter(|| invoke_prepared(&predict_udf, &prepared)), + ); + } + + // --- RangeArray: get vs get_offset_length micro-benchmark --- + // Isolates the overhead of array slicing vs offset/length lookup + for &(n, w) in params { + let step_ms = 1000i64; + let timestamps: Vec = (0..n as i64).map(|i| (i + 1) * step_ms).collect(); + let ts_array = Arc::new(TimestampMillisecondArray::from(timestamps)); + let num_windows = n - w as usize + 1; + let ranges: Vec<(u32, u32)> = (0..num_windows).map(|i| (i as u32, w)).collect(); + let range_array = RangeArray::from_ranges(ts_array, ranges).unwrap(); + + group.bench_with_input( + BenchmarkId::new("range_array_get", format!("n{n}_w{w}")), + &(), + |b, _| { + b.iter(|| { + for i in 0..range_array.len() { + std::hint::black_box(range_array.get(i)); + } + }) + }, + ); + + group.bench_with_input( + BenchmarkId::new("range_array_get_offset_length", format!("n{n}_w{w}")), + &(), + |b, _| { + b.iter(|| { + for i in 0..range_array.len() { + std::hint::black_box(range_array.get_offset_length(i)); + } + }) + }, + ); + } + + group.finish(); +} + +criterion_group!(benches, bench_range_functions); diff --git a/src/promql/src/functions.rs b/src/promql/src/functions.rs index e392d7dcf5..7c7452566a 100644 --- a/src/promql/src/functions.rs +++ b/src/promql/src/functions.rs @@ -31,9 +31,13 @@ pub use aggr_over_time::{ PresentOverTime, StddevOverTime, StdvarOverTime, SumOverTime, }; pub use changes::Changes; -use datafusion::arrow::array::{ArrayRef, Float64Array, TimestampMillisecondArray}; +use datafusion::arrow::array::{ + ArrayRef, DictionaryArray, Float64Array, TimestampMillisecondArray, +}; use datafusion::error::DataFusionError; use datafusion::physical_plan::ColumnarValue; +use datatypes::arrow::array::Array; +use datatypes::arrow::datatypes::Int64Type; pub use deriv::Deriv; pub use double_exponential_smoothing::DoubleExponentialSmoothing; pub use extrapolate_rate::{Delta, Increase, Rate}; @@ -44,6 +48,8 @@ pub use quantile_aggr::{QUANTILE_NAME, quantile_udaf}; pub use resets::Resets; pub use round::Round; +use crate::range_array::RangeArray; + /// Extracts an array from a `ColumnarValue`. /// /// If the `ColumnarValue` is a scalar, it converts it to an array of size 1. @@ -54,6 +60,24 @@ pub(crate) fn extract_array(columnar_value: &ColumnarValue) -> Result Result { + let array = extract_array(columnar_value)?; + let dict = array + .as_any() + .downcast_ref::>() + .ok_or_else(|| { + DataFusionError::Execution(format!( + "expected DictionaryArray, found {}", + array.data_type() + )) + })? + .clone(); + RangeArray::try_new(dict).map_err(DataFusionError::from) +} + /// compensation(Kahan) summation algorithm - a technique for reducing the numerical error /// in floating-point arithmetic. The algorithm also includes the modification ("Neumaier improvement") /// that reduces the numerical error further in cases @@ -78,6 +102,29 @@ pub(crate) fn linear_regression( values: &Float64Array, intercept_time: i64, ) -> (Option, Option) { + linear_regression_slice(times.values(), values, 0, values.len(), intercept_time) +} + +pub(crate) fn linear_regression_slice( + times: &[i64], + values: &Float64Array, + offset: usize, + len: usize, + intercept_time: i64, +) -> (Option, Option) { + linear_regression_slices(times, offset, values, offset, len, intercept_time) +} + +pub(crate) fn linear_regression_slices( + times: &[i64], + time_offset: usize, + values: &Float64Array, + value_offset: usize, + len: usize, + intercept_time: i64, +) -> (Option, Option) { + let raw_values = values.values(); + let has_nulls = values.null_count() > 0; let mut count: f64 = 0.0; let mut sum_x: f64 = 0.0; let mut sum_y: f64 = 0.0; @@ -89,15 +136,18 @@ pub(crate) fn linear_regression( let mut comp_x2: f64 = 0.0; let mut const_y = true; - let init_y: f64 = values.value(0); + let mut init_y = None; - for (i, value) in values.iter().enumerate() { - let time = times.value(i) as f64; - if value.is_none() { + for i in 0..len { + let time_idx = time_offset + i; + let value_idx = value_offset + i; + if has_nulls && values.is_null(value_idx) { continue; } - let value = value.unwrap(); - if const_y && i > 0 && value != init_y { + let value = raw_values[value_idx]; + let time = times[time_idx] as f64; + let initial = init_y.get_or_insert(value); + if const_y && count > 0.0 && value != *initial { const_y = false; } count += 1.0; @@ -113,6 +163,7 @@ pub(crate) fn linear_regression( } if const_y { + let init_y = init_y.unwrap(); if !init_y.is_finite() { return (None, None); } @@ -135,7 +186,14 @@ pub(crate) fn linear_regression( #[cfg(test)] mod test { + use std::sync::Arc; + + use datafusion::physical_plan::ColumnarValue; + use datatypes::arrow::array::Int64Array; + use datatypes::arrow::datatypes::Int64Type; + use super::*; + use crate::range_array::RangeArray; #[test] fn calculate_linear_regression_none() { @@ -253,4 +311,26 @@ mod test { } assert_eq!(sum + c, 2.0) } + + #[test] + fn extract_range_array_rejects_external_dictionary_with_null_keys() { + let keys = Int64Array::from_iter([Some(0), None]); + let values = Arc::new(Float64Array::from_iter([1.0, 2.0])); + let dict = DictionaryArray::::try_new(keys, values).unwrap(); + + let err = extract_range_array(&ColumnarValue::Array(Arc::new(dict))).unwrap_err(); + assert!(err.to_string().contains("Empty range is not expected")); + } + + #[test] + fn extract_range_array_accepts_internal_packed_ranges() { + let values = Arc::new(Float64Array::from_iter([1.0, 2.0, 3.0])); + let range_array = RangeArray::from_ranges(values, [(0, 2), (1, 2)]).unwrap(); + + let extracted = + extract_range_array(&ColumnarValue::Array(Arc::new(range_array.into_dict()))).unwrap(); + + assert_eq!(extracted.get_offset_length(0), Some((0, 2))); + assert_eq!(extracted.get_offset_length(1), Some((1, 2))); + } } diff --git a/src/promql/src/functions/idelta.rs b/src/promql/src/functions/idelta.rs index eeec9a4be9..0772b0bf1e 100644 --- a/src/promql/src/functions/idelta.rs +++ b/src/promql/src/functions/idelta.rs @@ -15,7 +15,7 @@ use std::fmt::Display; use std::sync::Arc; -use datafusion::arrow::array::{Float64Array, TimestampMillisecondArray}; +use datafusion::arrow::array::{Float64Array, Float64Builder, TimestampMillisecondArray}; use datafusion::arrow::datatypes::TimeUnit; use datafusion::common::DataFusionError; use datafusion::logical_expr::{ScalarUDF, Volatility}; @@ -94,49 +94,54 @@ impl IDelta { )), )?; - // calculation - let mut result_array = Vec::with_capacity(ts_range.len()); + let ts_values = ts_range.values(); + let ts_values = ts_values + .as_any() + .downcast_ref::() + .unwrap() + .values(); + + let value_values = value_range.values(); + let value_values = value_values + .as_any() + .downcast_ref::() + .unwrap() + .values(); + + let mut result_builder = Float64Builder::with_capacity(ts_range.len()); for index in 0..ts_range.len() { - let timestamps = ts_range.get(index).unwrap(); - let timestamps = timestamps - .as_any() - .downcast_ref::() - .unwrap() - .values(); - - let values = value_range.get(index).unwrap(); - let values = values - .as_any() - .downcast_ref::() - .unwrap() - .values(); + let (ts_offset, len) = ts_range.get_offset_length(index).unwrap(); + let (value_offset, value_len) = value_range.get_offset_length(index).unwrap(); error::ensure( - timestamps.len() == values.len(), + len == value_len, DataFusionError::Execution(format!( "{}: input arrays should have the same length, found {} and {}", Self::name(), - timestamps.len(), - values.len() + len, + value_len )), )?; - - let len = timestamps.len(); if len < 2 { - result_array.push(None); + result_builder.append_null(); continue; } - // if is delta + let last_offset = ts_offset + len - 1; + let prev_offset = last_offset - 1; + let sampled_interval = + (ts_values[last_offset] - ts_values[prev_offset]) as f64 / 1000.0; + + let last_value_offset = value_offset + len - 1; + let prev_value_offset = last_value_offset - 1; + let last_value = value_values[last_value_offset]; + let prev_value = value_values[prev_value_offset]; + if !IS_RATE { - result_array.push(Some(values[len - 1] - values[len - 2])); + result_builder.append_value(last_value - prev_value); continue; } - // else is rate - let sampled_interval = (timestamps[len - 1] - timestamps[len - 2]) as f64 / 1000.0; - let last_value = values[len - 1]; - let prev_value = values[len - 2]; let result_value = if last_value < prev_value { // counter reset last_value @@ -144,10 +149,10 @@ impl IDelta { last_value - prev_value }; - result_array.push(Some(result_value / sampled_interval as f64)); + result_builder.append_value(result_value / sampled_interval); } - let result = ColumnarValue::Array(Arc::new(Float64Array::from_iter(result_array))); + let result = ColumnarValue::Array(Arc::new(result_builder.finish())); Ok(result) } } diff --git a/src/promql/src/functions/predict_linear.rs b/src/promql/src/functions/predict_linear.rs index 09a46ed48f..dc49ec5d9f 100644 --- a/src/promql/src/functions/predict_linear.rs +++ b/src/promql/src/functions/predict_linear.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use datafusion::arrow::array::{Float64Array, TimestampMillisecondArray}; +use datafusion::arrow::array::{Float64Array, Float64Builder, TimestampMillisecondArray}; use datafusion::arrow::datatypes::TimeUnit; use datafusion::common::DataFusionError; use datafusion::logical_expr::{ScalarUDF, Volatility}; @@ -28,7 +28,7 @@ use datatypes::arrow::array::Array; use datatypes::arrow::datatypes::DataType; use crate::error; -use crate::functions::{extract_array, linear_regression}; +use crate::functions::{extract_range_array, linear_regression_slices}; use crate::range_array::RangeArray; pub struct PredictLinear; @@ -62,12 +62,10 @@ impl PredictLinear { DataFusionError::Plan("prom_predict_linear function should have 3 inputs".to_string()), )?; - let ts_array = extract_array(&input[0])?; - let value_array = extract_array(&input[1])?; let t_col = &input[2]; - let ts_range: RangeArray = RangeArray::try_new(ts_array.to_data().into())?; - let value_range: RangeArray = RangeArray::try_new(value_array.to_data().into())?; + let ts_range = extract_range_array(&input[0])?; + let value_range = extract_range_array(&input[1])?; error::ensure( ts_range.len() == value_range.len(), DataFusionError::Execution(format!( @@ -130,74 +128,85 @@ impl PredictLinear { Box::new(t_array.iter()) } }; - let mut result_array = Vec::with_capacity(ts_range.len()); + let all_timestamps = ts_range + .values() + .as_any() + .downcast_ref::() + .unwrap() + .values(); + let all_values = value_range + .values() + .as_any() + .downcast_ref::() + .unwrap(); + let mut result_builder = Float64Builder::with_capacity(ts_range.len()); for (index, t) in t_iter.enumerate() { - let (timestamps, values) = get_ts_values(&ts_range, &value_range, index, Self::name())?; - let ret = predict_linear_impl(×tamps, &values, t.unwrap()); - result_array.push(ret); + match predict_linear_impl( + &ts_range, + &value_range, + all_timestamps, + all_values, + index, + t.unwrap(), + Self::name(), + )? { + Some(value) => result_builder.append_value(value), + None => result_builder.append_null(), + } } - let result = ColumnarValue::Array(Arc::new(Float64Array::from_iter(result_array))); + let result = ColumnarValue::Array(Arc::new(result_builder.finish())); Ok(result) } } -fn get_ts_values( +fn predict_linear_impl( ts_range: &RangeArray, value_range: &RangeArray, + all_timestamps: &[i64], + all_values: &Float64Array, index: usize, + t: i64, func_name: &str, -) -> Result<(TimestampMillisecondArray, Float64Array), DataFusionError> { - let timestamps = ts_range - .get(index) - .unwrap() - .as_any() - .downcast_ref::() - .unwrap() - .clone(); - let values = value_range - .get(index) - .unwrap() - .as_any() - .downcast_ref::() - .unwrap() - .clone(); +) -> Result, DataFusionError> { + let (ts_offset, ts_len) = ts_range.get_offset_length(index).unwrap(); + let (value_offset, value_len) = value_range.get_offset_length(index).unwrap(); error::ensure( - timestamps.len() == values.len(), + ts_len == value_len, DataFusionError::Execution(format!( "{}: time and value arrays in a group should have the same length, found {} and {}", - func_name, - timestamps.len(), - values.len() + func_name, ts_len, value_len )), )?; - Ok((timestamps, values)) -} - -fn predict_linear_impl( - timestamps: &TimestampMillisecondArray, - values: &Float64Array, - t: i64, -) -> Option { - if timestamps.len() < 2 { - return None; + if ts_len < 2 { + return Ok(None); } // last timestamp is evaluation timestamp - let evaluate_ts = timestamps.value(timestamps.len() - 1); - let (slope, intercept) = linear_regression(timestamps, values, evaluate_ts); + let evaluate_ts = all_timestamps[ts_offset + ts_len - 1]; + let (slope, intercept) = linear_regression_slices( + all_timestamps, + ts_offset, + all_values, + value_offset, + value_len, + evaluate_ts, + ); if slope.is_none() || intercept.is_none() { - return None; + return Ok(None); } - Some(slope.unwrap() * t as f64 + intercept.unwrap()) + Ok(Some(slope.unwrap() * t as f64 + intercept.unwrap())) } #[cfg(test)] mod test { use std::vec; + use datafusion::arrow::array::{DictionaryArray, Int64Array}; + use datatypes::arrow::datatypes::Int64Type; + use super::*; use crate::functions::test_util::simple_range_udf_runner; @@ -304,4 +313,44 @@ mod test { vec![Some(82765.9090909091)], ); } + + #[test] + fn calculate_predict_linear_with_misaligned_offsets() { + let ts_values = Arc::new(TimestampMillisecondArray::from_iter( + [0i64, 1000, 2000, 3000].into_iter().map(Some), + )); + let value_values = Arc::new(Float64Array::from_iter([10.0, 20.0, 30.0])); + let ts_array = RangeArray::from_ranges(ts_values, [(1, 3)]).unwrap(); + let value_array = RangeArray::from_ranges(value_values, [(0, 3)]).unwrap(); + + simple_range_udf_runner( + PredictLinear::scalar_udf(), + ts_array, + value_array, + vec![ScalarValue::Int64(Some(0))], + vec![Some(30.0)], + ); + } + + #[test] + fn predict_linear_rejects_external_dictionary_with_null_keys() { + let ts_values = Arc::new(TimestampMillisecondArray::from_iter( + [0i64, 1000].into_iter().map(Some), + )); + let ts_keys = Int64Array::from_iter([Some(0), None]); + let ts_dict = DictionaryArray::::try_new(ts_keys, ts_values).unwrap(); + + let value_values = Arc::new(Float64Array::from_iter([1.0, 2.0])); + let value_keys = Int64Array::from_iter([Some(0), Some(1)]); + let value_dict = DictionaryArray::::try_new(value_keys, value_values).unwrap(); + + let err = PredictLinear::predict_linear(&[ + ColumnarValue::Array(Arc::new(ts_dict)), + ColumnarValue::Array(Arc::new(value_dict)), + ColumnarValue::Scalar(ScalarValue::Int64(Some(0))), + ]) + .unwrap_err(); + + assert!(err.to_string().contains("Empty range is not expected")); + } } diff --git a/src/promql/src/functions/quantile.rs b/src/promql/src/functions/quantile.rs index f368d5908c..93fc632d68 100644 --- a/src/promql/src/functions/quantile.rs +++ b/src/promql/src/functions/quantile.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use datafusion::arrow::array::Float64Array; +use datafusion::arrow::array::{Float64Array, Float64Builder}; use datafusion::arrow::datatypes::TimeUnit; use datafusion::common::DataFusionError; use datafusion::logical_expr::{ScalarUDF, Volatility}; @@ -93,8 +93,14 @@ impl QuantileOverTime { )), )?; - // calculation - let mut result_array = Vec::with_capacity(ts_range.len()); + let all_values = value_range + .values() + .as_any() + .downcast_ref::() + .unwrap() + .values(); + let mut result_builder = Float64Builder::with_capacity(ts_range.len()); + let mut scratch = Vec::new(); match quantile_col { ColumnarValue::Scalar(quantile_scalar) => { @@ -107,25 +113,26 @@ impl QuantileOverTime { }; for index in 0..ts_range.len() { - let timestamps = ts_range.get(index).unwrap(); - let values = value_range.get(index).unwrap(); - let values = values - .as_any() - .downcast_ref::() - .unwrap() - .values(); + let (_, ts_len) = ts_range.get_offset_length(index).unwrap(); + let (value_offset, value_len) = value_range.get_offset_length(index).unwrap(); error::ensure( - timestamps.len() == values.len(), + ts_len == value_len, DataFusionError::Execution(format!( "{}: time and value arrays in a group should have the same length, found {} and {}", Self::name(), - timestamps.len(), - values.len() + ts_len, + value_len )), )?; - let result = quantile_impl(values, quantile); - result_array.push(result); + match quantile_with_scratch( + &all_values[value_offset..value_offset + value_len], + quantile, + &mut scratch, + ) { + Some(value) => result_builder.append_value(value), + None => result_builder.append_null(), + } } } ColumnarValue::Array(quantile_array) => { @@ -150,20 +157,15 @@ impl QuantileOverTime { )), )?; for index in 0..ts_range.len() { - let timestamps = ts_range.get(index).unwrap(); - let values = value_range.get(index).unwrap(); - let values = values - .as_any() - .downcast_ref::() - .unwrap() - .values(); + let (_, ts_len) = ts_range.get_offset_length(index).unwrap(); + let (value_offset, value_len) = value_range.get_offset_length(index).unwrap(); error::ensure( - timestamps.len() == values.len(), + ts_len == value_len, DataFusionError::Execution(format!( "{}: time and value arrays in a group should have the same length, found {} and {}", Self::name(), - timestamps.len(), - values.len() + ts_len, + value_len )), )?; let quantile = if quantile_array.is_null(index) { @@ -171,19 +173,32 @@ impl QuantileOverTime { } else { quantile_array.value(index) }; - let result = quantile_impl(values, quantile); - result_array.push(result); + match quantile_with_scratch( + &all_values[value_offset..value_offset + value_len], + quantile, + &mut scratch, + ) { + Some(value) => result_builder.append_value(value), + None => result_builder.append_null(), + } } } } - let result = ColumnarValue::Array(Arc::new(Float64Array::from_iter(result_array))); + let result = ColumnarValue::Array(Arc::new(result_builder.finish())); Ok(result) } } /// Refer to pub(crate) fn quantile_impl(values: &[f64], quantile: f64) -> Option { + let mut scratch = Vec::new(); + quantile_with_scratch(values, quantile, &mut scratch) +} + +/// Same as [quantile_impl] but reuses a caller-provided scratch buffer to avoid +/// per-call allocation. +fn quantile_with_scratch(values: &[f64], quantile: f64, scratch: &mut Vec) -> Option { if quantile.is_nan() || values.is_empty() { return Some(f64::NAN); } @@ -194,17 +209,18 @@ pub(crate) fn quantile_impl(values: &[f64], quantile: f64) -> Option { return Some(f64::INFINITY); } - let mut values = values.to_vec(); - values.sort_unstable_by(f64::total_cmp); + scratch.clear(); + scratch.extend_from_slice(values); + scratch.sort_unstable_by(f64::total_cmp); - let length = values.len(); + let length = scratch.len(); let rank = quantile * (length - 1) as f64; let lower_index = 0.max(rank.floor() as usize); let upper_index = (length - 1).min(lower_index + 1); let weight = rank - rank.floor(); - let result = values[lower_index] * (1.0 - weight) + values[upper_index] * weight; + let result = scratch[lower_index] * (1.0 - weight) + scratch[upper_index] * weight; Some(result) } From 8e7e4a91d29c4c8cb17788a84d7c61a4fd9d74c1 Mon Sep 17 00:00:00 2001 From: cui Date: Mon, 30 Mar 2026 11:22:04 +0800 Subject: [PATCH 04/20] fix(datatypes): correct ConstantVector rhs comparison in vector equality (#7866) * fix(datatypes): compare ConstantVector rhs inner in vector equality When either operand is a ConstantVector, the recursive equal() call must compare lhs.inner() against rhs.inner(). The second argument incorrectly used lhs twice, breaking equality when only the rhs was constant. Signed-off-by: Weixie Cui * fix: review Signed-off-by: Weixie Cui --------- Signed-off-by: Weixie Cui --- src/datatypes/src/vectors/eq.rs | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/src/datatypes/src/vectors/eq.rs b/src/datatypes/src/vectors/eq.rs index 372b3ceee4..e8e942d3ff 100644 --- a/src/datatypes/src/vectors/eq.rs +++ b/src/datatypes/src/vectors/eq.rs @@ -67,7 +67,7 @@ fn equal(lhs: &dyn Vector, rhs: &dyn Vector) -> bool { .downcast_ref::() .unwrap() .inner(), - &**lhs + &**rhs .as_any() .downcast_ref::() .unwrap() @@ -259,6 +259,22 @@ mod tests { ]))); } + // Regression: second arm must downcast `rhs` (was `lhs`), or same-length ConstantVectors + // with different inners compare equal. + #[test] + fn test_constant_vector_eq_compares_both_inners() { + assert_vector_ref_ne( + Arc::new(ConstantVector::new( + Arc::new(BooleanVector::from(vec![true])), + 5, + )), + Arc::new(ConstantVector::new( + Arc::new(BooleanVector::from(vec![false])), + 5, + )), + ); + } + #[test] fn test_vector_ne() { assert_vector_ref_ne( From 3904df5397af595749dbce7334295b121750cef8 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Sun, 29 Mar 2026 20:32:49 -0700 Subject: [PATCH 05/20] chore: refine track memory metrics semantics (#7874) Signed-off-by: jeremyhi --- src/common/memory-manager/src/guard.rs | 2 +- src/common/memory-manager/src/lib.rs | 2 +- src/common/memory-manager/src/manager.rs | 5 +- src/common/recordbatch/src/lib.rs | 128 +++++++++++++++++++-- src/mito2/src/compaction/memory_manager.rs | 2 +- src/mito2/src/engine.rs | 9 +- src/mito2/src/metrics.rs | 9 +- src/servers/src/metrics.rs | 2 +- src/servers/src/request_memory_metrics.rs | 2 +- 9 files changed, 140 insertions(+), 21 deletions(-) diff --git a/src/common/memory-manager/src/guard.rs b/src/common/memory-manager/src/guard.rs index ad3111581b..784b72830d 100644 --- a/src/common/memory-manager/src/guard.rs +++ b/src/common/memory-manager/src/guard.rs @@ -172,7 +172,7 @@ impl MemoryGuard { true } Err(TryAcquireError::NoPermits) | Err(TryAcquireError::Closed) => { - quota.metrics.inc_rejected("try_acquire_additional"); + quota.metrics.inc_exhausted("try_acquire_additional"); false } } diff --git a/src/common/memory-manager/src/lib.rs b/src/common/memory-manager/src/lib.rs index b1d858ef89..983c6ca524 100644 --- a/src/common/memory-manager/src/lib.rs +++ b/src/common/memory-manager/src/lib.rs @@ -45,5 +45,5 @@ impl MemoryMetrics for NoOpMetrics { fn set_in_use(&self, _: i64) {} #[inline(always)] - fn inc_rejected(&self, _: &str) {} + fn inc_exhausted(&self, _: &str) {} } diff --git a/src/common/memory-manager/src/manager.rs b/src/common/memory-manager/src/manager.rs index 8cca5f220c..282f51e315 100644 --- a/src/common/memory-manager/src/manager.rs +++ b/src/common/memory-manager/src/manager.rs @@ -29,7 +29,8 @@ use crate::policy::OnExhaustedPolicy; pub trait MemoryMetrics: Clone + Send + Sync + 'static { fn set_limit(&self, bytes: i64); fn set_in_use(&self, bytes: i64); - fn inc_rejected(&self, reason: &str); + /// Record that immediate memory acquisition failed due to exhausted quota. + fn inc_exhausted(&self, reason: &str); } /// Generic memory manager for quota-controlled operations. @@ -171,7 +172,7 @@ impl MemoryManager { Some(MemoryGuard::limited(quota.clone(), permit)) } Err(TryAcquireError::NoPermits) | Err(TryAcquireError::Closed) => { - quota.metrics.inc_rejected("try_acquire"); + quota.metrics.inc_exhausted("try_acquire"); None } } diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 0a2d697407..d84e9e9d26 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -437,7 +437,8 @@ impl fmt::Debug for QueryMemoryTracker { .field("limit", &self.limit()) .field("on_exhausted_policy", &self.on_exhausted_policy) .field("on_update", &self.metrics.has_on_update()) - .field("on_reject", &self.metrics.has_on_reject()) + .field("on_exhausted", &self.metrics.has_on_exhausted()) + .field("on_rejected", &self.metrics.has_on_rejected()) .finish() } } @@ -452,6 +453,7 @@ impl QueryMemoryTracker { limit, on_exhausted_policy, on_update: None, + on_exhausted: None, on_reject: None, } } @@ -489,6 +491,10 @@ impl QueryMemoryTracker { ); error::ExceedMemoryLimitSnafu { msg }.build() } + + fn inc_rejected(&self) { + self.metrics.inc_rejected(); + } } /// Builder for constructing a [`QueryMemoryTracker`] with optional callbacks. @@ -496,6 +502,7 @@ pub struct QueryMemoryTrackerBuilder { limit: usize, on_exhausted_policy: OnExhaustedPolicy, on_update: Option, + on_exhausted: Option, on_reject: Option, } @@ -514,11 +521,21 @@ impl QueryMemoryTrackerBuilder { self } - /// Set a callback to be called when memory allocation is rejected. + /// Set a callback to be called when memory is unavailable for immediate acquisition. /// /// # Note - /// This is only called when `track()` fails due to exceeding the limit. + /// This is called when the non-blocking allocation fast path fails. + /// Requests using `OnExhaustedPolicy::Wait` may still succeed after waiting. /// It is never called when `limit == 0` (unlimited mode). + pub fn on_exhausted(mut self, on_exhausted: F) -> Self + where + F: Fn() + Send + Sync + 'static, + { + self.on_exhausted = Some(Arc::new(on_exhausted)); + self + } + + /// Set a callback to be called when the request ultimately fails due to memory pressure. pub fn on_reject(mut self, on_reject: F) -> Self where F: Fn() + Send + Sync + 'static, @@ -529,7 +546,7 @@ impl QueryMemoryTrackerBuilder { /// Build a [`QueryMemoryTracker`] from this builder. pub fn build(self) -> QueryMemoryTracker { - let metrics = CallbackMemoryMetrics::new(self.on_update, self.on_reject); + let metrics = CallbackMemoryMetrics::new(self.on_update, self.on_exhausted, self.on_reject); let manager = MemoryManager::with_granularity( self.limit as u64, PermitGranularity::Kilobyte, @@ -553,6 +570,10 @@ struct StreamMemoryTracker { type MemoryAcquireResult = std::result::Result<(), common_memory_manager::Error>; impl StreamMemoryTracker { + fn inc_rejected(&self) { + self.tracker.inc_rejected(); + } + fn try_track(&mut self, additional: usize) -> Result<()> { if self.guard.try_acquire_additional(additional as u64) { self.tracked_bytes = self.tracked_bytes.saturating_add(additional); @@ -613,18 +634,25 @@ struct CallbackMemoryMetrics { } type UpdateCallback = Arc; -type RejectCallback = Arc; +type UnitCallback = Arc; +type RejectCallback = UnitCallback; struct CallbackMemoryMetricsInner { on_update: Option, + on_exhausted: Option, on_reject: Option, } impl CallbackMemoryMetrics { - fn new(on_update: Option, on_reject: Option) -> Self { + fn new( + on_update: Option, + on_exhausted: Option, + on_reject: Option, + ) -> Self { Self { inner: Arc::new(CallbackMemoryMetricsInner { on_update, + on_exhausted, on_reject, }), } @@ -634,9 +662,19 @@ impl CallbackMemoryMetrics { self.inner.on_update.is_some() } - fn has_on_reject(&self) -> bool { + fn has_on_exhausted(&self) -> bool { + self.inner.on_exhausted.is_some() + } + + fn has_on_rejected(&self) -> bool { self.inner.on_reject.is_some() } + + fn inc_rejected(&self) { + if let Some(callback) = &self.inner.on_reject { + callback(); + } + } } impl MemoryMetrics for CallbackMemoryMetrics { @@ -648,8 +686,8 @@ impl MemoryMetrics for CallbackMemoryMetrics { } } - fn inc_rejected(&self, _: &str) { - if let Some(callback) = &self.inner.on_reject { + fn inc_exhausted(&self, _: &str) { + if let Some(callback) = &self.inner.on_exhausted { callback(); } } @@ -712,7 +750,10 @@ impl MemoryTrackedStream { Poll::Ready((tracker, batch, additional, result)) => { let output = match result { Ok(()) => Ok(batch), - Err(error) => Err(tracker.wait_error(additional, error)), + Err(error) => { + tracker.inc_rejected(); + Err(tracker.wait_error(additional, error)) + } }; self.waiting = None; self.tracker = Some(tracker); @@ -732,7 +773,10 @@ impl MemoryTrackedStream { if let Err(error) = tracker.try_track(additional) { match tracker.tracker.on_exhausted_policy { - OnExhaustedPolicy::Fail => return Poll::Ready(Some(Err(error))), + OnExhaustedPolicy::Fail => { + tracker.inc_rejected(); + return Poll::Ready(Some(Err(error))); + } // `Wait` is a deliberate tradeoff: the batch has already been materialized, so we // keep it in memory while waiting for quota instead of failing immediately. Under // contention, real memory usage can therefore exceed `scan_memory_limit` by up to @@ -786,6 +830,7 @@ impl RecordBatchStream for MemoryTrackedStream { #[cfg(test)] mod tests { use std::sync::Arc; + use std::sync::atomic::{AtomicUsize, Ordering}; use std::time::Duration; use common_memory_manager::{OnExhaustedPolicy, PermitGranularity}; @@ -988,12 +1033,22 @@ mod tests { #[tokio::test] async fn test_memory_tracked_stream_waits_for_capacity() { + let exhausted = Arc::new(AtomicUsize::new(0)); + let rejected = Arc::new(AtomicUsize::new(0)); + let exhausted_counter = exhausted.clone(); + let rejected_counter = rejected.clone(); let tracker = QueryMemoryTracker::builder( MB, OnExhaustedPolicy::Wait { timeout: Duration::from_millis(200), }, ) + .on_exhausted(move || { + exhausted_counter.fetch_add(1, Ordering::Relaxed); + }) + .on_reject(move || { + rejected_counter.fetch_add(1, Ordering::Relaxed); + }) .build(); let batch = large_string_batch(700 * 1024); let expected_bytes = aligned_tracked_bytes(batch.buffer_memory_size()); @@ -1025,16 +1080,28 @@ mod tests { drop(stream1); let second = waiter.await.unwrap().unwrap(); assert_eq!(second.num_rows(), 1); + assert_eq!(exhausted.load(Ordering::Relaxed), 1); + assert_eq!(rejected.load(Ordering::Relaxed), 0); } #[tokio::test] async fn test_memory_tracked_stream_wait_times_out() { + let exhausted = Arc::new(AtomicUsize::new(0)); + let rejected = Arc::new(AtomicUsize::new(0)); + let exhausted_counter = exhausted.clone(); + let rejected_counter = rejected.clone(); let tracker = QueryMemoryTracker::builder( MB, OnExhaustedPolicy::Wait { timeout: Duration::from_millis(50), }, ) + .on_exhausted(move || { + exhausted_counter.fetch_add(1, Ordering::Relaxed); + }) + .on_reject(move || { + rejected_counter.fetch_add(1, Ordering::Relaxed); + }) .build(); let batch = large_string_batch(700 * 1024); @@ -1058,5 +1125,44 @@ mod tests { .unwrap(); let error = result.unwrap().unwrap_err(); assert!(error.to_string().contains("timed out waiting")); + assert_eq!(exhausted.load(Ordering::Relaxed), 1); + assert_eq!(rejected.load(Ordering::Relaxed), 1); + } + + #[tokio::test] + async fn test_memory_tracked_stream_fail_policy_rejects_immediately() { + let exhausted = Arc::new(AtomicUsize::new(0)); + let rejected = Arc::new(AtomicUsize::new(0)); + let exhausted_counter = exhausted.clone(); + let rejected_counter = rejected.clone(); + let tracker = QueryMemoryTracker::builder(MB, OnExhaustedPolicy::Fail) + .on_exhausted(move || { + exhausted_counter.fetch_add(1, Ordering::Relaxed); + }) + .on_reject(move || { + rejected_counter.fetch_add(1, Ordering::Relaxed); + }) + .build(); + let batch = large_string_batch(700 * 1024); + + let mut stream1 = MemoryTrackedStream::new( + RecordBatches::try_new(batch.schema.clone(), vec![batch.clone()]) + .unwrap() + .as_stream(), + tracker.clone(), + ); + let first = stream1.next().await.unwrap().unwrap(); + assert_eq!(first.num_rows(), 1); + + let mut stream2 = MemoryTrackedStream::new( + RecordBatches::try_new(batch.schema.clone(), vec![batch]) + .unwrap() + .as_stream(), + tracker, + ); + let result = stream2.next().await.unwrap(); + assert!(result.is_err()); + assert_eq!(exhausted.load(Ordering::Relaxed), 1); + assert_eq!(rejected.load(Ordering::Relaxed), 1); } } diff --git a/src/mito2/src/compaction/memory_manager.rs b/src/mito2/src/compaction/memory_manager.rs index 8cbb5d293a..94b0779254 100644 --- a/src/mito2/src/compaction/memory_manager.rs +++ b/src/mito2/src/compaction/memory_manager.rs @@ -31,7 +31,7 @@ impl MemoryMetrics for CompactionMemoryMetrics { COMPACTION_MEMORY_IN_USE.set(bytes); } - fn inc_rejected(&self, reason: &str) { + fn inc_exhausted(&self, reason: &str) { COMPACTION_MEMORY_REJECTED .with_label_values(&[reason]) .inc(); diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index fbafe1da67..d1c30c3ff6 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -138,7 +138,8 @@ use crate::gc::GcLimiterRef; use crate::manifest::action::RegionEdit; use crate::memtable::MemtableStats; use crate::metrics::{ - HANDLE_REQUEST_ELAPSED, SCAN_MEMORY_USAGE_BYTES, SCAN_REQUESTS_REJECTED_TOTAL, + HANDLE_REQUEST_ELAPSED, SCAN_MEMORY_EXHAUSTED_TOTAL, SCAN_MEMORY_USAGE_BYTES, + SCAN_REQUESTS_REJECTED_TOTAL, }; use crate::read::scan_region::{ScanRegion, Scanner}; use crate::read::stream::ScanBatchStream; @@ -231,6 +232,9 @@ impl<'a, S: LogStore> MitoEngineBuilder<'a, S> { .on_update(|usage| { SCAN_MEMORY_USAGE_BYTES.set(usage as i64); }) + .on_exhausted(|| { + SCAN_MEMORY_EXHAUSTED_TOTAL.inc(); + }) .on_reject(|| { SCAN_REQUESTS_REJECTED_TOTAL.inc(); }) @@ -1380,6 +1384,9 @@ impl MitoEngine { .on_update(|usage| { SCAN_MEMORY_USAGE_BYTES.set(usage as i64); }) + .on_exhausted(|| { + SCAN_MEMORY_EXHAUSTED_TOTAL.inc(); + }) .on_reject(|| { SCAN_REQUESTS_REJECTED_TOTAL.inc(); }) diff --git a/src/mito2/src/metrics.rs b/src/mito2/src/metrics.rs index c0537567f9..30a7ac765c 100644 --- a/src/mito2/src/metrics.rs +++ b/src/mito2/src/metrics.rs @@ -244,10 +244,15 @@ lazy_static! { "greptime_mito_scan_memory_usage_bytes", "current scan memory usage in bytes" ).unwrap(); - /// Counter of rejected scan requests due to memory limit. + /// Counter of scan allocation attempts that could not acquire memory immediately. + pub static ref SCAN_MEMORY_EXHAUSTED_TOTAL: IntCounter = register_int_counter!( + "greptime_mito_scan_memory_exhausted_total", + "total number of times scan memory was unavailable for immediate acquisition" + ).unwrap(); + /// Counter of scan requests that ultimately failed due to memory pressure. pub static ref SCAN_REQUESTS_REJECTED_TOTAL: IntCounter = register_int_counter!( "greptime_mito_scan_requests_rejected_total", - "total number of scan requests rejected due to memory limit" + "total number of scan requests that ultimately failed due to memory limit" ).unwrap(); /// Gauge for active file range builders in the pruner. pub static ref PRUNER_ACTIVE_BUILDERS: IntGauge = register_int_gauge!( diff --git a/src/servers/src/metrics.rs b/src/servers/src/metrics.rs index 37f923b73d..e3bff7fdbc 100644 --- a/src/servers/src/metrics.rs +++ b/src/servers/src/metrics.rs @@ -361,7 +361,7 @@ lazy_static! { "maximum bytes allowed for all concurrent request bodies and messages" ).unwrap(); - /// Total number of rejected requests due to memory exhaustion. + /// Total number of requests rejected due to memory exhaustion. pub static ref REQUEST_MEMORY_REJECTED: IntCounterVec = register_int_counter_vec!( "greptime_servers_request_memory_rejected_total", "number of requests rejected due to memory limit", diff --git a/src/servers/src/request_memory_metrics.rs b/src/servers/src/request_memory_metrics.rs index 4298830f18..68f52816f4 100644 --- a/src/servers/src/request_memory_metrics.rs +++ b/src/servers/src/request_memory_metrics.rs @@ -34,7 +34,7 @@ impl MemoryMetrics for RequestMemoryMetrics { REQUEST_MEMORY_IN_USE.set(bytes); } - fn inc_rejected(&self, reason: &str) { + fn inc_exhausted(&self, reason: &str) { REQUEST_MEMORY_REJECTED.with_label_values(&[reason]).inc(); } } From 92e2d71f484771e34f02bb5a957131fb52cccbc1 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Mon, 30 Mar 2026 12:24:20 +0800 Subject: [PATCH 06/20] feat: implement prefilter framework and primary key prefilter (#7862) * feat: prefilter basic framework Signed-off-by: evenyag * refactor: move arguments to RowGroupBuildContext Signed-off-by: evenyag * refactor: skip prefiltered exprs in FlatPruneReader Signed-off-by: evenyag * refactor: remove unused functions Signed-off-by: evenyag * chore: update comment Signed-off-by: evenyag * feat: handle partition columns in prefilter Signed-off-by: evenyag * chore: fix clippy Signed-off-by: evenyag * fix: apply prefiltered selection by and_then Signed-off-by: evenyag * chore: fix clippy Signed-off-by: evenyag * fix: handle last row cache Signed-off-by: evenyag * fix: don't ignore error in PrimaryKeyFilter Signed-off-by: evenyag --------- Signed-off-by: evenyag --- src/common/recordbatch/src/filter.rs | 2 +- .../benches/bench_primary_key_filter.rs | 8 +- src/mito-codec/src/error.rs | 9 + src/mito-codec/src/primary_key_filter.rs | 168 ++++-- src/mito-codec/src/row_converter.rs | 3 +- src/mito-codec/src/row_converter/dense.rs | 2 + src/mito-codec/src/row_converter/sparse.rs | 2 + src/mito2/src/engine/row_selector_test.rs | 90 +++ src/mito2/src/memtable/bulk/part_reader.rs | 1 + .../src/memtable/partition_tree/partition.rs | 3 +- .../src/memtable/partition_tree/shard.rs | 5 +- .../memtable/partition_tree/shard_builder.rs | 5 +- src/mito2/src/sst/parquet.rs | 161 ++++++ src/mito2/src/sst/parquet/file_range.rs | 130 ++++- src/mito2/src/sst/parquet/flat_format.rs | 7 + src/mito2/src/sst/parquet/prefilter.rs | 541 +++++++----------- src/mito2/src/sst/parquet/reader.rs | 232 +++++++- src/mito2/src/sst/parquet/row_group.rs | 22 +- src/mito2/src/sst/parquet/row_selection.rs | 84 ++- 19 files changed, 1061 insertions(+), 414 deletions(-) diff --git a/src/common/recordbatch/src/filter.rs b/src/common/recordbatch/src/filter.rs index d7c522e656..9f1b596a49 100644 --- a/src/common/recordbatch/src/filter.rs +++ b/src/common/recordbatch/src/filter.rs @@ -48,7 +48,7 @@ use crate::error::{ArrowComputeSnafu, Result, ToArrowScalarSnafu, UnsupportedOpe /// /// This struct contains normalized predicate expr. In the form of /// `col` `op` `literal` where the `col` is provided from input. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct SimpleFilterEvaluator { /// Name of the referenced column. column_name: String, diff --git a/src/mito-codec/benches/bench_primary_key_filter.rs b/src/mito-codec/benches/bench_primary_key_filter.rs index 47158f87f8..2cc35abb22 100644 --- a/src/mito-codec/benches/bench_primary_key_filter.rs +++ b/src/mito-codec/benches/bench_primary_key_filter.rs @@ -246,18 +246,18 @@ fn bench_primary_key_filter(c: &mut Criterion) { let dense_pk = encode_dense_pk(&metadata, &row); let dense_codec = DensePrimaryKeyCodec::new(&metadata); - let mut dense_fast = dense_codec.primary_key_filter(&metadata, filters.clone()); + let mut dense_fast = dense_codec.primary_key_filter(&metadata, filters.clone(), false); let mut dense_offsets = Vec::new(); let sparse_pk = encode_sparse_pk(&metadata, &row); let sparse_codec = SparsePrimaryKeyCodec::new(&metadata); - let mut sparse_fast = sparse_codec.primary_key_filter(&metadata, filters.clone()); + let mut sparse_fast = sparse_codec.primary_key_filter(&metadata, filters.clone(), false); let mut sparse_offsets = std::collections::HashMap::new(); let mut group = c.benchmark_group(format!("primary_key_filter/{case_name}")); group.bench_function("dense/fast", |b| { - b.iter(|| black_box(dense_fast.matches(black_box(&dense_pk)))) + b.iter(|| black_box(dense_fast.matches(black_box(&dense_pk)).unwrap())) }); group.bench_function("dense/scalar", |b| { b.iter(|| { @@ -272,7 +272,7 @@ fn bench_primary_key_filter(c: &mut Criterion) { }); group.bench_function("sparse/fast", |b| { - b.iter(|| black_box(sparse_fast.matches(black_box(&sparse_pk)))) + b.iter(|| black_box(sparse_fast.matches(black_box(&sparse_pk)).unwrap())) }); group.bench_function("sparse/scalar", |b| { b.iter(|| { diff --git a/src/mito-codec/src/error.rs b/src/mito-codec/src/error.rs index 1be0074b1e..78a656415a 100644 --- a/src/mito-codec/src/error.rs +++ b/src/mito-codec/src/error.rs @@ -72,6 +72,14 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Failed to evaluate filter"))] + EvaluateFilter { + #[snafu(source(from(common_recordbatch::error::Error, Box::new)))] + source: Box, + #[snafu(implicit)] + location: Location, + }, } pub type Result = std::result::Result; @@ -86,6 +94,7 @@ impl ErrorExt for Error { StatusCode::InvalidArguments } NotSupportedField { .. } | UnsupportedOperation { .. } => StatusCode::Unsupported, + EvaluateFilter { source, .. } => source.status_code(), } } diff --git a/src/mito-codec/src/primary_key_filter.rs b/src/mito-codec/src/primary_key_filter.rs index 189c7a08cd..70fda7bf54 100644 --- a/src/mito-codec/src/primary_key_filter.rs +++ b/src/mito-codec/src/primary_key_filter.rs @@ -20,11 +20,12 @@ use common_recordbatch::filter::SimpleFilterEvaluator; use datatypes::data_type::ConcreteDataType; use datatypes::value::Value; use memcomparable::Serializer; +use snafu::ResultExt; use store_api::metadata::RegionMetadataRef; use store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME; use store_api::storage::ColumnId; -use crate::error::Result; +use crate::error::{EvaluateFilterSnafu, Result}; use crate::row_converter::{ DensePrimaryKeyCodec, PrimaryKeyFilter, SortField, SparsePrimaryKeyCodec, }; @@ -41,8 +42,12 @@ struct PrimaryKeyFilterInner { } impl PrimaryKeyFilterInner { - fn new(metadata: RegionMetadataRef, filters: Arc>) -> Self { - let compiled_filters = Self::compile_filters(&metadata, &filters); + fn new( + metadata: RegionMetadataRef, + filters: Arc>, + skip_partition_column: bool, + ) -> Self { + let compiled_filters = Self::compile_filters(&metadata, &filters, skip_partition_column); Self { filters, compiled_filters, @@ -52,6 +57,7 @@ impl PrimaryKeyFilterInner { fn compile_filters( metadata: &RegionMetadataRef, filters: &[SimpleFilterEvaluator], + skip_partition_column: bool, ) -> Vec { if filters.is_empty() || metadata.primary_key.is_empty() { return Vec::new(); @@ -59,7 +65,7 @@ impl PrimaryKeyFilterInner { let mut compiled_filters = Vec::with_capacity(filters.len()); for (filter_idx, filter) in filters.iter().enumerate() { - if is_partition_column(filter.column_name()) { + if skip_partition_column && is_partition_column(filter.column_name()) { continue; } @@ -91,43 +97,36 @@ impl PrimaryKeyFilterInner { compiled_filters } - fn evaluate_filters<'a>(&self, accessor: &mut impl PrimaryKeyValueAccessor<'a>) -> bool { + fn evaluate_filters<'a>( + &self, + accessor: &mut impl PrimaryKeyValueAccessor<'a>, + ) -> Result { if self.compiled_filters.is_empty() { - return true; + return Ok(true); } for compiled in &self.compiled_filters { let filter = &self.filters[compiled.filter_idx]; let passed = if let Some(fast_path) = &compiled.fast_path { - let encoded_value = match accessor.encoded_value(compiled) { - Ok(v) => v, - Err(e) => { - common_telemetry::error!(e; "Failed to decode primary key"); - return true; - } - }; + let encoded_value = accessor.encoded_value(compiled)?; fast_path.matches(encoded_value) } else { - let value = match accessor.decode_value(compiled) { - Ok(v) => v, - Err(e) => { - common_telemetry::error!(e; "Failed to decode primary key"); - return true; - } - }; + let value = accessor.decode_value(compiled)?; // Safety: arrow schema and datatypes are constructed from the same source. let scalar_value = value.try_to_scalar_value(&compiled.data_type).unwrap(); - filter.evaluate_scalar(&scalar_value).unwrap_or(true) + filter + .evaluate_scalar(&scalar_value) + .context(EvaluateFilterSnafu)? }; if !passed { - return false; + return Ok(false); } } - true + Ok(true) } } @@ -258,9 +257,10 @@ impl DensePrimaryKeyFilter { metadata: RegionMetadataRef, filters: Arc>, codec: DensePrimaryKeyCodec, + skip_partition_column: bool, ) -> Self { Self { - inner: PrimaryKeyFilterInner::new(metadata, filters), + inner: PrimaryKeyFilterInner::new(metadata, filters, skip_partition_column), codec, offsets_buf: Vec::new(), } @@ -268,7 +268,7 @@ impl DensePrimaryKeyFilter { } impl PrimaryKeyFilter for DensePrimaryKeyFilter { - fn matches(&mut self, pk: &[u8]) -> bool { + fn matches(&mut self, pk: &[u8]) -> Result { self.offsets_buf.clear(); let mut accessor = DensePrimaryKeyValueAccessor { pk, @@ -311,9 +311,10 @@ impl SparsePrimaryKeyFilter { metadata: RegionMetadataRef, filters: Arc>, codec: SparsePrimaryKeyCodec, + skip_partition_column: bool, ) -> Self { Self { - inner: PrimaryKeyFilterInner::new(metadata, filters), + inner: PrimaryKeyFilterInner::new(metadata, filters, skip_partition_column), codec, offsets_map: HashMap::new(), } @@ -321,7 +322,7 @@ impl SparsePrimaryKeyFilter { } impl PrimaryKeyFilter for SparsePrimaryKeyFilter { - fn matches(&mut self, pk: &[u8]) -> bool { + fn matches(&mut self, pk: &[u8]) -> Result { self.offsets_map.clear(); let mut accessor = SparsePrimaryKeyValueAccessor { pk, @@ -369,6 +370,7 @@ mod tests { use datatypes::schema::ColumnSchema; use datatypes::value::{OrderedFloat, ValueRef}; use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder}; + use store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME; use store_api::storage::{ColumnId, RegionId}; use super::*; @@ -423,6 +425,36 @@ mod tests { Arc::new(metadata) } + fn setup_partitioned_metadata() -> RegionMetadataRef { + let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1)); + builder + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + DATA_SCHEMA_TABLE_ID_COLUMN_NAME, + ConcreteDataType::uint32_datatype(), + false, + ), + semantic_type: SemanticType::Tag, + column_id: 10, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new("tag", ConcreteDataType::string_datatype(), true), + semantic_type: SemanticType::Tag, + column_id: 1, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + greptime_timestamp(), + ConcreteDataType::timestamp_nanosecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 2, + }) + .primary_key(vec![10, 1]); + Arc::new(builder.build().unwrap()) + } + fn create_test_row() -> Vec<(ColumnId, ValueRef<'static>)> { vec![ (1, ValueRef::String("greptime-frontend-6989d9899-22222")), @@ -479,8 +511,8 @@ mod tests { )]); let pk = encode_sparse_pk(&metadata, create_test_row()); let codec = SparsePrimaryKeyCodec::new(&metadata); - let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec); - assert!(filter.matches(&pk)); + let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec, false); + assert!(filter.matches(&pk).unwrap()); } #[test] @@ -492,8 +524,8 @@ mod tests { )]); let pk = encode_sparse_pk(&metadata, create_test_row()); let codec = SparsePrimaryKeyCodec::new(&metadata); - let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec); - assert!(!filter.matches(&pk)); + let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec, false); + assert!(!filter.matches(&pk).unwrap()); } #[test] @@ -505,8 +537,8 @@ mod tests { )]); let pk = encode_sparse_pk(&metadata, create_test_row()); let codec = SparsePrimaryKeyCodec::new(&metadata); - let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec); - assert!(filter.matches(&pk)); + let mut filter = SparsePrimaryKeyFilter::new(metadata, filters, codec, false); + assert!(filter.matches(&pk).unwrap()); } #[test] @@ -518,8 +550,8 @@ mod tests { )]); let pk = encode_dense_pk(&metadata, create_test_row()); let codec = DensePrimaryKeyCodec::new(&metadata); - let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec); - assert!(filter.matches(&pk)); + let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false); + assert!(filter.matches(&pk).unwrap()); } #[test] @@ -531,8 +563,8 @@ mod tests { )]); let pk = encode_dense_pk(&metadata, create_test_row()); let codec = DensePrimaryKeyCodec::new(&metadata); - let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec); - assert!(!filter.matches(&pk)); + let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false); + assert!(!filter.matches(&pk).unwrap()); } #[test] @@ -544,8 +576,8 @@ mod tests { )]); let pk = encode_dense_pk(&metadata, create_test_row()); let codec = DensePrimaryKeyCodec::new(&metadata); - let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec); - assert!(filter.matches(&pk)); + let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false); + assert!(filter.matches(&pk).unwrap()); } #[test] @@ -563,8 +595,9 @@ mod tests { for (op, value, expected) in cases { let filters = Arc::new(vec![create_filter_with_op("pod", op, value)]); - let mut filter = DensePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone()); - assert_eq!(expected, filter.matches(&pk)); + let mut filter = + DensePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone(), false); + assert_eq!(expected, filter.matches(&pk).unwrap()); } } @@ -583,8 +616,9 @@ mod tests { for (op, value, expected) in cases { let filters = Arc::new(vec![create_filter_with_op("pod", op, value)]); - let mut filter = SparsePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone()); - assert_eq!(expected, filter.matches(&pk)); + let mut filter = + SparsePrimaryKeyFilter::new(metadata.clone(), filters, codec.clone(), false); + assert_eq!(expected, filter.matches(&pk).unwrap()); } } @@ -616,8 +650,52 @@ mod tests { .unwrap(); let filters = Arc::new(vec![create_filter_with_op("f", Operator::Eq, 0.0_f64)]); - let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec); + let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false); - assert!(filter.matches(&pk)); + assert!(filter.matches(&pk).unwrap()); + } + + #[test] + fn test_dense_primary_key_filter_matches_partition_column_by_default() { + let metadata = setup_partitioned_metadata(); + let codec = DensePrimaryKeyCodec::new(&metadata); + let mut pk = Vec::new(); + codec + .encode_to_vec( + [ValueRef::UInt32(42), ValueRef::String("host-a")].into_iter(), + &mut pk, + ) + .unwrap(); + + let filters = Arc::new(vec![create_filter_with_op( + DATA_SCHEMA_TABLE_ID_COLUMN_NAME, + Operator::Eq, + 42_u32, + )]); + let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, false); + + assert!(filter.matches(&pk).unwrap()); + } + + #[test] + fn test_dense_primary_key_filter_can_skip_partition_column() { + let metadata = setup_partitioned_metadata(); + let codec = DensePrimaryKeyCodec::new(&metadata); + let mut pk = Vec::new(); + codec + .encode_to_vec( + [ValueRef::UInt32(42), ValueRef::String("host-a")].into_iter(), + &mut pk, + ) + .unwrap(); + + let filters = Arc::new(vec![create_filter_with_op( + DATA_SCHEMA_TABLE_ID_COLUMN_NAME, + Operator::Eq, + 7_u32, + )]); + let mut filter = DensePrimaryKeyFilter::new(metadata, filters, codec, true); + + assert!(filter.matches(&pk).unwrap()); } } diff --git a/src/mito-codec/src/row_converter.rs b/src/mito-codec/src/row_converter.rs index 6fe33a9ee7..fa57e1d96e 100644 --- a/src/mito-codec/src/row_converter.rs +++ b/src/mito-codec/src/row_converter.rs @@ -53,7 +53,7 @@ pub trait PrimaryKeyCodecExt { pub trait PrimaryKeyFilter: Send + Sync { /// Returns true if the primary key matches the filter. - fn matches(&mut self, pk: &[u8]) -> bool; + fn matches(&mut self, pk: &[u8]) -> Result; } /// Composite values decoded from primary key bytes. @@ -120,6 +120,7 @@ pub trait PrimaryKeyCodec: Send + Sync + Debug { &self, metadata: &RegionMetadataRef, filters: Arc>, + skip_partition_column: bool, ) -> Box; /// Returns the estimated size of the primary key. diff --git a/src/mito-codec/src/row_converter/dense.rs b/src/mito-codec/src/row_converter/dense.rs index 6cc70feaea..4bc774c941 100644 --- a/src/mito-codec/src/row_converter/dense.rs +++ b/src/mito-codec/src/row_converter/dense.rs @@ -556,11 +556,13 @@ impl PrimaryKeyCodec for DensePrimaryKeyCodec { &self, metadata: &RegionMetadataRef, filters: Arc>, + skip_partition_column: bool, ) -> Box { Box::new(DensePrimaryKeyFilter::new( metadata.clone(), filters, self.clone(), + skip_partition_column, )) } diff --git a/src/mito-codec/src/row_converter/sparse.rs b/src/mito-codec/src/row_converter/sparse.rs index 4638ddcefb..00ec51530c 100644 --- a/src/mito-codec/src/row_converter/sparse.rs +++ b/src/mito-codec/src/row_converter/sparse.rs @@ -357,11 +357,13 @@ impl PrimaryKeyCodec for SparsePrimaryKeyCodec { &self, metadata: &RegionMetadataRef, filters: Arc>, + skip_partition_column: bool, ) -> Box { Box::new(SparsePrimaryKeyFilter::new( metadata.clone(), filters, self.clone(), + skip_partition_column, )) } diff --git a/src/mito2/src/engine/row_selector_test.rs b/src/mito2/src/engine/row_selector_test.rs index d79152e57f..26d7327c2f 100644 --- a/src/mito2/src/engine/row_selector_test.rs +++ b/src/mito2/src/engine/row_selector_test.rs @@ -13,12 +13,15 @@ // limitations under the License. use api::v1::Rows; +use common_base::readable_size::ReadableSize; use common_recordbatch::RecordBatches; +use datafusion_expr::{col, lit}; use store_api::region_engine::RegionEngine; use store_api::region_request::RegionRequest; use store_api::storage::{RegionId, ScanRequest, TimeSeriesRowSelector}; use crate::config::MitoConfig; +use crate::engine::MitoEngine; use crate::test_util::batch_util::sort_batches_and_print; use crate::test_util::{ CreateRequestBuilder, TestEnv, build_rows_for_key, flush_region, put_rows, rows_schema, @@ -107,6 +110,27 @@ async fn test_last_row(append_mode: bool, flat_format: bool) { assert_eq!(expected, sort_batches_and_print(&batches, &["tag_0", "ts"])); } +async fn scan_last_row( + engine: &MitoEngine, + region_id: RegionId, + filters: Vec, +) -> String { + let scanner = engine + .scanner( + region_id, + ScanRequest { + filters, + series_row_selector: Some(TimeSeriesRowSelector::LastRow), + ..Default::default() + }, + ) + .await + .unwrap(); + let stream = scanner.scan().await.unwrap(); + let batches = RecordBatches::try_collect(stream).await.unwrap(); + sort_batches_and_print(&batches, &["tag_0", "ts"]) +} + #[tokio::test] async fn test_last_row_append_mode_disabled() { test_last_row(false, false).await; @@ -126,3 +150,69 @@ async fn test_last_row_flat_format_append_mode_disabled() { async fn test_last_row_flat_format_append_mode_enabled() { test_last_row(true, true).await; } + +#[tokio::test] +async fn test_last_row_flat_format_prefilter_does_not_poison_selector_cache() { + let mut env = TestEnv::new().await; + let engine = env + .create_engine(MitoConfig { + selector_result_cache_size: ReadableSize::mb(1), + ..Default::default() + }) + .await; + let region_id = RegionId::new(1, 1); + + env.get_schema_metadata_manager() + .register_region_table_info( + region_id.table_id(), + "test_table", + "test_catalog", + "test_schema", + None, + env.get_kv_backend(), + ) + .await; + + let request = CreateRequestBuilder::new() + .insert_option("sst_format", "flat") + .build(); + let column_schemas = rows_schema(&request); + engine + .handle_request(region_id, RegionRequest::Create(request)) + .await + .unwrap(); + + let rows = Rows { + schema: column_schemas, + rows: [ + build_rows_for_key("a", 0, 3, 0), + build_rows_for_key("b", 0, 3, 10), + ] + .concat(), + }; + put_rows(&engine, region_id, rows).await; + flush_region(&engine, region_id, Some(16)).await; + + let filtered = scan_last_row(&engine, region_id, vec![col("tag_0").eq(lit("a"))]).await; + assert_eq!( + "\ ++-------+---------+---------------------+ +| tag_0 | field_0 | ts | ++-------+---------+---------------------+ +| a | 2.0 | 1970-01-01T00:00:02 | ++-------+---------+---------------------+", + filtered + ); + + let unfiltered = scan_last_row(&engine, region_id, vec![]).await; + assert_eq!( + "\ ++-------+---------+---------------------+ +| tag_0 | field_0 | ts | ++-------+---------+---------------------+ +| a | 2.0 | 1970-01-01T00:00:02 | +| b | 12.0 | 1970-01-01T00:00:02 | ++-------+---------+---------------------+", + unfiltered + ); +} diff --git a/src/mito2/src/memtable/bulk/part_reader.rs b/src/mito2/src/memtable/bulk/part_reader.rs index edb9ff52d9..1375e79542 100644 --- a/src/mito2/src/memtable/bulk/part_reader.rs +++ b/src/mito2/src/memtable/bulk/part_reader.rs @@ -367,6 +367,7 @@ fn apply_combined_filters( let predicate_mask = context.base.compute_filter_mask_flat( &record_batch, skip_fields, + false, &mut tag_decode_state, )?; // If predicate filters out the entire batch, return None early diff --git a/src/mito2/src/memtable/partition_tree/partition.rs b/src/mito2/src/memtable/partition_tree/partition.rs index e6e3b8bf81..0ffbce4867 100644 --- a/src/mito2/src/memtable/partition_tree/partition.rs +++ b/src/mito2/src/memtable/partition_tree/partition.rs @@ -152,7 +152,8 @@ impl Partition { filters: &Arc>, ) -> Option> { if need_prune_key { - let filter = row_codec.primary_key_filter(metadata, filters.clone()); + // TODO(yingwen): Remove `skip_partition_column` after dropping PartitionTreeMemtable. + let filter = row_codec.primary_key_filter(metadata, filters.clone(), true); Some(filter) } else { None diff --git a/src/mito2/src/memtable/partition_tree/shard.rs b/src/mito2/src/memtable/partition_tree/shard.rs index 162f937a7c..c5dc25f573 100644 --- a/src/mito2/src/memtable/partition_tree/shard.rs +++ b/src/mito2/src/memtable/partition_tree/shard.rs @@ -19,9 +19,10 @@ use std::time::{Duration, Instant}; use mito_codec::key_values::KeyValue; use mito_codec::row_converter::PrimaryKeyFilter; +use snafu::ResultExt; use store_api::metadata::RegionMetadataRef; -use crate::error::Result; +use crate::error::{DecodeSnafu, Result}; use crate::memtable::partition_tree::data::{ DATA_INIT_CAP, DataBatch, DataParts, DataPartsReader, DataPartsReaderBuilder, }; @@ -243,7 +244,7 @@ impl ShardReader { // Safety: `key_filter` is some so the shard has primary keys. let key = self.key_dict.as_ref().unwrap().key_by_pk_index(pk_index); let now = Instant::now(); - if key_filter.matches(key) { + if key_filter.matches(key).context(DecodeSnafu)? { self.prune_pk_cost += now.elapsed(); self.last_yield_pk_index = Some(pk_index); self.keys_after_pruning += 1; diff --git a/src/mito2/src/memtable/partition_tree/shard_builder.rs b/src/mito2/src/memtable/partition_tree/shard_builder.rs index 26de85767d..78eeb463c6 100644 --- a/src/mito2/src/memtable/partition_tree/shard_builder.rs +++ b/src/mito2/src/memtable/partition_tree/shard_builder.rs @@ -20,9 +20,10 @@ use std::time::{Duration, Instant}; use mito_codec::key_values::KeyValue; use mito_codec::row_converter::PrimaryKeyFilter; +use snafu::ResultExt; use store_api::metadata::RegionMetadataRef; -use crate::error::Result; +use crate::error::{DecodeSnafu, Result}; use crate::memtable::partition_tree::data::{ DATA_INIT_CAP, DataBatch, DataBuffer, DataBufferReader, DataBufferReaderBuilder, DataParts, }; @@ -281,7 +282,7 @@ impl ShardBuilderReader { self.keys_before_pruning += 1; let key = self.dict_reader.key_by_pk_index(pk_index); let now = Instant::now(); - if key_filter.matches(key) { + if key_filter.matches(key).context(DecodeSnafu)? { self.prune_pk_cost += now.elapsed(); self.last_yield_pk_index = Some(pk_index); self.keys_after_pruning += 1; diff --git a/src/mito2/src/sst/parquet.rs b/src/mito2/src/sst/parquet.rs index 79a08a209d..4a3466a29c 100644 --- a/src/mito2/src/sst/parquet.rs +++ b/src/mito2/src/sst/parquet.rs @@ -142,6 +142,7 @@ mod tests { use crate::sst::index::{IndexBuildType, Indexer, IndexerBuilder, IndexerBuilderImpl}; use crate::sst::parquet::flat_format::FlatWriteFormat; use crate::sst::parquet::reader::{ParquetReader, ParquetReaderBuilder, ReaderMetrics}; + use crate::sst::parquet::row_selection::RowGroupSelection; use crate::sst::parquet::writer::ParquetWriter; use crate::sst::{ DEFAULT_WRITE_CONCURRENCY, FlatSchemaOptions, location, to_flat_sst_arrow_schema, @@ -1113,6 +1114,39 @@ mod tests { assert!(reader.next_record_batch().await.unwrap().is_none()); } + fn new_record_batch_from_rows(rows: &[(&str, &str, i64)]) -> RecordBatch { + let metadata = Arc::new(sst_region_metadata()); + let flat_schema = to_flat_sst_arrow_schema(&metadata, &FlatSchemaOptions::default()); + + let mut tag_0_builder = StringDictionaryBuilder::::new(); + let mut tag_1_builder = StringDictionaryBuilder::::new(); + let mut pk_builder = BinaryDictionaryBuilder::::new(); + let mut field_values = Vec::with_capacity(rows.len()); + let mut timestamps = Vec::with_capacity(rows.len()); + + for (tag_0, tag_1, ts) in rows { + tag_0_builder.append_value(*tag_0); + tag_1_builder.append_value(*tag_1); + pk_builder.append(new_primary_key(&[tag_0, tag_1])).unwrap(); + field_values.push(*ts as u64); + timestamps.push(*ts); + } + + RecordBatch::try_new( + flat_schema, + vec![ + Arc::new(tag_0_builder.finish()) as ArrayRef, + Arc::new(tag_1_builder.finish()) as ArrayRef, + Arc::new(UInt64Array::from(field_values)) as ArrayRef, + Arc::new(TimestampMillisecondArray::from(timestamps)) as ArrayRef, + Arc::new(pk_builder.finish()) as ArrayRef, + Arc::new(UInt64Array::from_value(1000, rows.len())) as ArrayRef, + Arc::new(UInt8Array::from_value(OpType::Put as u8, rows.len())) as ArrayRef, + ], + ) + .unwrap() + } + /// Creates a flat format RecordBatch for testing with sparse primary key encoding. /// Similar to `new_record_batch_by_range` but without individual primary key columns. fn new_record_batch_by_range_sparse( @@ -1642,6 +1676,133 @@ mod tests { assert_eq!(metrics.filter_metrics.rows_bloom_filtered, 100); } + #[tokio::test] + async fn test_reader_prefilter_with_outer_selection_and_trailing_filtered_rows() { + let mut env = TestEnv::new().await; + let object_store = env.init_object_store_manager(); + let file_path = RegionFilePathFactory::new(FILE_DIR.to_string(), PathType::Bare); + let metadata = Arc::new(sst_region_metadata()); + let row_group_size = 10; + + let flat_source = new_flat_source_from_record_batches(vec![ + new_record_batch_by_range(&["a", "d"], 0, 3), + new_record_batch_by_range(&["b", "d"], 3, 10), + ]); + let write_opts = WriteOptions { + row_group_size, + ..Default::default() + }; + let indexer_builder = create_test_indexer_builder( + &env, + object_store.clone(), + file_path.clone(), + metadata.clone(), + row_group_size, + ); + let info = write_flat_sst( + object_store.clone(), + metadata.clone(), + indexer_builder, + file_path, + flat_source, + &write_opts, + ) + .await; + let handle = create_file_handle_from_sst_info(&info, &metadata); + + let builder = + ParquetReaderBuilder::new(FILE_DIR.to_string(), PathType::Bare, handle, object_store) + .flat_format(true) + .predicate(Some(Predicate::new(vec![col("tag_0").eq(lit("a"))]))); + + let mut metrics = ReaderMetrics::default(); + let (context, _) = builder + .build_reader_input(&mut metrics) + .await + .unwrap() + .unwrap(); + let selection = RowGroupSelection::from_row_ranges( + vec![(0, std::iter::once(0..6).collect())], + row_group_size, + ); + + let mut reader = ParquetReader::new(Arc::new(context), selection) + .await + .unwrap(); + check_record_batch_reader_result( + &mut reader, + &[new_record_batch_by_range(&["a", "d"], 0, 3)], + ) + .await; + } + + #[tokio::test] + async fn test_reader_prefilter_with_outer_selection_disjoint_matches_and_trailing_gap() { + let mut env = TestEnv::new().await; + let object_store = env.init_object_store_manager(); + let file_path = RegionFilePathFactory::new(FILE_DIR.to_string(), PathType::Bare); + let metadata = Arc::new(sst_region_metadata()); + let row_group_size = 8; + + let flat_source = new_flat_source_from_record_batches(vec![ + new_record_batch_by_range(&["a", "d"], 0, 2), + new_record_batch_by_range(&["b", "d"], 2, 4), + new_record_batch_by_range(&["a", "d"], 4, 6), + new_record_batch_by_range(&["c", "d"], 6, 8), + ]); + let write_opts = WriteOptions { + row_group_size, + ..Default::default() + }; + let indexer_builder = create_test_indexer_builder( + &env, + object_store.clone(), + file_path.clone(), + metadata.clone(), + row_group_size, + ); + let info = write_flat_sst( + object_store.clone(), + metadata.clone(), + indexer_builder, + file_path, + flat_source, + &write_opts, + ) + .await; + let handle = create_file_handle_from_sst_info(&info, &metadata); + + let builder = + ParquetReaderBuilder::new(FILE_DIR.to_string(), PathType::Bare, handle, object_store) + .flat_format(true) + .predicate(Some(Predicate::new(vec![col("tag_0").eq(lit("a"))]))); + + let mut metrics = ReaderMetrics::default(); + let (context, _) = builder + .build_reader_input(&mut metrics) + .await + .unwrap() + .unwrap(); + let selection = RowGroupSelection::from_row_ranges( + vec![(0, std::iter::once(0..8).collect())], + row_group_size, + ); + + let mut reader = ParquetReader::new(Arc::new(context), selection) + .await + .unwrap(); + check_record_batch_reader_result( + &mut reader, + &[new_record_batch_from_rows(&[ + ("a", "d", 0), + ("a", "d", 1), + ("a", "d", 4), + ("a", "d", 5), + ])], + ) + .await; + } + #[tokio::test] async fn test_write_flat_read_with_inverted_index_sparse() { common_telemetry::init_default_ut_logging(); diff --git a/src/mito2/src/sst/parquet/file_range.rs b/src/mito2/src/sst/parquet/file_range.rs index 3a5251cb1a..8b4a61acb7 100644 --- a/src/mito2/src/sst/parquet/file_range.rs +++ b/src/mito2/src/sst/parquet/file_range.rs @@ -37,6 +37,7 @@ use store_api::metadata::RegionMetadataRef; use store_api::storage::{ColumnId, TimeSeriesRowSelector}; use table::predicate::Predicate; +use crate::cache::CacheStrategy; use crate::error::{ ComputeArrowSnafu, DataTypeMismatchSnafu, DecodeSnafu, DecodeStatsSnafu, EvalPartitionFilterSnafu, NewRecordBatchSnafu, RecordBatchSnafu, Result, StatsNotPresentSnafu, @@ -53,7 +54,8 @@ use crate::sst::parquet::flat_format::{ }; use crate::sst::parquet::format::ReadFormat; use crate::sst::parquet::reader::{ - FlatRowGroupReader, MaybeFilter, RowGroupReader, RowGroupReaderBuilder, SimpleFilterContext, + FlatRowGroupReader, MaybeFilter, RowGroupBuildContext, RowGroupReader, RowGroupReaderBuilder, + SimpleFilterContext, }; use crate::sst::parquet::row_group::ParquetFetchMetrics; use crate::sst::parquet::stats::RowGroupPruningStats; @@ -181,14 +183,17 @@ impl FileRange { if !self.in_dynamic_filter_range() { return Ok(None); } + // Compute skip_fields once for this row group + let skip_fields = self.context.should_skip_fields(self.row_group_idx); let parquet_reader = self .context .reader_builder - .build( + .build(self.context.build_context( self.row_group_idx, self.row_selection.clone(), fetch_metrics, - ) + skip_fields, + )) .await?; let use_last_row_reader = if selector @@ -210,9 +215,6 @@ impl FileRange { false }; - // Compute skip_fields once for this row group - let skip_fields = self.context.should_skip_fields(self.row_group_idx); - let prune_reader = if use_last_row_reader { // Row group is PUT only, use LastRowReader to skip unnecessary rows. let reader = RowGroupLastRowCachedReader::new( @@ -243,14 +245,17 @@ impl FileRange { if !self.in_dynamic_filter_range() { return Ok(None); } + // Compute skip_fields once for this row group + let skip_fields = self.context.should_skip_fields(self.row_group_idx); let parquet_reader = self .context .reader_builder - .build( + .build(self.context.build_context( self.row_group_idx, self.row_selection.clone(), fetch_metrics, - ) + skip_fields, + )) .await?; let use_last_row_reader = if selector @@ -271,16 +276,20 @@ impl FileRange { false }; - // Compute skip_fields once for this row group - let skip_fields = self.context.should_skip_fields(self.row_group_idx); - let flat_prune_reader = if use_last_row_reader { let flat_row_group_reader = FlatRowGroupReader::new(self.context.clone(), parquet_reader); + // Flat PK prefilter makes the input stream predicate-dependent, so cached + // selector results are not reusable across queries with different filters. + let cache_strategy = if self.context.reader_builder.has_flat_primary_key_prefilter() { + CacheStrategy::Disabled + } else { + self.context.reader_builder.cache_strategy().clone() + }; let reader = FlatRowGroupLastRowCachedReader::new( self.file_handle().file_id().file_id(), self.row_group_idx, - self.context.reader_builder.cache_strategy().clone(), + cache_strategy, self.context.read_format().projection_indices(), flat_row_group_reader, ); @@ -387,7 +396,11 @@ impl FileRangeContext { input: RecordBatch, skip_fields: bool, ) -> Result> { - self.base.precise_filter_flat(input, skip_fields) + self.base.precise_filter_flat( + input, + skip_fields, + self.reader_builder.has_flat_primary_key_prefilter(), + ) } /// Determines whether to skip field filters based on PreFilterMode and row group delete status. @@ -408,6 +421,23 @@ impl FileRangeContext { row_group_contains_delete(metadata, row_group_index, self.reader_builder.file_path()) } + /// Creates a [RowGroupBuildContext] for building row group readers with prefiltering. + pub(crate) fn build_context<'a>( + &'a self, + row_group_idx: usize, + row_selection: Option, + fetch_metrics: Option<&'a ParquetFetchMetrics>, + skip_fields: bool, + ) -> RowGroupBuildContext<'a> { + RowGroupBuildContext { + filters: &self.base.filters, + skip_fields, + row_group_idx, + row_selection, + fetch_metrics, + } + } + /// Returns the estimated memory size of this context. /// Mainly accounts for the parquet metadata size. pub(crate) fn memory_size(&self) -> usize { @@ -600,9 +630,15 @@ impl RangeBase { &self, input: RecordBatch, skip_fields: bool, + skip_prefiltered_pk_filters: bool, ) -> Result> { let mut tag_decode_state = TagDecodeState::new(); - let mask = self.compute_filter_mask_flat(&input, skip_fields, &mut tag_decode_state)?; + let mask = self.compute_filter_mask_flat( + &input, + skip_fields, + skip_prefiltered_pk_filters, + &mut tag_decode_state, + )?; // If mask is None, the entire batch is filtered out let Some(mut mask) = mask else { @@ -647,6 +683,7 @@ impl RangeBase { &self, input: &RecordBatch, skip_fields: bool, + skip_prefiltered_pk_filters: bool, tag_decode_state: &mut TagDecodeState, ) -> Result> { let mut mask = BooleanBuffer::new_set(input.num_rows()); @@ -674,6 +711,12 @@ impl RangeBase { continue; } + // Flat parquet PK prefiltering already applied these tag predicates while refining + // row selection, so skip them here to avoid decoding/evaluating the same condition twice. + if skip_prefiltered_pk_filters && filter_ctx.usable_primary_key_filter() { + continue; + } + // Get the column directly by its projected index. // If the column is missing and it's not a tag/time column, this filter is skipped. // Assumes the projection indices align with the input batch schema. @@ -926,3 +969,62 @@ impl RangeBase { RecordBatch::try_new(arrow_schema.clone(), columns).context(NewRecordBatchSnafu) } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use datafusion_expr::{col, lit}; + + use super::*; + use crate::sst::parquet::format::ReadFormat; + use crate::test_util::sst_util::{new_record_batch_with_custom_sequence, sst_region_metadata}; + + fn new_test_range_base(filters: Vec) -> RangeBase { + let metadata: RegionMetadataRef = Arc::new(sst_region_metadata()); + let read_format = ReadFormat::new_flat( + metadata.clone(), + metadata.column_metadatas.iter().map(|c| c.column_id), + None, + "test", + true, + ) + .unwrap(); + + RangeBase { + filters, + dyn_filters: vec![], + read_format, + expected_metadata: None, + prune_schema: metadata.schema.clone(), + codec: mito_codec::row_converter::build_primary_key_codec(metadata.as_ref()), + compat_batch: None, + compaction_projection_mapper: None, + pre_filter_mode: PreFilterMode::All, + partition_filter: None, + } + } + + #[test] + fn test_compute_filter_mask_flat_skips_prefiltered_pk_filters() { + let metadata: RegionMetadataRef = Arc::new(sst_region_metadata()); + let filters = vec![ + SimpleFilterContext::new_opt(&metadata, None, &col("tag_0").eq(lit("a"))).unwrap(), + SimpleFilterContext::new_opt(&metadata, None, &col("field_0").gt(lit(1_u64))).unwrap(), + ]; + let base = new_test_range_base(filters); + let batch = new_record_batch_with_custom_sequence(&["b", "x"], 0, 4, 1); + + let mask_without_skip = base + .compute_filter_mask_flat(&batch, false, false, &mut TagDecodeState::new()) + .unwrap() + .unwrap(); + assert_eq!(mask_without_skip.count_set_bits(), 0); + + let mask_with_skip = base + .compute_filter_mask_flat(&batch, false, true, &mut TagDecodeState::new()) + .unwrap() + .unwrap(); + assert_eq!(mask_with_skip.count_set_bits(), 2); + } +} diff --git a/src/mito2/src/sst/parquet/flat_format.rs b/src/mito2/src/sst/parquet/flat_format.rs index 8a59e9a97d..ca39cac7e1 100644 --- a/src/mito2/src/sst/parquet/flat_format.rs +++ b/src/mito2/src/sst/parquet/flat_format.rs @@ -282,6 +282,13 @@ impl FlatReadFormat { } } + /// Returns `true` if raw batches from parquet use the flat layout with a + /// dictionary-encoded `__primary_key` column (i.e., [`ParquetAdapter::Flat`]). + /// Returns `false` for the legacy primary-key-to-flat conversion path. + pub(crate) fn raw_batch_has_primary_key_dictionary(&self) -> bool { + matches!(&self.parquet_adapter, ParquetAdapter::Flat(_)) + } + /// Creates a sequence array to override. pub(crate) fn new_override_sequence_array(&self, length: usize) -> Option { self.override_sequence diff --git a/src/mito2/src/sst/parquet/prefilter.rs b/src/mito2/src/sst/parquet/prefilter.rs index 5de2e3512f..07efbd052f 100644 --- a/src/mito2/src/sst/parquet/prefilter.rs +++ b/src/mito2/src/sst/parquet/prefilter.rs @@ -12,31 +12,40 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Helpers for parquet prefiltering. +//! Prefilter framework for parquet reader. +//! +//! Prefilter optimization reduces I/O by reading only a subset of columns first +//! (the prefilter phase), applying filters to compute a refined row selection, +//! then reading the remaining columns with the refined selection. use std::ops::Range; +use std::sync::Arc; use api::v1::SemanticType; use common_recordbatch::filter::SimpleFilterEvaluator; -use datatypes::arrow::array::{BinaryArray, BooleanArray}; +use datatypes::arrow::array::BinaryArray; use datatypes::arrow::record_batch::RecordBatch; -use mito_codec::primary_key_filter::is_partition_column; -use mito_codec::row_converter::PrimaryKeyFilter; +use futures::StreamExt; +use mito_codec::row_converter::{PrimaryKeyCodec, PrimaryKeyFilter}; +use parquet::arrow::ProjectionMask; +use parquet::arrow::arrow_reader::RowSelection; +use parquet::schema::types::SchemaDescriptor; use snafu::{OptionExt, ResultExt}; use store_api::metadata::{RegionMetadata, RegionMetadataRef}; -use crate::error::{ComputeArrowSnafu, Result, UnexpectedSnafu}; +use crate::error::{DecodeSnafu, ReadParquetSnafu, Result, UnexpectedSnafu}; use crate::sst::parquet::flat_format::primary_key_column_index; -use crate::sst::parquet::format::PrimaryKeyArray; +use crate::sst::parquet::format::{PrimaryKeyArray, ReadFormat}; +use crate::sst::parquet::reader::{RowGroupBuildContext, RowGroupReaderBuilder}; +use crate::sst::parquet::row_selection::row_selection_from_row_ranges_exact; -#[cfg_attr(not(test), allow(dead_code))] pub(crate) fn matching_row_ranges_by_primary_key( input: &RecordBatch, + pk_column_index: usize, pk_filter: &mut dyn PrimaryKeyFilter, ) -> Result>> { - let primary_key_index = primary_key_column_index(input.num_columns()); let pk_dict_array = input - .column(primary_key_index) + .column(pk_column_index) .as_any() .downcast_ref::() .context(UnexpectedSnafu { @@ -65,7 +74,10 @@ pub(crate) fn matching_row_ranges_by_primary_key( end += 1; } - if pk_filter.matches(pk_values.value(key as usize)) { + if pk_filter + .matches(pk_values.value(key as usize)) + .context(DecodeSnafu)? + { if let Some(last) = matched_row_ranges.last_mut() && last.end == start { @@ -81,68 +93,15 @@ pub(crate) fn matching_row_ranges_by_primary_key( Ok(matched_row_ranges) } -#[cfg_attr(not(test), allow(dead_code))] -pub(crate) fn prefilter_flat_batch_by_primary_key( - input: RecordBatch, - pk_filter: &mut dyn PrimaryKeyFilter, -) -> Result> { - if input.num_rows() == 0 { - return Ok(Some(input)); - } - - let matched_row_ranges = matching_row_ranges_by_primary_key(&input, pk_filter)?; - if matched_row_ranges.is_empty() { - return Ok(None); - } - - if matched_row_ranges.len() == 1 - && matched_row_ranges[0].start == 0 - && matched_row_ranges[0].end == input.num_rows() - { - return Ok(Some(input)); - } - - if matched_row_ranges.len() == 1 { - let span = &matched_row_ranges[0]; - return Ok(Some(input.slice(span.start, span.end - span.start))); - } - - let mut mask = vec![false; input.num_rows()]; - for span in matched_row_ranges { - mask[span].fill(true); - } - - let filtered = - datatypes::arrow::compute::filter_record_batch(&input, &BooleanArray::from(mask)) - .context(ComputeArrowSnafu)?; - if filtered.num_rows() == 0 { - Ok(None) - } else { - Ok(Some(filtered)) - } -} - -#[cfg_attr(not(test), allow(dead_code))] -pub(crate) fn retain_usable_primary_key_filters( - sst_metadata: &RegionMetadataRef, - expected_metadata: Option<&RegionMetadata>, - filters: &mut Vec, -) { - filters.retain(|filter| is_usable_primary_key_filter(sst_metadata, expected_metadata, filter)); -} - -#[cfg_attr(not(test), allow(dead_code))] +/// Returns whether a filter can be applied by parquet primary-key prefiltering. +/// +/// Unlike `PartitionTreeMemtable`, parquet prefilter always supports predicates +/// on the partition column. pub(crate) fn is_usable_primary_key_filter( sst_metadata: &RegionMetadataRef, expected_metadata: Option<&RegionMetadata>, filter: &SimpleFilterEvaluator, ) -> bool { - // TODO(yingwen): The primary key filter always skips the partition column. Consider using a flag - // to control this behavior. We can remove this behavior after we remove the PartitionTreeMemtable. - if is_partition_column(filter.column_name()) { - return false; - } - let sst_column = match expected_metadata { Some(expected_metadata) => { let Some(expected_column) = expected_metadata.column_by_name(filter.column_name()) @@ -176,7 +135,6 @@ pub(crate) fn is_usable_primary_key_filter( .is_some() } -#[cfg_attr(not(test), allow(dead_code))] pub(crate) struct CachedPrimaryKeyFilter { inner: Box, last_primary_key: Vec, @@ -184,7 +142,6 @@ pub(crate) struct CachedPrimaryKeyFilter { } impl CachedPrimaryKeyFilter { - #[cfg_attr(not(test), allow(dead_code))] pub(crate) fn new(inner: Box) -> Self { Self { inner, @@ -195,49 +152,191 @@ impl CachedPrimaryKeyFilter { } impl PrimaryKeyFilter for CachedPrimaryKeyFilter { - fn matches(&mut self, pk: &[u8]) -> bool { + fn matches(&mut self, pk: &[u8]) -> mito_codec::error::Result { if let Some(last_match) = self.last_match && self.last_primary_key == pk { - return last_match; + return Ok(last_match); } - let matched = self.inner.matches(pk); + let matched = self.inner.matches(pk)?; self.last_primary_key.clear(); self.last_primary_key.extend_from_slice(pk); self.last_match = Some(matched); - matched + Ok(matched) } } -#[cfg_attr(not(test), allow(dead_code))] -pub(crate) fn batch_single_primary_key(batch: &RecordBatch) -> Result> { - let primary_key_index = primary_key_column_index(batch.num_columns()); - let pk_dict_array = batch - .column(primary_key_index) - .as_any() - .downcast_ref::() - .context(UnexpectedSnafu { - reason: "Primary key column is not a dictionary array", - })?; - let pk_values = pk_dict_array - .values() - .as_any() - .downcast_ref::() - .context(UnexpectedSnafu { - reason: "Primary key values are not binary array", - })?; - let keys = pk_dict_array.keys(); - if keys.is_empty() { - return Ok(None); +/// Context for prefiltering a row group. +/// +/// Currently supports primary key (PK) filtering only. +/// Will be extended with simple column filters and physical filters in the future. +pub(crate) struct PrefilterContext { + /// PK filter instance. + pk_filter: Box, + /// Projection mask for reading only the PK column. + pk_projection: ProjectionMask, + /// Index of the PK column within the prefilter projection batch. + /// This is 0 when we project only the PK column. + pk_column_index: usize, +} + +/// Pre-built state for constructing [PrefilterContext] per row group. +/// +/// Fields invariant across row groups (projection mask, codec, metadata, filters) +/// are computed once. A fresh [PrefilterContext] with its own mutable PK filter +/// is created via [PrefilterContextBuilder::build()] for each row group. +pub(crate) struct PrefilterContextBuilder { + pk_projection: ProjectionMask, + pk_column_index: usize, + codec: Arc, + metadata: RegionMetadataRef, + pk_filters: Arc>, +} + +impl PrefilterContextBuilder { + /// Creates a builder if prefiltering is applicable. + /// + /// Returns `None` if: + /// - No primary key filters are available + /// - The read format doesn't use flat layout with dictionary-encoded PKs + /// - The primary key is empty + pub(crate) fn new( + read_format: &ReadFormat, + codec: &Arc, + primary_key_filters: Option<&Arc>>, + parquet_schema: &SchemaDescriptor, + ) -> Option { + let pk_filters = primary_key_filters?; + if pk_filters.is_empty() { + return None; + } + + let metadata = read_format.metadata(); + if metadata.primary_key.is_empty() { + return None; + } + + // Only flat format with dictionary-encoded PKs supports PK prefiltering. + let flat_format = read_format.as_flat()?; + if !flat_format.raw_batch_has_primary_key_dictionary() { + return None; + } + + // Compute PK-only projection mask. + let num_parquet_columns = parquet_schema.num_columns(); + let pk_index = primary_key_column_index(num_parquet_columns); + let pk_projection = ProjectionMask::roots(parquet_schema, [pk_index]); + + // The PK column is the only column in the projection, so its index is 0. + let pk_column_index = 0; + + Some(Self { + pk_projection, + pk_column_index, + codec: Arc::clone(codec), + metadata: metadata.clone(), + pk_filters: Arc::clone(pk_filters), + }) } - let first_key = keys.value(0); - if first_key != keys.value(keys.len() - 1) { - return Ok(None); + /// Builds a [PrefilterContext] for a specific row group. + pub(crate) fn build(&self) -> PrefilterContext { + // Parquet PK prefilter always supports the partition column. Only + // PartitionTreeMemtable skips it after partition pruning. + let pk_filter = + self.codec + .primary_key_filter(&self.metadata, Arc::clone(&self.pk_filters), false); + let pk_filter = Box::new(CachedPrimaryKeyFilter::new(pk_filter)); + PrefilterContext { + pk_filter, + pk_projection: self.pk_projection.clone(), + pk_column_index: self.pk_column_index, + } + } +} + +/// Result of prefiltering a row group. +pub(crate) struct PrefilterResult { + /// Refined row selection after prefiltering. + pub(crate) refined_selection: RowSelection, + /// Number of rows filtered out by prefiltering. + pub(crate) filtered_rows: usize, +} + +/// Executes prefiltering on a row group. +/// +/// Reads only the prefilter columns (currently the PK dictionary column), +/// applies filters, and returns a refined [RowSelection]. +pub(crate) async fn execute_prefilter( + prefilter_ctx: &mut PrefilterContext, + reader_builder: &RowGroupReaderBuilder, + build_ctx: &RowGroupBuildContext<'_>, +) -> Result { + // Reads PK column only. + let mut pk_stream = reader_builder + .build_with_projection( + build_ctx.row_group_idx, + build_ctx.row_selection.clone(), + prefilter_ctx.pk_projection.clone(), + build_ctx.fetch_metrics, + ) + .await?; + + // Applies PK filter to each batch and collect matching row ranges. + let mut matched_row_ranges: Vec> = Vec::new(); + let mut row_offset = 0; + let mut rows_before_filter = 0usize; + + while let Some(batch_result) = pk_stream.next().await { + let batch = batch_result.context(ReadParquetSnafu { + path: reader_builder.file_path(), + })?; + let batch_num_rows = batch.num_rows(); + if batch_num_rows == 0 { + continue; + } + rows_before_filter += batch_num_rows; + + let ranges = matching_row_ranges_by_primary_key( + &batch, + prefilter_ctx.pk_column_index, + prefilter_ctx.pk_filter.as_mut(), + )?; + matched_row_ranges.extend( + ranges + .into_iter() + .map(|range| (range.start + row_offset)..(range.end + row_offset)), + ); + row_offset += batch_num_rows; } - Ok(Some(pk_values.value(first_key as usize))) + // Converts matched ranges to RowSelection. + let rows_selected: usize = matched_row_ranges.iter().map(|r| r.end - r.start).sum(); + let filtered_rows = rows_before_filter.saturating_sub(rows_selected); + + let refined_selection = if rows_selected == 0 { + RowSelection::from(vec![]) + } else { + // Build the prefilter selection relative to the yielded rows + // (not total_rows), since matched_row_ranges are offsets within + // the rows actually read from the stream. + let prefilter_selection = + row_selection_from_row_ranges_exact(matched_row_ranges.into_iter(), rows_before_filter); + + // Use and_then to apply prefilter selection within the context + // of the original selection, since prefilter offsets are relative + // to the original selection's selected rows. + match &build_ctx.row_selection { + Some(original) => original.and_then(&prefilter_selection), + None => prefilter_selection, + } + }; + + Ok(PrefilterResult { + refined_selection, + filtered_rows, + }) } #[cfg(test)] @@ -245,175 +344,14 @@ mod tests { use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; - use api::v1::SemanticType; use common_recordbatch::filter::SimpleFilterEvaluator; use datafusion_expr::{col, lit}; - use datatypes::arrow::array::{ - ArrayRef, BinaryArray, DictionaryArray, TimestampMillisecondArray, UInt8Array, UInt32Array, - UInt64Array, - }; - use datatypes::arrow::datatypes::{Schema, UInt32Type}; - use datatypes::arrow::record_batch::RecordBatch; - use datatypes::prelude::ConcreteDataType; - use mito_codec::row_converter::{PrimaryKeyFilter, build_primary_key_codec}; + use mito_codec::row_converter::PrimaryKeyFilter; use store_api::codec::PrimaryKeyEncoding; - use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder}; - use store_api::storage::ColumnSchema; use super::*; - use crate::sst::internal_fields; use crate::sst::parquet::format::ReadFormat; - use crate::test_util::sst_util::{ - new_primary_key, sst_region_metadata, sst_region_metadata_with_encoding, - }; - - fn new_test_filters(exprs: &[datafusion_expr::Expr]) -> Vec { - exprs - .iter() - .filter_map(SimpleFilterEvaluator::try_new) - .collect() - } - - fn expected_metadata_with_reused_tag_name( - old_metadata: &RegionMetadata, - ) -> Arc { - let mut builder = RegionMetadataBuilder::new(old_metadata.region_id); - builder - .push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new( - "tag_0".to_string(), - ConcreteDataType::string_datatype(), - true, - ), - semantic_type: SemanticType::Tag, - column_id: 10, - }) - .push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new( - "tag_1".to_string(), - ConcreteDataType::string_datatype(), - true, - ), - semantic_type: SemanticType::Tag, - column_id: 1, - }) - .push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new( - "field_0".to_string(), - ConcreteDataType::uint64_datatype(), - true, - ), - semantic_type: SemanticType::Field, - column_id: 2, - }) - .push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new( - "ts".to_string(), - ConcreteDataType::timestamp_millisecond_datatype(), - false, - ), - semantic_type: SemanticType::Timestamp, - column_id: 3, - }) - .primary_key(vec![10, 1]); - - Arc::new(builder.build().unwrap()) - } - - fn new_raw_batch_with_metadata( - metadata: Arc, - primary_keys: &[&[u8]], - field_values: &[u64], - ) -> RecordBatch { - assert_eq!(primary_keys.len(), field_values.len()); - - let arrow_schema = metadata.schema.arrow_schema(); - let field_column = arrow_schema - .field(arrow_schema.index_of("field_0").unwrap()) - .clone(); - let time_index_column = arrow_schema - .field(arrow_schema.index_of("ts").unwrap()) - .clone(); - let mut fields = vec![field_column, time_index_column]; - fields.extend( - internal_fields() - .into_iter() - .map(|field| field.as_ref().clone()), - ); - let schema = Arc::new(Schema::new(fields)); - - let mut dict_values = Vec::new(); - let mut keys = Vec::with_capacity(primary_keys.len()); - for pk in primary_keys { - let key = dict_values - .iter() - .position(|existing: &&[u8]| existing == pk) - .unwrap_or_else(|| { - dict_values.push(*pk); - dict_values.len() - 1 - }); - keys.push(key as u32); - } - - let pk_array: ArrayRef = Arc::new(DictionaryArray::::new( - UInt32Array::from(keys), - Arc::new(BinaryArray::from_iter_values(dict_values.iter().copied())), - )); - - RecordBatch::try_new( - schema, - vec![ - Arc::new(UInt64Array::from(field_values.to_vec())), - Arc::new(TimestampMillisecondArray::from_iter_values( - 0..primary_keys.len() as i64, - )), - pk_array, - Arc::new(UInt64Array::from(vec![1; primary_keys.len()])), - Arc::new(UInt8Array::from(vec![1; primary_keys.len()])), - ], - ) - .unwrap() - } - - fn new_raw_batch(primary_keys: &[&[u8]], field_values: &[u64]) -> RecordBatch { - new_raw_batch_with_metadata(Arc::new(sst_region_metadata()), primary_keys, field_values) - } - - fn field_values(batch: &RecordBatch) -> Vec { - batch - .column(0) - .as_any() - .downcast_ref::() - .unwrap() - .values() - .to_vec() - } - - #[test] - fn test_retain_usable_primary_key_filters_skips_non_tag_filters() { - let metadata = Arc::new(sst_region_metadata()); - let mut filters = - new_test_filters(&[col("field_0").eq(lit(1_u64)), col("ts").gt(lit(0_i64))]); - - retain_usable_primary_key_filters(&metadata, None, &mut filters); - - assert!(filters.is_empty()); - } - - #[test] - fn test_retain_usable_primary_key_filters_skips_reused_expected_tag_name() { - let metadata = Arc::new(sst_region_metadata()); - let expected_metadata = expected_metadata_with_reused_tag_name(&metadata); - let mut filters = new_test_filters(&[col("tag_0").eq(lit("b"))]); - - retain_usable_primary_key_filters( - &metadata, - Some(expected_metadata.as_ref()), - &mut filters, - ); - - assert!(filters.is_empty()); - } + use crate::test_util::sst_util::{new_primary_key, sst_region_metadata_with_encoding}; #[test] fn test_is_usable_primary_key_filter_skips_legacy_primary_key_batches() { @@ -435,52 +373,16 @@ mod tests { } #[test] - fn test_prefilter_primary_key_drops_single_dictionary_batch() { - let metadata = Arc::new(sst_region_metadata()); - let filters = Arc::new(new_test_filters(&[col("tag_0").eq(lit("b"))])); - let mut primary_key_filter = - build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters); - let pk_a = new_primary_key(&["a", "x"]); - let batch = new_raw_batch(&[pk_a.as_slice(), pk_a.as_slice()], &[10, 11]); + fn test_is_usable_primary_key_filter_supports_partition_column_by_default() { + let metadata = Arc::new(sst_region_metadata_with_encoding( + PrimaryKeyEncoding::Sparse, + )); + let filter = SimpleFilterEvaluator::try_new( + &col(store_api::metric_engine_consts::DATA_SCHEMA_TABLE_ID_COLUMN_NAME).eq(lit(1_u32)), + ) + .unwrap(); - let filtered = - prefilter_flat_batch_by_primary_key(batch, primary_key_filter.as_mut()).unwrap(); - - assert!(filtered.is_none()); - } - - #[test] - fn test_prefilter_primary_key_builds_mask_for_fragmented_matches() { - let metadata = Arc::new(sst_region_metadata()); - let filters = Arc::new(new_test_filters(&[col("tag_0") - .eq(lit("a")) - .or(col("tag_0").eq(lit("c")))])); - let mut primary_key_filter = - build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters); - let pk_a = new_primary_key(&["a", "x"]); - let pk_b = new_primary_key(&["b", "x"]); - let pk_c = new_primary_key(&["c", "x"]); - let pk_d = new_primary_key(&["d", "x"]); - let batch = new_raw_batch( - &[ - pk_a.as_slice(), - pk_a.as_slice(), - pk_b.as_slice(), - pk_b.as_slice(), - pk_c.as_slice(), - pk_c.as_slice(), - pk_d.as_slice(), - pk_d.as_slice(), - ], - &[10, 11, 12, 13, 14, 15, 16, 17], - ); - - let filtered = prefilter_flat_batch_by_primary_key(batch, primary_key_filter.as_mut()) - .unwrap() - .unwrap(); - - assert_eq!(filtered.num_rows(), 4); - assert_eq!(field_values(&filtered), vec![10, 11, 14, 15]); + assert!(is_usable_primary_key_filter(&metadata, None, &filter)); } struct CountingPrimaryKeyFilter { @@ -489,9 +391,9 @@ mod tests { } impl PrimaryKeyFilter for CountingPrimaryKeyFilter { - fn matches(&mut self, pk: &[u8]) -> bool { + fn matches(&mut self, pk: &[u8]) -> mito_codec::error::Result { self.hits.fetch_add(1, Ordering::Relaxed); - pk == self.expected.as_slice() + Ok(pk == self.expected.as_slice()) } } @@ -504,25 +406,14 @@ mod tests { expected: expected.clone(), })); - assert!(filter.matches(expected.as_slice())); - assert!(filter.matches(expected.as_slice())); - assert!(!filter.matches(new_primary_key(&["b", "x"]).as_slice())); + assert!(filter.matches(expected.as_slice()).unwrap()); + assert!(filter.matches(expected.as_slice()).unwrap()); + assert!( + !filter + .matches(new_primary_key(&["b", "x"]).as_slice()) + .unwrap() + ); assert_eq!(hits.load(Ordering::Relaxed), 2); } - - #[test] - fn test_batch_single_primary_key() { - let pk_a = new_primary_key(&["a", "x"]); - let pk_b = new_primary_key(&["b", "x"]); - - let batch = new_raw_batch(&[pk_a.as_slice(), pk_a.as_slice()], &[10, 11]); - assert_eq!( - batch_single_primary_key(&batch).unwrap(), - Some(pk_a.as_slice()) - ); - - let batch = new_raw_batch(&[pk_a.as_slice(), pk_b.as_slice()], &[10, 11]); - assert_eq!(batch_single_primary_key(&batch).unwrap(), None); - } } diff --git a/src/mito2/src/sst/parquet/reader.rs b/src/mito2/src/sst/parquet/reader.rs index f152c97075..8832cd4a16 100644 --- a/src/mito2/src/sst/parquet/reader.rs +++ b/src/mito2/src/sst/parquet/reader.rs @@ -76,6 +76,9 @@ use crate::sst::parquet::file_range::{ }; use crate::sst::parquet::format::{ReadFormat, need_override_sequence}; use crate::sst::parquet::metadata::MetadataLoader; +use crate::sst::parquet::prefilter::{ + PrefilterContextBuilder, execute_prefilter, is_usable_primary_key_filter, +}; use crate::sst::parquet::row_group::ParquetFetchMetrics; use crate::sst::parquet::row_selection::RowGroupSelection; use crate::sst::parquet::stats::RowGroupPruningStats; @@ -459,16 +462,6 @@ impl ParquetReaderBuilder { ArrowReaderMetadata::try_new(parquet_meta.clone(), arrow_reader_options) .context(ReadDataPartSnafu)?; - let reader_builder = RowGroupReaderBuilder { - file_handle: self.file_handle.clone(), - file_path, - parquet_meta, - arrow_metadata, - object_store: self.object_store.clone(), - projection: projection_mask, - cache_strategy: self.cache_strategy.clone(), - }; - let filters = if let Some(predicate) = &self.predicate { predicate .exprs() @@ -493,6 +486,33 @@ impl ParquetReaderBuilder { let codec = build_primary_key_codec(read_format.metadata()); + // Extract primary key filters from precomputed filter contexts for prefiltering. + let primary_key_filters = { + let pk_filters = filters + .iter() + .filter_map(SimpleFilterContext::primary_key_prefilter) + .collect::>(); + (!pk_filters.is_empty()).then_some(Arc::new(pk_filters)) + }; + + let prefilter_builder = PrefilterContextBuilder::new( + &read_format, + &codec, + primary_key_filters.as_ref(), + parquet_meta.file_metadata().schema_descr(), + ); + + let reader_builder = RowGroupReaderBuilder { + file_handle: self.file_handle.clone(), + file_path, + parquet_meta, + arrow_metadata, + object_store: self.object_store.clone(), + projection: projection_mask, + cache_strategy: self.cache_strategy.clone(), + prefilter_builder, + }; + let partition_filter = self.build_partition_filter(&read_format, &prune_schema)?; let context = FileRangeContext::new( @@ -1658,6 +1678,25 @@ pub(crate) struct RowGroupReaderBuilder { projection: ProjectionMask, /// Cache. cache_strategy: CacheStrategy, + /// Pre-built prefilter state. `None` if prefiltering is not applicable. + prefilter_builder: Option, +} + +/// Context passed to [RowGroupReaderBuilder::build()] carrying all information +/// needed for prefiltering decisions. +pub(crate) struct RowGroupBuildContext<'a> { + /// Simple filters pushed down. Used by prefilter on other columns. + #[allow(dead_code)] + pub(crate) filters: &'a [SimpleFilterContext], + /// Whether to skip field filters. Used by prefilter on other columns. + #[allow(dead_code)] + pub(crate) skip_fields: bool, + /// Index of the row group to read. + pub(crate) row_group_idx: usize, + /// Row selection for the row group. `None` means all rows. + pub(crate) row_selection: Option, + /// Metrics for tracking fetch operations. + pub(crate) fetch_metrics: Option<&'a ParquetFetchMetrics>, } impl RowGroupReaderBuilder { @@ -1679,11 +1718,58 @@ impl RowGroupReaderBuilder { &self.cache_strategy } + pub(crate) fn has_flat_primary_key_prefilter(&self) -> bool { + self.prefilter_builder.is_some() + } + /// Builds a [ParquetRecordBatchStream] to read the row group at `row_group_idx`. + /// + /// If prefiltering is applicable (based on `build_ctx`), this performs a two-phase read: + /// 1. Reads only the prefilter columns (e.g. PK column), applies filters to get a refined row selection + /// 2. Reads the full projection with the refined row selection pub(crate) async fn build( + &self, + build_ctx: RowGroupBuildContext<'_>, + ) -> Result> { + let prefilter_ctx = self.prefilter_builder.as_ref().map(|b| b.build()); + + let Some(mut prefilter_ctx) = prefilter_ctx else { + // No prefilter applicable, build stream with full projection. + return self + .build_with_projection( + build_ctx.row_group_idx, + build_ctx.row_selection, + self.projection.clone(), + build_ctx.fetch_metrics, + ) + .await; + }; + + let prefilter_start = Instant::now(); + let prefilter_result = execute_prefilter(&mut prefilter_ctx, self, &build_ctx).await?; + if let Some(metrics) = build_ctx.fetch_metrics { + let mut data = metrics.data.lock().unwrap(); + data.prefilter_cost += prefilter_start.elapsed(); + data.prefilter_filtered_rows += prefilter_result.filtered_rows; + } + + let refined_selection = Some(prefilter_result.refined_selection); + + self.build_with_projection( + build_ctx.row_group_idx, + refined_selection, + self.projection.clone(), + build_ctx.fetch_metrics, + ) + .await + } + + /// Builds a [ParquetRecordBatchStream] with a custom projection mask. + pub(crate) async fn build_with_projection( &self, row_group_idx: usize, row_selection: Option, + projection: ProjectionMask, fetch_metrics: Option<&ParquetFetchMetrics>, ) -> Result> { // Create async file reader with caching support. @@ -1704,7 +1790,7 @@ impl RowGroupReaderBuilder { ); builder = builder .with_row_groups(vec![row_group_idx]) - .with_projection(self.projection.clone()) + .with_projection(projection) .with_batch_size(DEFAULT_READ_BATCH_SIZE); if let Some(selection) = row_selection { @@ -1739,6 +1825,8 @@ pub(crate) struct SimpleFilterContext { semantic_type: SemanticType, /// The data type of the column. data_type: ConcreteDataType, + /// Whether this filter can be applied by flat parquet primary-key prefiltering. + usable_primary_key_filter: bool, } impl SimpleFilterContext { @@ -1752,6 +1840,10 @@ impl SimpleFilterContext { expr: &Expr, ) -> Option { let filter = SimpleFilterEvaluator::try_new(expr)?; + // Parquet PK prefilter always supports the partition column. Only + // PartitionTreeMemtable skips it after partition pruning. + let usable_primary_key_filter = + is_usable_primary_key_filter(sst_meta, expected_meta, &filter); let (column_metadata, maybe_filter) = match expected_meta { Some(meta) => { // Gets the column metadata from the expected metadata. @@ -1782,11 +1874,15 @@ impl SimpleFilterContext { } }; + let usable_primary_key_filter = + matches!(maybe_filter, MaybeFilter::Filter(_)) && usable_primary_key_filter; + Some(Self { filter: maybe_filter, column_id: column_metadata.column_id, semantic_type: column_metadata.semantic_type, data_type: column_metadata.column_schema.data_type.clone(), + usable_primary_key_filter, }) } @@ -1809,6 +1905,23 @@ impl SimpleFilterContext { pub(crate) fn data_type(&self) -> &ConcreteDataType { &self.data_type } + + /// Returns whether this filter is eligible for flat parquet PK prefiltering. + pub(crate) fn usable_primary_key_filter(&self) -> bool { + self.usable_primary_key_filter + } + + /// Returns the filter evaluator when it is eligible for PK prefiltering. + pub(crate) fn primary_key_prefilter(&self) -> Option { + if !self.usable_primary_key_filter { + return None; + } + + match &self.filter { + MaybeFilter::Filter(filter) => Some(filter.clone()), + MaybeFilter::Matched | MaybeFilter::Pruned => None, + } + } } /// Prune a column by its default value. @@ -1856,17 +1969,17 @@ impl ParquetReader { return Ok(None); }; + let skip_fields = self.context.should_skip_fields(row_group_idx); let parquet_reader = self .context .reader_builder() - .build( + .build(self.context.build_context( row_group_idx, Some(row_selection), Some(&self.fetch_metrics), - ) + skip_fields, + )) .await?; - - let skip_fields = self.context.should_skip_fields(row_group_idx); self.reader = Some(FlatPruneReader::new_with_row_group_reader( self.context.clone(), FlatRowGroupReader::new(self.context.clone(), parquet_reader), @@ -1889,11 +2002,16 @@ impl ParquetReader { debug_assert!(context.read_format().as_flat().is_some()); let fetch_metrics = ParquetFetchMetrics::default(); let reader = if let Some((row_group_idx, row_selection)) = selection.pop_first() { + let skip_fields = context.should_skip_fields(row_group_idx); let parquet_reader = context .reader_builder() - .build(row_group_idx, Some(row_selection), Some(&fetch_metrics)) + .build(context.build_context( + row_group_idx, + Some(row_selection), + Some(&fetch_metrics), + skip_fields, + )) .await?; - let skip_fields = context.should_skip_fields(row_group_idx); Some(FlatPruneReader::new_with_row_group_reader( context.clone(), FlatRowGroupReader::new(context.clone(), parquet_reader), @@ -2111,11 +2229,15 @@ mod tests { use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ ColumnarValue, Expr, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, Volatility, + col, lit, }; use datatypes::arrow::array::{ArrayRef, Int64Array}; use datatypes::arrow::record_batch::RecordBatch; + use datatypes::prelude::ConcreteDataType; + use datatypes::schema::ColumnSchema; use object_store::services::Memory; use parquet::arrow::ArrowWriter; + use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder}; use store_api::region_request::PathType; use table::predicate::Predicate; @@ -2207,4 +2329,80 @@ mod tests { assert!(!selection.is_empty()); } + + fn expected_metadata_with_reused_tag_name( + old_metadata: &RegionMetadata, + ) -> Arc { + let mut builder = RegionMetadataBuilder::new(old_metadata.region_id); + builder + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "tag_0".to_string(), + ConcreteDataType::string_datatype(), + true, + ), + semantic_type: SemanticType::Tag, + column_id: 10, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "tag_1".to_string(), + ConcreteDataType::string_datatype(), + true, + ), + semantic_type: SemanticType::Tag, + column_id: 1, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "field_0".to_string(), + ConcreteDataType::uint64_datatype(), + true, + ), + semantic_type: SemanticType::Field, + column_id: 2, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "ts".to_string(), + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 3, + }) + .primary_key(vec![10, 1]); + + Arc::new(builder.build().unwrap()) + } + + #[test] + fn test_simple_filter_context_marks_usable_primary_key_filter() { + let metadata: RegionMetadataRef = Arc::new(sst_region_metadata()); + let ctx = + SimpleFilterContext::new_opt(&metadata, None, &col("tag_0").eq(lit("a"))).unwrap(); + + assert!(ctx.usable_primary_key_filter()); + assert!(ctx.primary_key_prefilter().is_some()); + } + + #[test] + fn test_simple_filter_context_skips_non_usable_primary_key_filter() { + let metadata: RegionMetadataRef = Arc::new(sst_region_metadata()); + + let field_ctx = + SimpleFilterContext::new_opt(&metadata, None, &col("field_0").eq(lit(1_u64))).unwrap(); + assert!(!field_ctx.usable_primary_key_filter()); + assert!(field_ctx.primary_key_prefilter().is_none()); + + let expected_metadata = expected_metadata_with_reused_tag_name(metadata.as_ref()); + let mismatched_ctx = SimpleFilterContext::new_opt( + &metadata, + Some(expected_metadata.as_ref()), + &col("tag_0").eq(lit("a")), + ) + .unwrap(); + assert!(!mismatched_ctx.usable_primary_key_filter()); + assert!(mismatched_ctx.primary_key_prefilter().is_none()); + } } diff --git a/src/mito2/src/sst/parquet/row_group.rs b/src/mito2/src/sst/parquet/row_group.rs index 38ef62c6b8..8822882c5d 100644 --- a/src/mito2/src/sst/parquet/row_group.rs +++ b/src/mito2/src/sst/parquet/row_group.rs @@ -48,12 +48,16 @@ pub struct ParquetFetchMetricsData { pub store_fetch_elapsed: std::time::Duration, /// Total elapsed time for fetching row groups. pub total_fetch_elapsed: std::time::Duration, + /// Elapsed time for prefilter execution. + pub prefilter_cost: std::time::Duration, + /// Number of rows filtered out by prefiltering. + pub prefilter_filtered_rows: usize, } impl ParquetFetchMetricsData { /// Returns true if the metrics are empty (contain no meaningful data). fn is_empty(&self) -> bool { - self.total_fetch_elapsed.is_zero() + self.total_fetch_elapsed.is_zero() && self.prefilter_cost.is_zero() } } @@ -84,6 +88,8 @@ impl std::fmt::Debug for ParquetFetchMetrics { write_cache_fetch_elapsed, store_fetch_elapsed, total_fetch_elapsed, + prefilter_cost, + prefilter_filtered_rows, } = *data; write!(f, "{{")?; @@ -142,6 +148,16 @@ impl std::fmt::Debug for ParquetFetchMetrics { if !store_fetch_elapsed.is_zero() { write!(f, ", \"store_fetch_elapsed\":\"{:?}\"", store_fetch_elapsed)?; } + if !prefilter_cost.is_zero() { + write!(f, ", \"prefilter_cost\":\"{:?}\"", prefilter_cost)?; + } + if prefilter_filtered_rows > 0 { + write!( + f, + ", \"prefilter_filtered_rows\":{}", + prefilter_filtered_rows + )?; + } write!(f, "}}") } @@ -169,6 +185,8 @@ impl ParquetFetchMetrics { write_cache_fetch_elapsed, store_fetch_elapsed, total_fetch_elapsed, + prefilter_cost, + prefilter_filtered_rows, } = *other.data.lock().unwrap(); let mut data = self.data.lock().unwrap(); @@ -185,6 +203,8 @@ impl ParquetFetchMetrics { data.write_cache_fetch_elapsed += write_cache_fetch_elapsed; data.store_fetch_elapsed += store_fetch_elapsed; data.total_fetch_elapsed += total_fetch_elapsed; + data.prefilter_cost += prefilter_cost; + data.prefilter_filtered_rows += prefilter_filtered_rows; } } diff --git a/src/mito2/src/sst/parquet/row_selection.rs b/src/mito2/src/sst/parquet/row_selection.rs index 595f1d352a..763e244ef2 100644 --- a/src/mito2/src/sst/parquet/row_selection.rs +++ b/src/mito2/src/sst/parquet/row_selection.rs @@ -554,11 +554,43 @@ fn intersect_row_selections(left: &RowSelection, right: &RowSelection) -> RowSel /// or if there's a gap that requires skipping rows. It handles both "select" and "skip" actions, /// optimizing the list of selectors by merging contiguous actions of the same type. /// +/// The returned selection intentionally stops at the end of the last matched range and may omit a +/// trailing `skip` that would extend it to `total_row_count`. That is fine when the selection is +/// used directly by the parquet reader, which simply stops once the selectors are exhausted. +/// /// Note: overlapping ranges are not supported and will result in an incorrect selection. pub(crate) fn row_selection_from_row_ranges( row_ranges: impl Iterator>, total_row_count: usize, ) -> RowSelection { + let (selectors, _) = build_selectors_from_row_ranges(row_ranges, total_row_count); + RowSelection::from(selectors) +} + +/// Like [`row_selection_from_row_ranges`] but guarantees the resulting selection +/// covers exactly `total_row_count` rows by appending a trailing skip if needed. +/// +/// Required when the result is used as the inner operand of [`RowSelection::and_then`], because +/// `and_then` expects the inner selection to account for every row selected by the outer one. +pub(crate) fn row_selection_from_row_ranges_exact( + row_ranges: impl Iterator>, + total_row_count: usize, +) -> RowSelection { + let (mut selectors, last_processed_end) = + build_selectors_from_row_ranges(row_ranges, total_row_count); + if last_processed_end < total_row_count { + // Preserve the full logical length of the selection even when the final rows are all + // filtered out. Without this trailing skip, `and_then` sees an undersized inner + // selection and panics. + add_or_merge_selector(&mut selectors, total_row_count - last_processed_end, true); + } + RowSelection::from(selectors) +} + +fn build_selectors_from_row_ranges( + row_ranges: impl Iterator>, + total_row_count: usize, +) -> (Vec, usize) { let mut selectors: Vec = Vec::new(); let mut last_processed_end = 0; @@ -572,7 +604,7 @@ pub(crate) fn row_selection_from_row_ranges( last_processed_end = end; } - RowSelection::from(selectors) + (selectors, last_processed_end) } /// Converts an iterator of sorted row IDs into a `RowSelection`. @@ -707,6 +739,56 @@ mod tests { assert_eq!(selection, expected); } + #[test] + fn test_exact_single_range_with_trailing_skip() { + let selection = row_selection_from_row_ranges_exact(Some(0..3).into_iter(), 6); + let expected = RowSelection::from(vec![RowSelector::select(3), RowSelector::skip(3)]); + assert_eq!(selection, expected); + assert_eq!(selection.row_count(), 3); + } + + #[test] + fn test_exact_non_contiguous_ranges() { + let ranges = [1..3, 5..8]; + let selection = row_selection_from_row_ranges_exact(ranges.iter().cloned(), 10); + let expected = RowSelection::from(vec![ + RowSelector::skip(1), + RowSelector::select(2), + RowSelector::skip(2), + RowSelector::select(3), + RowSelector::skip(2), + ]); + assert_eq!(selection, expected); + assert_eq!(selection.row_count(), 5); + } + + #[test] + fn test_exact_empty_ranges() { + let selection = row_selection_from_row_ranges_exact([].iter().cloned(), 10); + let expected = RowSelection::from(vec![RowSelector::skip(10)]); + assert_eq!(selection, expected); + assert_eq!(selection.row_count(), 0); + } + + #[test] + fn test_exact_range_covers_all_rows() { + let selection = row_selection_from_row_ranges_exact(Some(0..10).into_iter(), 10); + let expected = RowSelection::from(vec![RowSelector::select(10)]); + assert_eq!(selection, expected); + assert_eq!(selection.row_count(), 10); + } + + #[test] + fn test_exact_compatible_with_and_then() { + // Outer selects rows 0..6 out of 10. + let outer = RowSelection::from(vec![RowSelector::select(6), RowSelector::skip(4)]); + // Inner: within those 6 rows, select only rows 0..3. + let inner = row_selection_from_row_ranges_exact(Some(0..3).into_iter(), 6); + let result = outer.and_then(&inner); + let expected = RowSelection::from(vec![RowSelector::select(3), RowSelector::skip(7)]); + assert_eq!(result, expected); + } + #[test] fn test_row_ids_to_selection() { let row_ids = [1, 3, 5, 7, 9].into_iter(); From 6bd14aaf9f1d051c1967fdc09638f8d3abb82544 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Mon, 30 Mar 2026 16:22:37 +0800 Subject: [PATCH 07/20] fix: correct app-name for dashboard (#7884) --- Cargo.lock | 1 - src/cmd/Cargo.toml | 1 - src/cmd/src/cli.rs | 2 +- src/cmd/src/datanode.rs | 2 +- src/cmd/src/flownode.rs | 3 +-- src/cmd/src/frontend.rs | 2 +- src/cmd/src/metasrv.rs | 3 +-- src/cmd/src/standalone.rs | 3 +-- 8 files changed, 6 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0f3b58b373..0cab5067bb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2055,7 +2055,6 @@ dependencies = [ "common-time", "common-version", "common-wal", - "const_format", "datafusion", "datafusion-common", "datafusion-physical-plan", diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 003f1434f4..d547ec6e81 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -54,7 +54,6 @@ common-telemetry = { workspace = true, features = [ common-time.workspace = true common-version.workspace = true common-wal.workspace = true -const_format.workspace = true datafusion.workspace = true datafusion-common.workspace = true datafusion-physical-plan.workspace = true diff --git a/src/cmd/src/cli.rs b/src/cmd/src/cli.rs index 501b7b1615..84e797c291 100644 --- a/src/cmd/src/cli.rs +++ b/src/cmd/src/cli.rs @@ -21,7 +21,7 @@ use tracing_appender::non_blocking::WorkerGuard; use crate::options::GlobalOptions; use crate::{App, Result, error}; -pub const APP_NAME: &str = const_format::concatcp!(common_version::product_name(), "-cli"); +pub const APP_NAME: &str = "greptime-cli"; use async_trait::async_trait; pub struct Instance { diff --git a/src/cmd/src/datanode.rs b/src/cmd/src/datanode.rs index 2fadb1d210..06e2568b72 100644 --- a/src/cmd/src/datanode.rs +++ b/src/cmd/src/datanode.rs @@ -43,7 +43,7 @@ use crate::error::{ }; use crate::options::{GlobalOptions, GreptimeOptions}; -pub const APP_NAME: &str = const_format::concatcp!(common_version::product_name(), "-datanode"); +pub const APP_NAME: &str = "greptime-datanode"; type DatanodeOptions = GreptimeOptions; diff --git a/src/cmd/src/flownode.rs b/src/cmd/src/flownode.rs index 8e3277cdb3..3f8458cddf 100644 --- a/src/cmd/src/flownode.rs +++ b/src/cmd/src/flownode.rs @@ -35,7 +35,6 @@ use common_stat::ResourceStatImpl; use common_telemetry::info; use common_telemetry::logging::{DEFAULT_LOGGING_DIR, TracingOptions}; use common_version::{short_version, verbose_version}; -use const_format::concatcp; use flow::{ FlownodeBuilder, FlownodeInstance, FlownodeServiceBuilder, FrontendClient, FrontendInvoker, get_flow_auth_options, @@ -53,7 +52,7 @@ use crate::error::{ use crate::options::{GlobalOptions, GreptimeOptions}; use crate::{App, create_resource_limit_metrics, log_versions, maybe_activate_heap_profile}; -pub const APP_NAME: &str = concatcp!(common_version::product_name(), "-flownode"); +pub const APP_NAME: &str = "greptime-flownode"; type FlownodeOptions = GreptimeOptions; diff --git a/src/cmd/src/frontend.rs b/src/cmd/src/frontend.rs index 07c9f775f2..cb802791c5 100644 --- a/src/cmd/src/frontend.rs +++ b/src/cmd/src/frontend.rs @@ -72,7 +72,7 @@ pub struct Instance { _guard: Vec, } -pub const APP_NAME: &str = const_format::concatcp!(common_version::product_name(), "-frontend"); +pub const APP_NAME: &str = "greptime-frontend"; impl Instance { pub fn new(frontend: Frontend, _guard: Vec) -> Self { diff --git a/src/cmd/src/metasrv.rs b/src/cmd/src/metasrv.rs index dec9edc193..2ce5fb3a02 100644 --- a/src/cmd/src/metasrv.rs +++ b/src/cmd/src/metasrv.rs @@ -24,7 +24,6 @@ use common_meta::distributed_time_constants::init_distributed_time_constants; use common_telemetry::info; use common_telemetry::logging::{DEFAULT_LOGGING_DIR, TracingOptions}; use common_version::{short_version, verbose_version}; -use const_format::concatcp; use meta_srv::bootstrap::{MetasrvInstance, metasrv_builder}; use meta_srv::metasrv::BackendImpl; use snafu::ResultExt; @@ -36,7 +35,7 @@ use crate::{App, create_resource_limit_metrics, log_versions, maybe_activate_hea type MetasrvOptions = GreptimeOptions; -pub const APP_NAME: &str = concatcp!(common_version::product_name(), "-metasrv"); +pub const APP_NAME: &str = "greptime-metasrv"; pub struct Instance { instance: MetasrvInstance, diff --git a/src/cmd/src/standalone.rs b/src/cmd/src/standalone.rs index 196ff07c92..215bea0ec5 100644 --- a/src/cmd/src/standalone.rs +++ b/src/cmd/src/standalone.rs @@ -48,7 +48,6 @@ use common_telemetry::info; use common_telemetry::logging::{DEFAULT_LOGGING_DIR, TracingOptions}; use common_time::timezone::set_default_timezone; use common_version::{short_version, verbose_version}; -use const_format::concatcp; use datanode::config::DatanodeOptions; use datanode::datanode::{Datanode, DatanodeBuilder}; use datanode::region_server::RegionServer; @@ -76,7 +75,7 @@ use crate::error::{OtherSnafu, Result, StartFlownodeSnafu}; use crate::options::{GlobalOptions, GreptimeOptions}; use crate::{App, create_resource_limit_metrics, error, log_versions, maybe_activate_heap_profile}; -pub const APP_NAME: &str = concatcp!(common_version::product_name(), "-standalone"); +pub const APP_NAME: &str = "greptime-standalone"; #[derive(Parser)] pub struct Command { From a8fe6b5e44fc47ccbebd33ac08d8541efbc91490 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Mon, 30 Mar 2026 16:30:32 +0800 Subject: [PATCH 08/20] fix: allow auto type upscale conversion in trace ingestion (#7870) * fix: allow auto type upscale conversion in trace ingestion Signed-off-by: shuiyisong * fix: immediate return when parse fails Signed-off-by: shuiyisong * fix: typos Signed-off-by: shuiyisong * test: add integration test and fix Signed-off-by: shuiyisong * feat: add Int/Float/Bool to String conversion Signed-off-by: shuiyisong * refactor: coerce rows together Signed-off-by: shuiyisong * refactor: extract coerce Signed-off-by: shuiyisong * refactor: save clone Signed-off-by: shuiyisong * chore: add comments Signed-off-by: shuiyisong * refactor: unify in- and cross-batch check Signed-off-by: shuiyisong * chore: add comments Signed-off-by: shuiyisong --------- Signed-off-by: shuiyisong --- src/frontend/src/instance/otlp.rs | 233 ++++++++++++++- src/servers/src/otlp/trace.rs | 1 + src/servers/src/otlp/trace/coerce.rs | 343 ++++++++++++++++++++++ src/servers/src/otlp/trace/v1.rs | 265 +++++++++++++++-- src/servers/src/row_writer.rs | 29 ++ tests-integration/tests/http.rs | 418 +++++++++++++++++++++++++++ 6 files changed, 1258 insertions(+), 31 deletions(-) create mode 100644 src/servers/src/otlp/trace/coerce.rs diff --git a/src/frontend/src/instance/otlp.rs b/src/frontend/src/instance/otlp.rs index 52df274780..9b21f9924f 100644 --- a/src/frontend/src/instance/otlp.rs +++ b/src/frontend/src/instance/otlp.rs @@ -14,6 +14,8 @@ use std::sync::Arc; +use api::helper::ColumnDataTypeWrapper; +use api::v1::{ColumnDataType, RowInsertRequests}; use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; @@ -24,10 +26,14 @@ use opentelemetry_proto::tonic::collector::logs::v1::ExportLogsServiceRequest; use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; use otel_arrow_rust::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; use pipeline::{GreptimePipelineParams, PipelineWay}; -use servers::error::{self, AuthSnafu, Result as ServerResult}; +use servers::error::{self, AuthSnafu, CatalogSnafu, Result as ServerResult}; use servers::http::prom_store::PHYSICAL_TABLE_PARAM; use servers::interceptor::{OpenTelemetryProtocolInterceptor, OpenTelemetryProtocolInterceptorRef}; use servers::otlp; +use servers::otlp::trace::coerce::{ + coerce_value_data, is_supported_trace_coercion, resolve_new_trace_column_type, + trace_value_datatype, +}; use servers::query_handler::{OpenTelemetryProtocolHandler, PipelineHandlerRef}; use session::context::QueryContextRef; use snafu::ResultExt; @@ -124,7 +130,7 @@ impl OpenTelemetryProtocolHandler for Instance { let is_trace_v1_model = matches!(pipeline, PipelineWay::OtlpTraceDirectV1); - let (requests, rows) = otlp::trace::to_grpc_insert_requests( + let (mut requests, rows) = otlp::trace::to_grpc_insert_requests( request, pipeline, pipeline_params, @@ -136,6 +142,8 @@ impl OpenTelemetryProtocolHandler for Instance { OTLP_TRACES_ROWS.inc_by(rows as u64); if is_trace_v1_model { + self.reconcile_trace_column_types(&mut requests, &ctx) + .await?; self.handle_trace_inserts(requests, ctx) .await .map_err(BoxedError::new) @@ -200,3 +208,224 @@ impl OpenTelemetryProtocolHandler for Instance { Ok(outputs) } } + +impl Instance { + /// Picks the final datatype for one trace column. + /// + /// Existing table schema is authoritative when present. Otherwise we resolve the + /// request-local observed types using the shared trace coercion rules. + fn choose_trace_target_type( + observed_types: &[ColumnDataType], + existing_type: Option, + ) -> ServerResult> { + let Some(existing_type) = existing_type else { + return resolve_new_trace_column_type(observed_types.iter().copied()).map_err(|_| { + error::InvalidParameterSnafu { + reason: "unsupported trace type mix".to_string(), + } + .build() + }); + }; + + if observed_types.iter().copied().all(|request_type| { + request_type == existing_type + || is_supported_trace_coercion(request_type, existing_type) + }) { + Ok(Some(existing_type)) + } else { + error::InvalidParameterSnafu { + reason: "unsupported trace type mix".to_string(), + } + .fail() + } + } + + /// Coerce request column types and values to match the existing table schema + /// for compatible type pairs. Existing table schema wins when present; + /// otherwise the full request batch decides a stable target type. + async fn reconcile_trace_column_types( + &self, + requests: &mut RowInsertRequests, + ctx: &QueryContextRef, + ) -> ServerResult<()> { + let catalog = ctx.current_catalog(); + let schema = ctx.current_schema(); + + for req in &mut requests.inserts { + let table = self + .catalog_manager + .table(catalog, &schema, &req.table_name, None) + .await + .context(CatalogSnafu)?; + + let Some(rows) = req.rows.as_mut() else { + continue; + }; + + let table_schema = table.map(|table| table.schema()); + let mut pending_coercions = Vec::new(); + + for (col_idx, col_schema) in rows.schema.iter().enumerate() { + let Some(current_type) = ColumnDataType::try_from(col_schema.datatype).ok() else { + continue; + }; + + let mut observed_types = Vec::new(); + push_observed_trace_type(&mut observed_types, current_type); + + // Scan the full request first so the final type decision is not affected + // by row order inside the batch. + for row in &rows.rows { + let Some(value) = row + .values + .get(col_idx) + .and_then(|value| value.value_data.as_ref()) + else { + continue; + }; + + let Some(value_type) = trace_value_datatype(value) else { + continue; + }; + push_observed_trace_type(&mut observed_types, value_type); + } + + let existing_type = table_schema + .as_ref() + .and_then(|schema| schema.column_schema_by_name(&col_schema.column_name)) + .and_then(|table_col| { + ColumnDataTypeWrapper::try_from(table_col.data_type.clone()) + .ok() + .map(|wrapper| wrapper.datatype()) + }); + + if !observed_types + .iter() + .copied() + .any(is_trace_reconcile_candidate_type) + && existing_type + .map(|datatype| !is_trace_reconcile_candidate_type(datatype)) + .unwrap_or(true) + { + continue; + } + + // Decide the final type once per column, then rewrite all affected cells + // together in one row pass below. + let Some(target_type) = + Self::choose_trace_target_type(&observed_types, existing_type).map_err( + |_| { + enrich_trace_reconcile_error( + &req.table_name, + &col_schema.column_name, + &observed_types, + existing_type, + ) + }, + )? + else { + continue; + }; + + if observed_types + .iter() + .all(|observed| *observed == target_type) + && col_schema.datatype == target_type as i32 + { + continue; + } + + pending_coercions.push((col_idx, target_type, col_schema.column_name.clone())); + } + + if pending_coercions.is_empty() { + continue; + } + + // Update schema metadata before mutating row values so both stay in sync. + for (col_idx, target_type, ..) in &pending_coercions { + rows.schema[*col_idx].datatype = *target_type as i32; + } + + // Apply all pending column rewrites in one row pass. + for row in &mut rows.rows { + for (col_idx, target_type, column_name) in &pending_coercions { + let Some(value) = row.values.get_mut(*col_idx) else { + continue; + }; + let Some(request_type) = + value.value_data.as_ref().and_then(trace_value_datatype) + else { + continue; + }; + if request_type == *target_type { + continue; + } + + value.value_data = coerce_value_data( + &value.value_data, + *target_type, + request_type, + ) + .map_err(|_| { + error::InvalidParameterSnafu { + reason: format!( + "failed to coerce trace column '{}' in table '{}' from {:?} to {:?}", + column_name, req.table_name, request_type, target_type + ), + } + .build() + })?; + } + } + } + + Ok(()) + } +} + +fn enrich_trace_reconcile_error( + table_name: &str, + column_name: &str, + observed_types: &[ColumnDataType], + existing_type: Option, +) -> servers::error::Error { + let observed_types = observed_types + .iter() + .map(|datatype| format!("{datatype:?}")) + .collect::>() + .join(", "); + + error::InvalidParameterSnafu { + reason: match existing_type { + Some(existing_type) => format!( + "failed to reconcile trace column '{}' in table '{}' with observed types [{}] against existing {:?}", + column_name, table_name, observed_types, existing_type + ), + None => format!( + "failed to reconcile trace column '{}' in table '{}' with observed types [{}]", + column_name, table_name, observed_types + ), + }, + } + .build() +} + +/// Only these trace scalar types participate in reconciliation. Other column kinds +/// such as JSON and binary keep their original write path and schema checks. +fn is_trace_reconcile_candidate_type(datatype: ColumnDataType) -> bool { + matches!( + datatype, + ColumnDataType::String + | ColumnDataType::Boolean + | ColumnDataType::Int64 + | ColumnDataType::Float64 + ) +} + +/// Keeps the observed type list small without depending on enum ordering. +fn push_observed_trace_type(observed_types: &mut Vec, datatype: ColumnDataType) { + if !observed_types.contains(&datatype) { + observed_types.push(datatype); + } +} diff --git a/src/servers/src/otlp/trace.rs b/src/servers/src/otlp/trace.rs index b724bb1d22..ca56f9b868 100644 --- a/src/servers/src/otlp/trace.rs +++ b/src/servers/src/otlp/trace.rs @@ -13,6 +13,7 @@ // limitations under the License. pub mod attributes; +pub mod coerce; pub mod span; pub mod v0; pub mod v1; diff --git a/src/servers/src/otlp/trace/coerce.rs b/src/servers/src/otlp/trace/coerce.rs new file mode 100644 index 0000000000..febec0fda9 --- /dev/null +++ b/src/servers/src/otlp/trace/coerce.rs @@ -0,0 +1,343 @@ +// 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 api::v1::ColumnDataType; +use api::v1::value::ValueData; + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum TraceCoerceError { + Unsupported, +} + +// For now we support the following coercions: +// - Int64 to Float64 +// - Int64 to String +// - Float64 to String +// - Boolean to String +// The following coercions are supported with parse, which could fail: +// If fails, we will return TraceCoerceError::Unsupported. +// - String to Int64 +// - String to Float64 +// - String to Boolean +pub fn is_supported_trace_coercion( + request_type: ColumnDataType, + target_type: ColumnDataType, +) -> bool { + matches!( + (request_type, target_type), + (ColumnDataType::Int64, ColumnDataType::Float64) + | (ColumnDataType::Int64, ColumnDataType::String) + | (ColumnDataType::Float64, ColumnDataType::String) + | (ColumnDataType::Boolean, ColumnDataType::String) + | (ColumnDataType::String, ColumnDataType::Int64) + | (ColumnDataType::String, ColumnDataType::Float64) + | (ColumnDataType::String, ColumnDataType::Boolean) + ) +} + +pub fn coerce_value_data( + value: &Option, + target: ColumnDataType, + request_type: ColumnDataType, +) -> Result, TraceCoerceError> { + let Some(v) = value else { + return Ok(None); + }; + + let Some(value) = coerce_non_null_value(target, request_type, v) else { + return Err(TraceCoerceError::Unsupported); + }; + Ok(Some(value)) +} + +pub fn coerce_non_null_value( + target: ColumnDataType, + request_type: ColumnDataType, + value: &ValueData, +) -> Option { + match (request_type, target, value) { + (ColumnDataType::Int64, ColumnDataType::Float64, ValueData::I64Value(n)) => { + Some(ValueData::F64Value(*n as f64)) + } + (ColumnDataType::Int64, ColumnDataType::String, ValueData::I64Value(n)) => { + Some(ValueData::StringValue(n.to_string())) + } + (ColumnDataType::Float64, ColumnDataType::String, ValueData::F64Value(n)) => { + Some(ValueData::StringValue(n.to_string())) + } + (ColumnDataType::Boolean, ColumnDataType::String, ValueData::BoolValue(b)) => { + Some(ValueData::StringValue(b.to_string())) + } + (ColumnDataType::String, ColumnDataType::Int64, ValueData::StringValue(s)) => { + s.parse::().ok().map(ValueData::I64Value) + } + (ColumnDataType::String, ColumnDataType::Float64, ValueData::StringValue(s)) => { + s.parse::().ok().map(ValueData::F64Value) + } + (ColumnDataType::String, ColumnDataType::Boolean, ValueData::StringValue(s)) => { + s.parse::().ok().map(ValueData::BoolValue) + } + _ => None, + } +} + +pub fn trace_value_datatype(value: &ValueData) -> Option { + match value { + ValueData::StringValue(_) => Some(ColumnDataType::String), + ValueData::BoolValue(_) => Some(ColumnDataType::Boolean), + ValueData::I64Value(_) => Some(ColumnDataType::Int64), + ValueData::F64Value(_) => Some(ColumnDataType::Float64), + ValueData::BinaryValue(_) => Some(ColumnDataType::Binary), + _ => None, + } +} + +/// Resolves the final datatype for a new trace column when there is no existing +/// table schema to override the request-local observations. +pub fn resolve_new_trace_column_type( + observed_types: impl IntoIterator, +) -> Result, TraceCoerceError> { + let mut observed_types = observed_types.into_iter().collect::>(); + observed_types.dedup(); + + match observed_types.as_slice() { + [] => Ok(None), + [datatype] => Ok(Some(*datatype)), + [_, _] + if observed_types.contains(&ColumnDataType::String) + && observed_types.contains(&ColumnDataType::Boolean) => + { + Ok(Some(ColumnDataType::Boolean)) + } + [_, _] + if observed_types.contains(&ColumnDataType::String) + && observed_types.contains(&ColumnDataType::Int64) => + { + Ok(Some(ColumnDataType::Int64)) + } + [_, _] + if observed_types.contains(&ColumnDataType::String) + && observed_types.contains(&ColumnDataType::Float64) => + { + Ok(Some(ColumnDataType::Float64)) + } + [_, _] + if observed_types.contains(&ColumnDataType::Int64) + && observed_types.contains(&ColumnDataType::Float64) => + { + Ok(Some(ColumnDataType::Float64)) + } + _ => Err(TraceCoerceError::Unsupported), + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_coerce_int64_to_float64() { + let result = coerce_value_data( + &Some(ValueData::I64Value(42)), + ColumnDataType::Float64, + ColumnDataType::Int64, + ); + assert_eq!(result, Ok(Some(ValueData::F64Value(42.0)))); + } + + #[test] + fn test_coerce_string_to_int64() { + let result = coerce_value_data( + &Some(ValueData::StringValue("123".to_string())), + ColumnDataType::Int64, + ColumnDataType::String, + ); + assert_eq!(result, Ok(Some(ValueData::I64Value(123)))); + } + + #[test] + fn test_coerce_int64_to_string() { + let result = coerce_value_data( + &Some(ValueData::I64Value(123)), + ColumnDataType::String, + ColumnDataType::Int64, + ); + assert_eq!(result, Ok(Some(ValueData::StringValue("123".to_string())))); + } + + #[test] + fn test_coerce_string_to_float64() { + let result = coerce_value_data( + &Some(ValueData::StringValue("1.5".to_string())), + ColumnDataType::Float64, + ColumnDataType::String, + ); + assert_eq!(result, Ok(Some(ValueData::F64Value(1.5)))); + } + + #[test] + fn test_coerce_float64_to_string() { + let result = coerce_value_data( + &Some(ValueData::F64Value(1.5)), + ColumnDataType::String, + ColumnDataType::Float64, + ); + assert_eq!(result, Ok(Some(ValueData::StringValue("1.5".to_string())))); + } + + #[test] + fn test_coerce_string_to_boolean() { + let result = coerce_value_data( + &Some(ValueData::StringValue("true".to_string())), + ColumnDataType::Boolean, + ColumnDataType::String, + ); + assert_eq!(result, Ok(Some(ValueData::BoolValue(true)))); + + let result = coerce_value_data( + &Some(ValueData::StringValue("false".to_string())), + ColumnDataType::Boolean, + ColumnDataType::String, + ); + assert_eq!(result, Ok(Some(ValueData::BoolValue(false)))); + } + + #[test] + fn test_coerce_boolean_to_string() { + let result = coerce_value_data( + &Some(ValueData::BoolValue(true)), + ColumnDataType::String, + ColumnDataType::Boolean, + ); + assert_eq!(result, Ok(Some(ValueData::StringValue("true".to_string())))); + } + + #[test] + fn test_coerce_unparsable_string() { + let result = coerce_value_data( + &Some(ValueData::StringValue("not_a_number".to_string())), + ColumnDataType::Int64, + ColumnDataType::String, + ); + assert_eq!(result, Err(TraceCoerceError::Unsupported)); + } + + #[test] + fn test_coerce_float64_to_int64_not_supported() { + let result = coerce_value_data( + &Some(ValueData::F64Value(1.5)), + ColumnDataType::Int64, + ColumnDataType::Float64, + ); + assert_eq!(result, Err(TraceCoerceError::Unsupported)); + } + + #[test] + fn test_coerce_none_value() { + let result = coerce_value_data(&None, ColumnDataType::Float64, ColumnDataType::Int64); + assert_eq!(result, Ok(None)); + } + + #[test] + fn test_is_supported_trace_coercion() { + assert!(is_supported_trace_coercion( + ColumnDataType::Int64, + ColumnDataType::Float64 + )); + assert!(is_supported_trace_coercion( + ColumnDataType::Int64, + ColumnDataType::String + )); + assert!(is_supported_trace_coercion( + ColumnDataType::Float64, + ColumnDataType::String + )); + assert!(is_supported_trace_coercion( + ColumnDataType::Boolean, + ColumnDataType::String + )); + assert!(is_supported_trace_coercion( + ColumnDataType::String, + ColumnDataType::Int64 + )); + assert!(is_supported_trace_coercion( + ColumnDataType::String, + ColumnDataType::Float64 + )); + assert!(is_supported_trace_coercion( + ColumnDataType::String, + ColumnDataType::Boolean + )); + assert!(!is_supported_trace_coercion( + ColumnDataType::Binary, + ColumnDataType::Json + )); + } + + #[test] + fn test_trace_value_datatype() { + assert_eq!( + trace_value_datatype(&ValueData::StringValue("x".to_string())), + Some(ColumnDataType::String) + ); + assert_eq!( + trace_value_datatype(&ValueData::BoolValue(true)), + Some(ColumnDataType::Boolean) + ); + assert_eq!( + trace_value_datatype(&ValueData::I64Value(1)), + Some(ColumnDataType::Int64) + ); + assert_eq!( + trace_value_datatype(&ValueData::F64Value(1.0)), + Some(ColumnDataType::Float64) + ); + assert_eq!( + trace_value_datatype(&ValueData::BinaryValue(vec![1_u8])), + Some(ColumnDataType::Binary) + ); + } + + #[test] + fn test_resolve_new_trace_column_type() { + assert_eq!( + resolve_new_trace_column_type([ColumnDataType::Int64]), + Ok(Some(ColumnDataType::Int64)) + ); + assert_eq!( + resolve_new_trace_column_type([ColumnDataType::String, ColumnDataType::Int64]), + Ok(Some(ColumnDataType::Int64)) + ); + assert_eq!( + resolve_new_trace_column_type([ColumnDataType::String, ColumnDataType::Float64]), + Ok(Some(ColumnDataType::Float64)) + ); + assert_eq!( + resolve_new_trace_column_type([ColumnDataType::String, ColumnDataType::Boolean]), + Ok(Some(ColumnDataType::Boolean)) + ); + assert_eq!( + resolve_new_trace_column_type([ColumnDataType::Int64, ColumnDataType::Float64]), + Ok(Some(ColumnDataType::Float64)) + ); + assert_eq!( + resolve_new_trace_column_type([ + ColumnDataType::String, + ColumnDataType::Int64, + ColumnDataType::Float64, + ]), + Err(TraceCoerceError::Unsupported) + ); + } +} diff --git a/src/servers/src/otlp/trace/v1.rs b/src/servers/src/otlp/trace/v1.rs index 86f8229769..11e986de04 100644 --- a/src/servers/src/otlp/trace/v1.rs +++ b/src/servers/src/otlp/trace/v1.rs @@ -230,7 +230,7 @@ fn write_trace_operations_to_row( Ok(()) } -fn write_attributes( +pub(crate) fn write_attributes( writer: &mut TableData, prefix: &str, attributes: Attributes, @@ -247,44 +247,40 @@ fn write_attributes( let key = format!("{}.{}", prefix, key_suffix); match attr.value.and_then(|v| v.value) { Some(OtlpValue::StringValue(v)) => { - row_writer::write_fields( - writer, - std::iter::once(make_string_column_data(&key, Some(v))), + // Keep the raw request value here. Mixed trace types are reconciled later + // in the frontend once we can also see the existing table schema. + writer.write_field_unchecked( + &key, + ColumnDataType::String, + Some(ValueData::StringValue(v)), row, - )?; + ); } Some(OtlpValue::BoolValue(v)) => { - row_writer::write_fields( - writer, - std::iter::once(make_column_data( - &key, - ColumnDataType::Boolean, - Some(ValueData::BoolValue(v)), - )), + // Do not coerce or promote types while building the request-local rows. + writer.write_field_unchecked( + &key, + ColumnDataType::Boolean, + Some(ValueData::BoolValue(v)), row, - )?; + ); } Some(OtlpValue::IntValue(v)) => { - row_writer::write_fields( - writer, - std::iter::once(make_column_data( - &key, - ColumnDataType::Int64, - Some(ValueData::I64Value(v)), - )), + // Preserving the original value avoids order-dependent behavior inside one batch. + writer.write_field_unchecked( + &key, + ColumnDataType::Int64, + Some(ValueData::I64Value(v)), row, - )?; + ); } Some(OtlpValue::DoubleValue(v)) => { - row_writer::write_fields( - writer, - std::iter::once(make_column_data( - &key, - ColumnDataType::Float64, - Some(ValueData::F64Value(v)), - )), + writer.write_field_unchecked( + &key, + ColumnDataType::Float64, + Some(ValueData::F64Value(v)), row, - )?; + ); } Some(OtlpValue::ArrayValue(v)) => row_writer::write_json( writer, @@ -315,3 +311,214 @@ fn write_attributes( Ok(()) } + +#[cfg(test)] +mod tests { + use api::v1::value::ValueData; + use opentelemetry_proto::tonic::common::v1::any_value::Value as OtlpValue; + use opentelemetry_proto::tonic::common::v1::{AnyValue, KeyValue}; + + use super::*; + use crate::otlp::trace::attributes::Attributes; + use crate::row_writer::TableData; + + fn make_kv(key: &str, value: OtlpValue) -> KeyValue { + KeyValue { + key: key.to_string(), + value: Some(AnyValue { value: Some(value) }), + } + } + + #[test] + fn test_keep_mixed_numeric_values_until_frontend_reconciliation() { + let mut writer = TableData::new(4, 2); + + let attrs1 = Attributes::from(vec![make_kv("val", OtlpValue::DoubleValue(1.5))]); + let mut row1 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs1, &mut row1).unwrap(); + writer.add_row(row1); + + let attrs2 = Attributes::from(vec![make_kv("val", OtlpValue::IntValue(42))]); + let mut row2 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs2, &mut row2).unwrap(); + writer.add_row(row2); + + let (schema, rows) = writer.into_schema_and_rows(); + + let col_idx = schema + .iter() + .position(|c| c.column_name == "attr.val") + .unwrap(); + assert_eq!(schema[col_idx].datatype, ColumnDataType::Float64 as i32); + + assert_eq!( + rows[0].values[col_idx].value_data, + Some(ValueData::F64Value(1.5)) + ); + assert_eq!( + rows[1].values[col_idx].value_data, + Some(ValueData::I64Value(42)) + ); + } + + #[test] + fn test_keep_mixed_string_and_int_values_until_frontend_reconciliation() { + let mut writer = TableData::new(4, 2); + + let attrs1 = Attributes::from(vec![make_kv("val", OtlpValue::IntValue(10))]); + let mut row1 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs1, &mut row1).unwrap(); + writer.add_row(row1); + + let attrs2 = Attributes::from(vec![make_kv( + "val", + OtlpValue::StringValue("20".to_string()), + )]); + let mut row2 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs2, &mut row2).unwrap(); + writer.add_row(row2); + + let (schema, rows) = writer.into_schema_and_rows(); + let col_idx = schema + .iter() + .position(|c| c.column_name == "attr.val") + .unwrap(); + assert_eq!(schema[col_idx].datatype, ColumnDataType::Int64 as i32); + assert_eq!( + rows[1].values[col_idx].value_data, + Some(ValueData::StringValue("20".to_string())) + ); + } + + #[test] + fn test_keep_first_seen_schema_until_frontend_reconciliation() { + let mut writer = TableData::new(4, 2); + + let attrs1 = Attributes::from(vec![make_kv( + "val", + OtlpValue::StringValue("10".to_string()), + )]); + let mut row1 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs1, &mut row1).unwrap(); + writer.add_row(row1); + + let attrs2 = Attributes::from(vec![make_kv("val", OtlpValue::IntValue(20))]); + let mut row2 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs2, &mut row2).unwrap(); + writer.add_row(row2); + + let (schema, rows) = writer.into_schema_and_rows(); + let col_idx = schema + .iter() + .position(|c| c.column_name == "attr.val") + .unwrap(); + assert_eq!(schema[col_idx].datatype, ColumnDataType::String as i32); + assert_eq!( + rows[0].values[col_idx].value_data, + Some(ValueData::StringValue("10".to_string())) + ); + assert_eq!( + rows[1].values[col_idx].value_data, + Some(ValueData::I64Value(20)) + ); + } + + #[test] + fn test_keep_mixed_string_and_float_values_until_frontend_reconciliation() { + let mut writer = TableData::new(4, 2); + + let attrs1 = Attributes::from(vec![make_kv("val", OtlpValue::DoubleValue(1.5))]); + let mut row1 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs1, &mut row1).unwrap(); + writer.add_row(row1); + + let attrs2 = Attributes::from(vec![make_kv( + "val", + OtlpValue::StringValue("1.5".to_string()), + )]); + let mut row2 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs2, &mut row2).unwrap(); + writer.add_row(row2); + + let (schema, rows) = writer.into_schema_and_rows(); + let col_idx = schema + .iter() + .position(|c| c.column_name == "attr.val") + .unwrap(); + assert_eq!(schema[col_idx].datatype, ColumnDataType::Float64 as i32); + assert_eq!( + rows[1].values[col_idx].value_data, + Some(ValueData::StringValue("1.5".to_string())) + ); + } + + #[test] + fn test_keep_mixed_string_and_bool_values_until_frontend_reconciliation() { + let mut writer = TableData::new(4, 2); + + let attrs1 = Attributes::from(vec![make_kv( + "val", + OtlpValue::StringValue("true".to_string()), + )]); + let mut row1 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs1, &mut row1).unwrap(); + writer.add_row(row1); + + let attrs2 = Attributes::from(vec![make_kv("val", OtlpValue::BoolValue(false))]); + let mut row2 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs2, &mut row2).unwrap(); + writer.add_row(row2); + + let (schema, rows) = writer.into_schema_and_rows(); + let col_idx = schema + .iter() + .position(|c| c.column_name == "attr.val") + .unwrap(); + assert_eq!(schema[col_idx].datatype, ColumnDataType::String as i32); + assert_eq!( + rows[0].values[col_idx].value_data, + Some(ValueData::StringValue("true".to_string())) + ); + assert_eq!( + rows[1].values[col_idx].value_data, + Some(ValueData::BoolValue(false)) + ); + } + + #[test] + fn test_keep_mixed_binary_and_string_values_until_frontend_reconciliation() { + let mut writer = TableData::new(4, 2); + + let attrs1 = Attributes::from(vec![make_kv( + "val", + OtlpValue::BytesValue(vec![1_u8, 2, 3]), + )]); + let mut row1 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs1, &mut row1).unwrap(); + writer.add_row(row1); + + let attrs2 = Attributes::from(vec![make_kv( + "val", + OtlpValue::StringValue("false".to_string()), + )]); + let mut row2 = writer.alloc_one_row(); + write_attributes(&mut writer, "attr", attrs2, &mut row2).unwrap(); + writer.add_row(row2); + + let (schema, rows) = writer.into_schema_and_rows(); + let col_idx = schema + .iter() + .position(|c| c.column_name == "attr.val") + .unwrap(); + assert_eq!(schema[col_idx].datatype, ColumnDataType::Binary as i32); + assert_eq!( + rows[0].values[col_idx].value_data, + Some(ValueData::BinaryValue(vec![1_u8, 2, 3])) + ); + assert_eq!( + rows[1].values[col_idx].value_data, + Some(ValueData::StringValue("false".to_string())) + ); + } + // Conversion matrix coverage lives in the shared coercion helper tests. +} diff --git a/src/servers/src/row_writer.rs b/src/servers/src/row_writer.rs index ec439a8659..fca2c21b41 100644 --- a/src/servers/src/row_writer.rs +++ b/src/servers/src/row_writer.rs @@ -77,6 +77,35 @@ impl TableData { pub fn into_schema_and_rows(self) -> (Vec, Vec) { (self.schema, self.rows) } + + /// Writes a field value without enforcing that later writes use the same datatype + /// as the first-seen schema entry. + /// + /// The OTLP trace v1 path uses this to preserve raw mixed values inside one request + /// so the frontend can reconcile them later against both the full batch and the + /// existing table schema. + pub fn write_field_unchecked( + &mut self, + name: impl ToString, + datatype: ColumnDataType, + value: Option, + one_row: &mut Vec, + ) { + let name = name.to_string(); + if let Some(index) = self.column_indexes.get(&name).copied() { + one_row[index].value_data = value; + } else { + let index = self.schema.len(); + self.schema.push(ColumnSchema { + column_name: name.clone(), + datatype: datatype as i32, + semantic_type: SemanticType::Field as i32, + ..Default::default() + }); + self.column_indexes.insert(name, index); + one_row.push(Value { value_data: value }); + } + } } pub struct MultiTableData { diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 933fcadf6b..c0d858a592 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -5283,6 +5283,315 @@ pub async fn test_otlp_traces_v1(store_type: StorageType) { ) .await; + let coercion_table_name = "trace_type_coercion"; + let coercion_req = make_trace_v1_request( + "type-coercion", + vec![make_trace_v1_span( + "00000000000000000000000000000001", + "0000000000000001", + "coercion-seed", + 1_736_480_942_444_376_000, + 1_736_480_942_444_499_000, + vec![ + make_double_attr("attr_float", 1.5), + make_int_attr("attr_int", 10), + make_bool_attr("attr_bool", true), + ], + )], + ); + let res = send_trace_v1_req(&client, coercion_table_name, coercion_req, false).await; + assert_eq!(StatusCode::OK, res.status()); + + let coercion_req = make_trace_v1_request( + "type-coercion", + vec![make_trace_v1_span( + "00000000000000000000000000000002", + "0000000000000002", + "coercion-apply", + 1_736_480_942_444_589_000, + 1_736_480_942_444_712_000, + vec![ + make_int_attr("attr_float", 2), + make_string_attr("attr_int", "20"), + make_string_attr("attr_bool", "false"), + ], + )], + ); + let res = send_trace_v1_req(&client, coercion_table_name, coercion_req, false).await; + assert_eq!(StatusCode::OK, res.status()); + + let string_target_table_name = "trace_type_coercion_to_string"; + let string_target_seed_req = make_trace_v1_request( + "type-coercion-string", + vec![make_trace_v1_span( + "00000000000000000000000000000021", + "0000000000000021", + "string-target-seed", + 1_736_480_942_444_720_000, + 1_736_480_942_444_820_000, + vec![ + make_string_attr("attr_int", "seed"), + make_string_attr("attr_float", "seed"), + make_string_attr("attr_bool", "seed"), + ], + )], + ); + let res = send_trace_v1_req( + &client, + string_target_table_name, + string_target_seed_req, + false, + ) + .await; + assert_eq!(StatusCode::OK, res.status()); + + let string_target_req = make_trace_v1_request( + "type-coercion-string", + vec![make_trace_v1_span( + "00000000000000000000000000000022", + "0000000000000022", + "string-target-apply", + 1_736_480_942_444_830_000, + 1_736_480_942_444_930_000, + vec![ + make_int_attr("attr_int", 20), + make_double_attr("attr_float", 2.5), + make_bool_attr("attr_bool", false), + ], + )], + ); + let res = send_trace_v1_req(&client, string_target_table_name, string_target_req, false).await; + assert_eq!(StatusCode::OK, res.status()); + + validate_data( + "otlp_traces_v1_type_coercion_to_string_rows", + &client, + &format!( + "select trace_id, \"span_attributes.attr_bool\", \"span_attributes.attr_float\", \"span_attributes.attr_int\" from {} order by trace_id;", + string_target_table_name + ), + r#"[["00000000000000000000000000000021","seed","seed","seed"],["00000000000000000000000000000022","false","2.5","20"]]"#, + ) + .await; + validate_data( + "otlp_traces_v1_type_coercion_to_string_schema", + &client, + "select column_name, lower(data_type), semantic_type from information_schema.columns where table_name = 'trace_type_coercion_to_string' and column_name in ('span_attributes.attr_bool', 'span_attributes.attr_float', 'span_attributes.attr_int') order by column_name;", + r#"[["span_attributes.attr_bool","string","FIELD"],["span_attributes.attr_float","string","FIELD"],["span_attributes.attr_int","string","FIELD"]]"#, + ) + .await; + + let intra_batch_prefer_non_string_table_name = "trace_type_prefer_non_string"; + let intra_batch_prefer_non_string_req = make_trace_v1_request( + "type-prefer-non-string", + vec![ + make_trace_v1_span( + "00000000000000000000000000000031", + "0000000000000031", + "prefer-non-string-seed", + 1_736_480_942_444_940_000, + 1_736_480_942_445_040_000, + vec![ + make_string_attr("attr_int", "10"), + make_string_attr("attr_float", "1.5"), + make_string_attr("attr_bool", "true"), + ], + ), + make_trace_v1_span( + "00000000000000000000000000000032", + "0000000000000032", + "prefer-non-string-apply", + 1_736_480_942_445_050_000, + 1_736_480_942_445_150_000, + vec![ + make_int_attr("attr_int", 20), + make_double_attr("attr_float", 2.5), + make_bool_attr("attr_bool", false), + ], + ), + ], + ); + let res = send_trace_v1_req( + &client, + intra_batch_prefer_non_string_table_name, + intra_batch_prefer_non_string_req, + false, + ) + .await; + assert_eq!(StatusCode::OK, res.status()); + + validate_data( + "otlp_traces_v1_prefer_non_string_rows", + &client, + &format!( + "select trace_id, \"span_attributes.attr_bool\", \"span_attributes.attr_float\", \"span_attributes.attr_int\" from {} order by trace_id;", + intra_batch_prefer_non_string_table_name + ), + r#"[["00000000000000000000000000000031",true,1.5,10],["00000000000000000000000000000032",false,2.5,20]]"#, + ) + .await; + validate_data( + "otlp_traces_v1_prefer_non_string_schema", + &client, + "select column_name, lower(data_type), semantic_type from information_schema.columns where table_name = 'trace_type_prefer_non_string' and column_name in ('span_attributes.attr_bool', 'span_attributes.attr_float', 'span_attributes.attr_int') order by column_name;", + r#"[["span_attributes.attr_bool","boolean","FIELD"],["span_attributes.attr_float","double","FIELD"],["span_attributes.attr_int","bigint","FIELD"]]"#, + ) + .await; + + let existing_float_table_name = "trace_type_existing_float_prefers_schema"; + let existing_float_seed_req = make_trace_v1_request( + "type-existing-float", + vec![make_trace_v1_span( + "00000000000000000000000000000041", + "0000000000000041", + "existing-float-seed", + 1_736_480_942_445_160_000, + 1_736_480_942_445_260_000, + vec![make_double_attr("attr_num", 1.25)], + )], + ); + let res = send_trace_v1_req( + &client, + existing_float_table_name, + existing_float_seed_req, + false, + ) + .await; + assert_eq!(StatusCode::OK, res.status()); + + let existing_float_req = make_trace_v1_request( + "type-existing-float", + vec![ + make_trace_v1_span( + "00000000000000000000000000000042", + "0000000000000042", + "existing-float-int-first", + 1_736_480_942_445_270_000, + 1_736_480_942_445_370_000, + vec![make_int_attr("attr_num", 2)], + ), + make_trace_v1_span( + "00000000000000000000000000000043", + "0000000000000043", + "existing-float-float-later", + 1_736_480_942_445_380_000, + 1_736_480_942_445_480_000, + vec![make_double_attr("attr_num", 3.5)], + ), + ], + ); + let res = send_trace_v1_req( + &client, + existing_float_table_name, + existing_float_req, + false, + ) + .await; + assert_eq!(StatusCode::OK, res.status()); + + validate_data( + "otlp_traces_v1_existing_float_prefers_schema_rows", + &client, + &format!( + "select trace_id, \"span_attributes.attr_num\" from {} order by trace_id;", + existing_float_table_name + ), + r#"[["00000000000000000000000000000041",1.25],["00000000000000000000000000000042",2.0],["00000000000000000000000000000043",3.5]]"#, + ) + .await; + validate_data( + "otlp_traces_v1_existing_float_prefers_schema_type", + &client, + "select column_name, lower(data_type), semantic_type from information_schema.columns where table_name = 'trace_type_existing_float_prefers_schema' and column_name = 'span_attributes.attr_num';", + r#"[["span_attributes.attr_num","double","FIELD"]]"#, + ) + .await; + + validate_data( + "otlp_traces_v1_type_coercion_rows", + &client, + &format!( + "select trace_id, \"span_attributes.attr_int\", \"span_attributes.attr_bool\" from {} order by trace_id;", + coercion_table_name + ), + r#"[["00000000000000000000000000000001",10,true],["00000000000000000000000000000002",20,false]]"#, + ) + .await; + validate_data( + "otlp_traces_v1_type_coercion_float_sum", + &client, + &format!( + "select sum(\"span_attributes.attr_float\") from {};", + coercion_table_name + ), + r#"[[3.5]]"#, + ) + .await; + validate_data( + "otlp_traces_v1_type_coercion_schema", + &client, + "select column_name, lower(data_type), semantic_type from information_schema.columns where table_name = 'trace_type_coercion' and column_name in ('span_attributes.attr_bool', 'span_attributes.attr_float', 'span_attributes.attr_int') order by column_name;", + r#"[["span_attributes.attr_bool","boolean","FIELD"],["span_attributes.attr_float","double","FIELD"],["span_attributes.attr_int","bigint","FIELD"]]"#, + ) + .await; + + let abort_table_name = "trace_type_abort"; + let abort_seed_req = make_trace_v1_request( + "type-abort", + vec![make_trace_v1_span( + "00000000000000000000000000000011", + "0000000000000011", + "abort-seed", + 1_736_480_942_444_800_000, + 1_736_480_942_444_900_000, + vec![make_int_attr("attr_int", 10)], + )], + ); + let res = send_trace_v1_req(&client, abort_table_name, abort_seed_req, false).await; + assert_eq!(StatusCode::OK, res.status()); + + let abort_req = make_trace_v1_request( + "type-abort", + vec![ + make_trace_v1_span( + "00000000000000000000000000000012", + "0000000000000012", + "abort-parseable", + 1_736_480_942_445_000_000, + 1_736_480_942_445_100_000, + vec![make_string_attr("attr_int", "20")], + ), + make_trace_v1_span( + "00000000000000000000000000000013", + "0000000000000013", + "abort-unparsable", + 1_736_480_942_445_200_000, + 1_736_480_942_445_300_000, + vec![make_string_attr("attr_int", "not_a_number")], + ), + ], + ); + let res = send_trace_v1_req(&client, abort_table_name, abort_req, false).await; + assert_eq!(StatusCode::BAD_REQUEST, res.status()); + let body: Value = res.json().await; + assert!( + body["error"].as_str().unwrap().contains( + "failed to coerce trace column 'span_attributes.attr_int' in table 'trace_type_abort'" + ), + "unexpected error body: {body}" + ); + + validate_data( + "otlp_traces_v1_type_abort_rows", + &client, + &format!( + "select trace_id, \"span_attributes.attr_int\" from {} order by trace_id;", + abort_table_name + ), + r#"[["00000000000000000000000000000011",10]]"#, + ) + .await; + guard.remove_all().await; } @@ -7515,6 +7824,115 @@ async fn send_req( req.header("content-length", len).send().await } +async fn send_trace_v1_req( + client: &TestClient, + table_name: &str, + req: ExportTraceServiceRequest, + with_gzip: bool, +) -> TestResponse { + send_req( + client, + vec![ + ( + HeaderName::from_static("content-type"), + HeaderValue::from_static("application/x-protobuf"), + ), + ( + HeaderName::from_static("x-greptime-pipeline-name"), + HeaderValue::from_static(GREPTIME_INTERNAL_TRACE_PIPELINE_V1_NAME), + ), + ( + HeaderName::from_static("x-greptime-trace-table-name"), + HeaderValue::from_str(table_name).unwrap(), + ), + ], + "/v1/otlp/v1/traces", + req.encode_to_vec(), + with_gzip, + ) + .await +} + +fn make_trace_v1_request(service_name: &str, spans: Vec) -> ExportTraceServiceRequest { + serde_json::from_value(json!({ + "resourceSpans": [{ + "resource": { + "attributes": [{ + "key": "service.name", + "value": { "stringValue": service_name } + }] + }, + "scopeSpans": [{ + "scope": { + "name": "trace-v1-type-tests" + }, + "spans": spans + }], + "schemaUrl": "https://opentelemetry.io/schemas/1.4.0" + }] + })) + .unwrap() +} + +fn make_trace_v1_span( + trace_id: &str, + span_id: &str, + name: &str, + start_time_unix_nano: i64, + end_time_unix_nano: i64, + attributes: Vec, +) -> Value { + json!({ + "traceId": trace_id, + "spanId": span_id, + "name": name, + "kind": 2, + "startTimeUnixNano": start_time_unix_nano.to_string(), + "endTimeUnixNano": end_time_unix_nano.to_string(), + "attributes": attributes, + "status": { + "message": "", + "code": 0 + } + }) +} + +fn make_string_attr(key: &str, value: &str) -> Value { + json!({ + "key": key, + "value": { + "stringValue": value + } + }) +} + +fn make_int_attr(key: &str, value: i64) -> Value { + json!({ + "key": key, + "value": { + "intValue": value.to_string() + } + }) +} + +fn make_double_attr(key: &str, value: f64) -> Value { + json!({ + "key": key, + "value": { + "doubleValue": value + } + }) +} + +fn make_bool_attr(key: &str, value: bool) -> Value { + json!({ + "key": key, + "value": { + "boolValue": value + } + }) +} + fn get_rows_from_output(output: &str) -> String { let resp: Value = serde_json::from_str(output).unwrap(); resp.get("output") From e14404c677ce386ce128bd107c611019ec4094a4 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Mon, 30 Mar 2026 20:13:14 +0800 Subject: [PATCH 09/20] chore: update rust toolchain to 2026-03-21 (#7849) * chore: update rust toolchain to 2026-03-21 * chore: new format * fix: lint * chore: resolve lint issues * chore: remove as_millis_f64 * chore: deps up --- Cargo.lock | 32 +++---- flake.lock | 18 ++-- flake.nix | 2 +- rust-toolchain.toml | 2 +- src/auth/tests/mod.rs | 3 +- src/catalog/src/lib.rs | 3 - src/catalog/src/memory/manager.rs | 20 ++--- .../information_schema/tables.rs | 24 ++--- src/catalog/src/system_schema/pg_catalog.rs | 10 +-- src/catalog/src/table_source.rs | 2 +- src/cli/src/data/export.rs | 6 +- src/client/src/database.rs | 2 +- src/client/src/lib.rs | 2 - src/cmd/src/bin/greptime.rs | 1 + src/cmd/src/datanode.rs | 2 +- src/cmd/src/datanode/scanbench.rs | 2 +- src/cmd/src/lib.rs | 2 +- .../datasource/src/file_format/tests.rs | 3 +- src/common/datasource/src/lib.rs | 3 - src/common/frontend/src/error.rs | 2 +- src/common/frontend/src/selector.rs | 1 + src/common/function/src/lib.rs | 1 - src/common/function/src/scalars/matches.rs | 14 ++- src/common/function/src/scalars/vector.rs | 6 +- .../scalars/vector/convert/parse_vector.rs | 2 +- src/common/meta/src/cluster.rs | 2 +- .../meta/src/ddl/drop_database/start.rs | 2 +- .../meta/src/ddl/drop_table/executor.rs | 2 +- src/common/meta/src/ddl/test_util.rs | 2 +- .../src/ddl/tests/alter_logical_tables.rs | 2 +- src/common/meta/src/ddl/tests/alter_table.rs | 6 +- src/common/meta/src/ddl/tests/create_flow.rs | 2 +- .../src/ddl/tests/create_logical_tables.rs | 2 +- src/common/meta/src/ddl/tests/create_table.rs | 2 +- src/common/meta/src/ddl/tests/create_view.rs | 2 +- src/common/meta/src/ddl/tests/drop_flow.rs | 2 +- src/common/meta/src/ddl/tests/drop_table.rs | 2 +- src/common/meta/src/election/rds/mysql.rs | 3 +- src/common/meta/src/election/rds/postgres.rs | 3 +- src/common/meta/src/key/flow.rs | 2 +- src/common/meta/src/key/topic_name.rs | 2 +- src/common/meta/src/kv_backend/rds.rs | 4 +- src/common/meta/src/lib.rs | 3 - src/common/meta/src/range_stream.rs | 2 +- .../update_table_infos.rs | 2 +- src/common/meta/src/reconciliation/utils.rs | 2 +- src/common/meta/src/sequence.rs | 2 +- src/common/meta/src/snapshot.rs | 2 +- src/common/meta/src/state_store.rs | 3 +- src/common/meta/src/wal_provider.rs | 2 +- .../meta/src/wal_provider/topic_pool.rs | 2 +- src/common/procedure/src/lib.rs | 2 - src/common/procedure/src/local.rs | 2 +- src/common/procedure/src/local/runner.rs | 2 +- src/common/procedure/src/store/util.rs | 4 +- src/common/recordbatch/src/lib.rs | 4 +- src/common/recordbatch/src/recordbatch.rs | 2 +- src/common/sql/src/default_constraint.rs | 2 +- src/common/sql/src/lib.rs | 2 - src/common/telemetry/src/lib.rs | 4 +- src/common/wal/src/lib.rs | 4 +- src/datanode/src/datanode.rs | 7 +- src/datanode/src/heartbeat/handler.rs | 2 +- .../src/heartbeat/handler/close_region.rs | 3 +- .../src/heartbeat/handler/downgrade_region.rs | 2 +- .../src/heartbeat/handler/open_region.rs | 2 +- .../src/heartbeat/handler/upgrade_region.rs | 7 +- src/datanode/src/lib.rs | 2 - src/datanode/src/region_server.rs | 2 +- src/datatypes/src/json.rs | 4 +- src/datatypes/src/lib.rs | 3 - src/datatypes/src/types/json_type.rs | 2 +- src/datatypes/src/value.rs | 18 ++-- src/datatypes/src/vectors.rs | 9 +- src/datatypes/src/vectors/binary.rs | 2 +- src/datatypes/src/vectors/json/builder.rs | 4 +- src/file-engine/src/lib.rs | 2 - src/file-engine/src/region.rs | 2 +- src/flow/src/adapter/flownode_impl.rs | 2 +- src/flow/src/lib.rs | 1 - src/flow/src/utils.rs | 6 +- src/frontend/src/lib.rs | 2 - .../inverted_index/format/reader/footer.rs | 2 +- src/index/src/lib.rs | 1 - src/log-query/src/log_query.rs | 88 +++++++++---------- src/log-store/src/kafka/index/iterator.rs | 6 +- src/log-store/src/kafka/log_store.rs | 2 +- src/log-store/src/kafka/util/record.rs | 2 +- src/log-store/src/lib.rs | 3 - src/meta-client/src/client/cluster.rs | 4 +- src/meta-client/src/client/procedure.rs | 4 +- src/meta-srv/src/gc/candidate.rs | 2 +- src/meta-srv/src/gc/handler.rs | 2 +- src/meta-srv/src/handler.rs | 18 ++-- src/meta-srv/src/lib.rs | 2 - .../src/procedure/region_migration.rs | 2 +- .../downgrade_leader_region.rs | 2 +- .../region_migration/flush_leader_region.rs | 2 +- .../src/procedure/region_migration/manager.rs | 2 +- .../region_migration/migration_start.rs | 2 +- .../region_migration/open_candidate_region.rs | 2 +- .../procedure/region_migration/test_util.rs | 2 +- .../downgrade_leader_region.rs | 2 +- .../rollback_downgraded_region.rs | 2 +- .../upgrade_candidate_region.rs | 2 +- .../upgrade_candidate_region.rs | 2 +- .../src/procedure/region_migration/utils.rs | 2 +- .../src/procedure/repartition/group.rs | 2 +- .../repartition/group/enter_staging_region.rs | 2 +- .../repartition/group/repartition_start.rs | 2 +- .../repartition/group/sync_region.rs | 2 +- src/meta-srv/src/procedure/wal_prune.rs | 2 +- .../src/procedure/wal_prune/manager.rs | 2 +- src/meta-srv/src/region/supervisor.rs | 2 +- src/meta-srv/src/service/mailbox.rs | 2 +- src/meta-srv/src/state.rs | 2 +- src/metric-engine/src/engine.rs | 2 +- src/metric-engine/src/engine/bulk_insert.rs | 2 +- .../src/engine/create/extract_new_columns.rs | 2 +- src/metric-engine/src/lib.rs | 2 +- src/metric-engine/src/repeated_task.rs | 2 +- src/metric-engine/src/test_util.rs | 2 +- src/mito-codec/src/key_values.rs | 4 +- src/mito2/src/cache/index.rs | 4 +- src/mito2/src/compaction.rs | 13 ++- src/mito2/src/compaction/run.rs | 2 +- src/mito2/src/compaction/twcs.rs | 4 +- src/mito2/src/engine/alter_test.rs | 2 +- .../src/engine/apply_staging_manifest_test.rs | 3 +- src/mito2/src/engine/catchup_test.rs | 2 +- src/mito2/src/engine/copy_region_from_test.rs | 3 +- src/mito2/src/engine/remap_manifests_test.rs | 2 +- src/mito2/src/engine/staging_test.rs | 3 +- src/mito2/src/engine/sync_test.rs | 2 +- src/mito2/src/lib.rs | 2 - src/mito2/src/manifest/tests/checkpoint.rs | 2 +- src/mito2/src/region.rs | 2 +- src/mito2/src/region/opener.rs | 4 +- src/mito2/src/sst/index.rs | 4 +- src/mito2/src/sst/parquet/row_group.rs | 2 +- src/operator/src/insert.rs | 6 +- src/operator/src/lib.rs | 3 - src/operator/src/statement.rs | 2 +- src/partition/src/lib.rs | 1 - src/partition/src/multi_dim.rs | 2 +- src/pipeline/src/etl/ctx_req.rs | 2 +- src/pipeline/src/etl/processor/dissect.rs | 2 +- src/pipeline/src/etl/processor/filter.rs | 8 +- .../src/extension_plan/range_manipulate.rs | 2 +- .../src/extension_plan/series_divide.rs | 2 +- src/promql/src/functions/extrapolate_rate.rs | 2 +- src/promql/src/functions/quantile.rs | 2 +- .../fs_puffin_manager/reader.rs | 2 +- src/query/src/dist_plan/analyzer/utils.rs | 2 +- src/query/src/optimizer/parallelize_scan.rs | 2 +- src/query/src/optimizer/scan_hint.rs | 8 +- src/query/src/promql/planner.rs | 22 ++--- src/query/src/sql.rs | 22 ++--- src/query/src/window_sort.rs | 2 +- src/servers/src/grpc/flight.rs | 3 +- src/servers/src/http/authorize.rs | 2 +- src/servers/src/http/header.rs | 12 ++- src/servers/src/http/prometheus.rs | 20 ++--- .../src/http/result/prometheus_resp.rs | 16 ++-- src/servers/src/lib.rs | 3 - src/servers/src/mysql/handler.rs | 4 +- src/servers/tests/mysql/mysql_server_test.rs | 4 +- src/sql/src/lib.rs | 2 - src/sql/src/parsers/alter_parser.rs | 2 +- src/sql/src/parsers/comment_parser.rs | 2 +- src/sql/src/parsers/copy_parser.rs | 2 +- src/sql/src/parsers/create_parser.rs | 2 +- src/sql/src/parsers/delete_parser.rs | 2 +- src/sql/src/parsers/insert_parser.rs | 2 +- src/sql/src/parsers/show_parser.rs | 2 +- src/sql/src/statements/alter.rs | 2 +- src/sql/src/statements/copy.rs | 2 +- src/sql/src/statements/create.rs | 2 +- src/sql/src/statements/describe.rs | 2 +- src/sql/src/statements/drop.rs | 2 +- src/sql/src/statements/set_variables.rs | 2 +- src/sql/src/statements/show.rs | 2 +- src/sql/src/statements/truncate.rs | 2 +- src/store-api/src/lib.rs | 2 - src/table/src/lib.rs | 1 - src/table/src/metadata.rs | 2 +- tests-integration/Cargo.toml | 5 ++ tests-integration/src/cluster.rs | 4 +- tests-integration/src/lib.rs | 2 +- .../src/tests/instance_kafka_wal_test.rs | 2 +- tests-integration/tests/main.rs | 2 + 191 files changed, 353 insertions(+), 446 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0cab5067bb..cfec1c5f54 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -565,7 +565,7 @@ dependencies = [ "arrow-schema 57.3.0", "arrow-select 57.3.0", "flatbuffers", - "lz4_flex 0.12.0", + "lz4_flex 0.12.1", "zstd", ] @@ -6277,7 +6277,7 @@ dependencies = [ "js-sys", "log", "wasm-bindgen", - "windows-core 0.61.2", + "windows-core 0.57.0", ] [[package]] @@ -7592,18 +7592,18 @@ dependencies = [ [[package]] name = "lz4_flex" -version = "0.11.5" +version = "0.11.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" +checksum = "373f5eceeeab7925e0c1098212f2fbc4d416adec9d35051a6ab251e824c1854a" dependencies = [ "twox-hash", ] [[package]] name = "lz4_flex" -version = "0.12.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab6473172471198271ff72e9379150e9dfd70d8e533e0752a27e515b48dd375e" +checksum = "98c23545df7ecf1b16c303910a69b079e8e251d60f7dd2cc9b4177f2afaf1746" dependencies = [ "twox-hash", ] @@ -9122,7 +9122,7 @@ dependencies = [ "flate2", "futures", "futures-util", - "lz4_flex 0.11.5", + "lz4_flex 0.11.6", "lzokay-native", "num", "prost 0.13.5", @@ -9345,7 +9345,7 @@ dependencies = [ "futures", "half", "hashbrown 0.16.1", - "lz4_flex 0.12.0", + "lz4_flex 0.12.1", "num-bigint", "num-integer", "num-traits", @@ -10574,7 +10574,7 @@ dependencies = [ "common-test-util", "derive_builder 0.20.2", "futures", - "lz4_flex 0.11.5", + "lz4_flex 0.11.6", "moka", "pin-project", "prometheus 0.14.0", @@ -11306,9 +11306,9 @@ dependencies = [ [[package]] name = "rsasl" -version = "2.2.0" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8b534a23662bb559c5c73213be63ecd6524e774d291f3618c2b04b723d184eb" +checksum = "9f1bcb95b531681a622f3d6972eaab523e17e2aad6d6209f0276628eb1cb5038" dependencies = [ "base64 0.22.1", "core2", @@ -11320,7 +11320,7 @@ dependencies = [ "serde_json", "sha2", "stringprep", - "thiserror 1.0.69", + "thiserror 2.0.17", ] [[package]] @@ -12308,9 +12308,9 @@ dependencies = [ [[package]] name = "slab" -version = "0.4.10" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04dc19736151f35336d325007ac991178d504a119863a2fcb3758cdb5e52c50d" +checksum = "0c790de23124f9ab44544d7ac05d60440adc586479ce501c1d6d7da3cd8c9cf5" [[package]] name = "slotmap" @@ -13224,7 +13224,7 @@ dependencies = [ "levenshtein_automata", "log", "lru", - "lz4_flex 0.11.5", + "lz4_flex 0.11.6", "measure_time", "memmap2", "once_cell", @@ -14714,7 +14714,7 @@ dependencies = [ "itertools 0.14.0", "lalrpop", "lalrpop-util", - "lz4_flex 0.11.5", + "lz4_flex 0.11.6", "md-5", "nom 7.1.3", "ofb", diff --git a/flake.lock b/flake.lock index bec6e18d9a..3c3ee4bd67 100644 --- a/flake.lock +++ b/flake.lock @@ -8,11 +8,11 @@ "rust-analyzer-src": "rust-analyzer-src" }, "locked": { - "lastModified": 1770794449, - "narHash": "sha256-1nFkhcZx9+Sdw5OXwJqp5TxvGncqRqLeK781v0XV3WI=", + "lastModified": 1774250935, + "narHash": "sha256-mWID0WFgTnd9hbEeaPNX+YYWF70JN3r7zBouEqERJOE=", "owner": "nix-community", "repo": "fenix", - "rev": "b19d93fdf9761e6101f8cb5765d638bacebd9a1b", + "rev": "64d7705e8c37d650cfb1aa99c24a8ce46597f29e", "type": "github" }, "original": { @@ -41,11 +41,11 @@ }, "nixpkgs": { "locked": { - "lastModified": 1770617025, - "narHash": "sha256-1jZvgZoAagZZB6NwGRv2T2ezPy+X6EFDsJm+YSlsvEs=", + "lastModified": 1774244481, + "narHash": "sha256-4XfMXU0DjN83o6HWZoKG9PegCvKvIhNUnRUI19vzTcQ=", "owner": "NixOS", "repo": "nixpkgs", - "rev": "2db38e08fdadcc0ce3232f7279bab59a15b94482", + "rev": "4590696c8693fea477850fe379a01544293ca4e2", "type": "github" }, "original": { @@ -65,11 +65,11 @@ "rust-analyzer-src": { "flake": false, "locked": { - "lastModified": 1770702974, - "narHash": "sha256-CbvWu72rpGHK5QynoXwuOnVzxX7njF2LYgk8wRSiAQ0=", + "lastModified": 1774221325, + "narHash": "sha256-aEIdkqB8gtQZtEbogdUb5iyfcZpKIlD3FkG8ANu73/I=", "owner": "rust-lang", "repo": "rust-analyzer", - "rev": "07a594815f7c1d6e7e39f21ddeeedb75b21795f4", + "rev": "b42b63f390a4dab14e6efa34a70e67f5b087cc62", "type": "github" }, "original": { diff --git a/flake.nix b/flake.nix index 6a02f4f05f..8dc84136a0 100644 --- a/flake.nix +++ b/flake.nix @@ -20,7 +20,7 @@ lib = nixpkgs.lib; rustToolchain = fenix.packages.${system}.fromToolchainName { name = (lib.importTOML ./rust-toolchain.toml).toolchain.channel; - sha256 = "sha256-GCGEXGZeJySLND0KU5TdtTrqFV76TF3UdvAHSUegSsk="; + sha256 = "sha256-rboGKQLH4eDuiY01SINOqmXUFUNr9F4awoFZGzib17o="; }; in { diff --git a/rust-toolchain.toml b/rust-toolchain.toml index 58b88a3894..d16edecca8 100644 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -1,2 +1,2 @@ [toolchain] -channel = "nightly-2025-10-01" +channel = "nightly-2026-03-21" diff --git a/src/auth/tests/mod.rs b/src/auth/tests/mod.rs index 65db96a13f..4abbf89e5c 100644 --- a/src/auth/tests/mod.rs +++ b/src/auth/tests/mod.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] -use std::assert_matches::assert_matches; +use std::assert_matches; use std::sync::Arc; use api::v1::greptime_request::Request; diff --git a/src/catalog/src/lib.rs b/src/catalog/src/lib.rs index 9c31e809fd..a701473551 100644 --- a/src/catalog/src/lib.rs +++ b/src/catalog/src/lib.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] -#![feature(try_blocks)] - use std::any::Any; use std::fmt::{Debug, Formatter}; use std::sync::Arc; diff --git a/src/catalog/src/memory/manager.rs b/src/catalog/src/memory/manager.rs index 571cd06468..6e747f62ed 100644 --- a/src/catalog/src/memory/manager.rs +++ b/src/catalog/src/memory/manager.rs @@ -132,15 +132,13 @@ impl CatalogManager for MemoryCatalogManager { table_name: &str, _query_ctx: Option<&QueryContext>, ) -> Result> { - let result = try { - self.catalogs - .read() - .unwrap() - .get(catalog)? - .get(schema)? - .get(table_name) - .cloned()? - }; + let catalogs = self.catalogs.read().unwrap(); + let result = catalogs + .get(catalog) + .and_then(|c| c.get(schema)) + .and_then(|s| s.get(table_name)) + .cloned(); + Ok(result) } @@ -149,8 +147,8 @@ impl CatalogManager for MemoryCatalogManager { .catalogs .read() .unwrap() - .iter() - .flat_map(|(_, schema_entries)| schema_entries.values()) + .values() + .flat_map(|schema_entries| schema_entries.values()) .flat_map(|tables| tables.values()) .find(|t| t.table_info().ident.table_id == table_id) .map(|t| t.table_info())) diff --git a/src/catalog/src/system_schema/information_schema/tables.rs b/src/catalog/src/system_schema/information_schema/tables.rs index 248fb243dd..6175c17d39 100644 --- a/src/catalog/src/system_schema/information_schema/tables.rs +++ b/src/catalog/src/system_schema/information_schema/tables.rs @@ -372,22 +372,16 @@ impl InformationSchemaTablesBuilder { self.table_types.push(Some(table_type_text)); self.table_ids.push(Some(table_id)); - let data_length = region_stats.iter().map(|stat| stat.sst_size).sum(); - let table_rows = region_stats.iter().map(|stat| stat.num_rows).sum(); - let index_length = region_stats.iter().map(|stat| stat.index_size).sum(); + let data_length: u64 = region_stats.iter().map(|stat| stat.sst_size).sum(); + let table_rows: u64 = region_stats.iter().map(|stat| stat.num_rows).sum(); + let index_length: u64 = region_stats.iter().map(|stat| stat.index_size).sum(); - // It's not precise, but it is acceptable for long-term data storage. - let avg_row_length = if table_rows > 0 { - let total_data_length = data_length - + region_stats - .iter() - .map(|stat| stat.memtable_size) - .sum::(); - - total_data_length / table_rows - } else { - 0 - }; + let total_data_length: u64 = data_length + + region_stats + .iter() + .map(|stat| stat.memtable_size) + .sum::(); + let avg_row_length = total_data_length.checked_div(table_rows).unwrap_or(0); self.data_length.push(Some(data_length)); self.index_length.push(Some(index_length)); diff --git a/src/catalog/src/system_schema/pg_catalog.rs b/src/catalog/src/system_schema/pg_catalog.rs index 08aad2d6dd..feec46ff90 100644 --- a/src/catalog/src/system_schema/pg_catalog.rs +++ b/src/catalog/src/system_schema/pg_catalog.rs @@ -74,12 +74,10 @@ impl PGCatalogProvider { ) .expect("Failed to initialize PgCatalogSchemaProvider"); - let mut table_ids = HashMap::new(); - let mut table_id = PG_CATALOG_TABLE_ID_START; - for name in PG_CATALOG_TABLES { - table_ids.insert(*name, table_id); - table_id += 1; - } + let table_ids: HashMap<_, _> = (PG_CATALOG_TABLE_ID_START..) + .zip(PG_CATALOG_TABLES.iter()) + .map(|(id, name)| (*name, id)) + .collect(); let mut provider = Self { catalog_name, diff --git a/src/catalog/src/table_source.rs b/src/catalog/src/table_source.rs index 8aabf64e99..f7ba51722f 100644 --- a/src/catalog/src/table_source.rs +++ b/src/catalog/src/table_source.rs @@ -195,7 +195,7 @@ impl DfTableSourceProvider { plan_columns .iter() .map(|c| c.as_str()) - .zip(columns.into_iter()) + .zip(columns) .collect(), ) .context(ProjectViewColumnsSnafu)? diff --git a/src/cli/src/data/export.rs b/src/cli/src/data/export.rs index b5d547d4f3..051c07da35 100644 --- a/src/cli/src/data/export.rs +++ b/src/cli/src/data/export.rs @@ -458,8 +458,10 @@ impl Export { /// build operator with preference for file system async fn build_prefer_fs_operator(&self) -> Result { - if self.storage_type.is_remote_storage() && self.ddl_local_dir.is_some() { - let root = self.ddl_local_dir.as_ref().unwrap().clone(); + if self.storage_type.is_remote_storage() + && let Some(ddl_local_dir) = &self.ddl_local_dir + { + let root = ddl_local_dir.clone(); let op = new_fs_object_store(&root).map_err(|e| Error::Other { source: e, location: snafu::location!(), diff --git a/src/client/src/database.rs b/src/client/src/database.rs index 6a7ac62fc3..e12c2ec0fc 100644 --- a/src/client/src/database.rs +++ b/src/client/src/database.rs @@ -512,7 +512,7 @@ struct FlightContext { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use api::v1::auth_header::AuthScheme; use api::v1::{AuthHeader, Basic}; diff --git a/src/client/src/lib.rs b/src/client/src/lib.rs index bf383acff9..0c9334b7d4 100644 --- a/src/client/src/lib.rs +++ b/src/client/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] - mod client; pub mod client_manager; pub mod database; diff --git a/src/cmd/src/bin/greptime.rs b/src/cmd/src/bin/greptime.rs index 7ddc2cd176..9c48d1fd6a 100644 --- a/src/cmd/src/bin/greptime.rs +++ b/src/cmd/src/bin/greptime.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![recursion_limit = "256"] #![doc = include_str!("../../../../README.md")] use clap::{Parser, Subcommand}; diff --git a/src/cmd/src/datanode.rs b/src/cmd/src/datanode.rs index 06e2568b72..9b06f24ecb 100644 --- a/src/cmd/src/datanode.rs +++ b/src/cmd/src/datanode.rs @@ -356,7 +356,7 @@ impl StartCommand { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::io::Write; use std::time::Duration; diff --git a/src/cmd/src/datanode/scanbench.rs b/src/cmd/src/datanode/scanbench.rs index fdda1d97bb..6bfe177fc1 100644 --- a/src/cmd/src/datanode/scanbench.rs +++ b/src/cmd/src/datanode/scanbench.rs @@ -662,7 +662,7 @@ impl ScanbenchCommand { // Sort ranges within each partition by start time ascending for partition in &mut partitions { - partition.sort_by(|a, b| a.start.cmp(&b.start)); + partition.sort_by_key(|a| a.start); } scanner diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 46ca4c8a76..27564597d7 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] +#![recursion_limit = "256"] use async_trait::async_trait; use common_error::ext::ErrorExt; diff --git a/src/common/datasource/src/file_format/tests.rs b/src/common/datasource/src/file_format/tests.rs index ad54472d33..75d74b53cd 100644 --- a/src/common/datasource/src/file_format/tests.rs +++ b/src/common/datasource/src/file_format/tests.rs @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; use std::collections::HashMap; use std::sync::Arc; -use std::vec; +use std::{assert_matches, vec}; use common_test_util::find_workspace_path; use datafusion::assert_batches_eq; diff --git a/src/common/datasource/src/lib.rs b/src/common/datasource/src/lib.rs index 91663ce22c..f4c7fdf120 100644 --- a/src/common/datasource/src/lib.rs +++ b/src/common/datasource/src/lib.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] -#![feature(type_alias_impl_trait)] - pub mod buffered_writer; pub mod compressed_writer; pub mod compression; diff --git a/src/common/frontend/src/error.rs b/src/common/frontend/src/error.rs index cee8c6df77..429489326c 100644 --- a/src/common/frontend/src/error.rs +++ b/src/common/frontend/src/error.rs @@ -52,7 +52,7 @@ pub enum Error { #[snafu(display("Failed to invoke list process service"))] CreateChannel { - source: common_grpc::error::Error, + source: Box, #[snafu(implicit)] location: Location, }, diff --git a/src/common/frontend/src/selector.rs b/src/common/frontend/src/selector.rs index 804169d1dd..5bbd8bb52c 100644 --- a/src/common/frontend/src/selector.rs +++ b/src/common/frontend/src/selector.rs @@ -91,6 +91,7 @@ impl FrontendSelector for MetaClientSelector { let channel = self .channel_manager .get(node.peer.addr) + .map_err(Box::new) .context(error::CreateChannelSnafu)?; let client = frontend_client::FrontendClient::new(channel); Ok(Box::new(client) as FrontendClientPtr) diff --git a/src/common/function/src/lib.rs b/src/common/function/src/lib.rs index 36fd27381d..7abd595367 100644 --- a/src/common/function/src/lib.rs +++ b/src/common/function/src/lib.rs @@ -13,7 +13,6 @@ // limitations under the License. #![feature(try_blocks)] -#![feature(assert_matches)] mod admin; mod flush_flow; diff --git a/src/common/function/src/scalars/matches.rs b/src/common/function/src/scalars/matches.rs index 821a1b0581..b5de60dc85 100644 --- a/src/common/function/src/scalars/matches.rs +++ b/src/common/function/src/scalars/matches.rs @@ -794,16 +794,12 @@ impl Tokenizer { is_quote_present = true; break; } - ' ' => { - if !is_quoted { - break; - } + ' ' if !is_quoted => { + break; } - '(' | ')' | '+' | '-' => { - if !is_quoted { - self.rewind_one(); - break; - } + '(' | ')' | '+' | '-' if !is_quoted => { + self.rewind_one(); + break; } '\\' => { let Some(next) = self.consume_next(pattern) else { diff --git a/src/common/function/src/scalars/vector.rs b/src/common/function/src/scalars/vector.rs index 968231aa0a..2f8772c410 100644 --- a/src/common/function/src/scalars/vector.rs +++ b/src/common/function/src/scalars/vector.rs @@ -141,7 +141,7 @@ where results.push((self.func)(v0, v1)?); } - let results = ScalarValue::iter_to_array(results.into_iter())?; + let results = ScalarValue::iter_to_array(results)?; Ok(ColumnarValue::Array(results)) } } @@ -200,7 +200,7 @@ where } } - let results = ScalarValue::iter_to_array(results.into_iter())?; + let results = ScalarValue::iter_to_array(results)?; Ok(ColumnarValue::Array(results)) } } @@ -232,7 +232,7 @@ where results.push((self.func)(&v)?); } - let results = ScalarValue::iter_to_array(results.into_iter())?; + let results = ScalarValue::iter_to_array(results)?; Ok(ColumnarValue::Array(results)) } } diff --git a/src/common/function/src/scalars/vector/convert/parse_vector.rs b/src/common/function/src/scalars/vector/convert/parse_vector.rs index 0d83f098db..7a112a4453 100644 --- a/src/common/function/src/scalars/vector/convert/parse_vector.rs +++ b/src/common/function/src/scalars/vector/convert/parse_vector.rs @@ -167,7 +167,7 @@ mod tests { "External error: Invalid vector string: [7.0,hello,9.0]", ]; - for (input, expected) in inputs.into_iter().zip(expected.into_iter()) { + for (input, expected) in inputs.into_iter().zip(expected) { let args = ScalarFunctionArgs { args: vec![ColumnarValue::Array(Arc::new(input))], arg_fields: vec![], diff --git a/src/common/meta/src/cluster.rs b/src/common/meta/src/cluster.rs index 78af133e8f..527ad589f1 100644 --- a/src/common/meta/src/cluster.rs +++ b/src/common/meta/src/cluster.rs @@ -303,7 +303,7 @@ impl TryFrom for Role { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_workload::DatanodeWorkloadType; diff --git a/src/common/meta/src/ddl/drop_database/start.rs b/src/common/meta/src/ddl/drop_database/start.rs index 775071d684..4da83e367f 100644 --- a/src/common/meta/src/ddl/drop_database/start.rs +++ b/src/common/meta/src/ddl/drop_database/start.rs @@ -72,7 +72,7 @@ impl State for DropDatabaseStart { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use crate::ddl::drop_database::cursor::DropDatabaseCursor; diff --git a/src/common/meta/src/ddl/drop_table/executor.rs b/src/common/meta/src/ddl/drop_table/executor.rs index c342487365..271bdbfede 100644 --- a/src/common/meta/src/ddl/drop_table/executor.rs +++ b/src/common/meta/src/ddl/drop_table/executor.rs @@ -322,7 +322,7 @@ impl DropTableExecutor { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/common/meta/src/ddl/test_util.rs b/src/common/meta/src/ddl/test_util.rs index 1dd1f783dc..36d422216f 100644 --- a/src/common/meta/src/ddl/test_util.rs +++ b/src/common/meta/src/ddl/test_util.rs @@ -19,7 +19,7 @@ pub mod datanode_handler; pub mod flownode_handler; pub mod region_metadata; -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use api::v1::meta::Partition; diff --git a/src/common/meta/src/ddl/tests/alter_logical_tables.rs b/src/common/meta/src/ddl/tests/alter_logical_tables.rs index e6bb7676f5..bca56bafc2 100644 --- a/src/common/meta/src/ddl/tests/alter_logical_tables.rs +++ b/src/common/meta/src/ddl/tests/alter_logical_tables.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::sync::Arc; use api::region::RegionResponse; diff --git a/src/common/meta/src/ddl/tests/alter_table.rs b/src/common/meta/src/ddl/tests/alter_table.rs index d935aa6a15..14ee71b3e6 100644 --- a/src/common/meta/src/ddl/tests/alter_table.rs +++ b/src/common/meta/src/ddl/tests/alter_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; @@ -256,7 +256,7 @@ async fn test_on_submit_alter_request() { results.push(result); } rx.try_recv().unwrap_err(); - results.sort_unstable_by(|(a, _), (b, _)| a.id.cmp(&b.id)); + results.sort_unstable_by_key(|(a, _)| a.id); let (peer, request) = results.remove(0); assert_alter_request(peer, request, 1, RegionId::new(table_id, 1)); @@ -310,7 +310,7 @@ async fn test_on_submit_alter_request_without_sync_request() { results.push(result); } rx.try_recv().unwrap_err(); - results.sort_unstable_by(|(a, _), (b, _)| a.id.cmp(&b.id)); + results.sort_unstable_by_key(|(a, _)| a.id); let (peer, request) = results.remove(0); assert_alter_request(peer, request, 1, RegionId::new(table_id, 1)); diff --git a/src/common/meta/src/ddl/tests/create_flow.rs b/src/common/meta/src/ddl/tests/create_flow.rs index 5b22c81857..344fc05024 100644 --- a/src/common/meta/src/ddl/tests/create_flow.rs +++ b/src/common/meta/src/ddl/tests/create_flow.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/common/meta/src/ddl/tests/create_logical_tables.rs b/src/common/meta/src/ddl/tests/create_logical_tables.rs index f7dd397f9f..e2927e8df6 100644 --- a/src/common/meta/src/ddl/tests/create_logical_tables.rs +++ b/src/common/meta/src/ddl/tests/create_logical_tables.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::sync::Arc; use api::region::RegionResponse; diff --git a/src/common/meta/src/ddl/tests/create_table.rs b/src/common/meta/src/ddl/tests/create_table.rs index 5cc1db71cb..5355ac8c7c 100644 --- a/src/common/meta/src/ddl/tests/create_table.rs +++ b/src/common/meta/src/ddl/tests/create_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/common/meta/src/ddl/tests/create_view.rs b/src/common/meta/src/ddl/tests/create_view.rs index e4fefa8944..cc98bb1bae 100644 --- a/src/common/meta/src/ddl/tests/create_view.rs +++ b/src/common/meta/src/ddl/tests/create_view.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashSet; use std::sync::Arc; diff --git a/src/common/meta/src/ddl/tests/drop_flow.rs b/src/common/meta/src/ddl/tests/drop_flow.rs index 8de42f5c96..af34da4809 100644 --- a/src/common/meta/src/ddl/tests/drop_flow.rs +++ b/src/common/meta/src/ddl/tests/drop_flow.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/common/meta/src/ddl/tests/drop_table.rs b/src/common/meta/src/ddl/tests/drop_table.rs index 65c3915adc..fb2c882da0 100644 --- a/src/common/meta/src/ddl/tests/drop_table.rs +++ b/src/common/meta/src/ddl/tests/drop_table.rs @@ -172,7 +172,7 @@ async fn test_on_datanode_drop_regions() { let result = rx.try_recv().unwrap(); results.push(result); } - results.sort_unstable_by(|(a, _), (b, _)| a.id.cmp(&b.id)); + results.sort_unstable_by_key(|(a, _)| a.id); let (peer, request) = results.remove(0); check(peer, request, 1, RegionId::new(table_id, 1), false); diff --git a/src/common/meta/src/election/rds/mysql.rs b/src/common/meta/src/election/rds/mysql.rs index 80f3d8ca7c..bd694e4ae1 100644 --- a/src/common/meta/src/election/rds/mysql.rs +++ b/src/common/meta/src/election/rds/mysql.rs @@ -987,8 +987,7 @@ impl MySqlElection { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; - use std::env; + use std::{assert_matches, env}; use common_telemetry::init_default_ut_logging; use sqlx::MySqlPool; diff --git a/src/common/meta/src/election/rds/postgres.rs b/src/common/meta/src/election/rds/postgres.rs index 01910335a0..220b33bb60 100644 --- a/src/common/meta/src/election/rds/postgres.rs +++ b/src/common/meta/src/election/rds/postgres.rs @@ -829,8 +829,7 @@ impl PgElection { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; - use std::env; + use std::{assert_matches, env}; use deadpool_postgres::{Config, Runtime}; use tokio_postgres::NoTls; diff --git a/src/common/meta/src/key/flow.rs b/src/common/meta/src/key/flow.rs index 546071f2a0..1ebd52da1a 100644 --- a/src/common/meta/src/key/flow.rs +++ b/src/common/meta/src/key/flow.rs @@ -390,7 +390,7 @@ impl std::fmt::Debug for FlowMetadataManager { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::BTreeMap; use std::sync::Arc; diff --git a/src/common/meta/src/key/topic_name.rs b/src/common/meta/src/key/topic_name.rs index 5497fbe478..99ae631a72 100644 --- a/src/common/meta/src/key/topic_name.rs +++ b/src/common/meta/src/key/topic_name.rs @@ -237,7 +237,7 @@ impl TopicNameManager { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use super::*; diff --git a/src/common/meta/src/kv_backend/rds.rs b/src/common/meta/src/kv_backend/rds.rs index 8acab1eb65..fd88496bc3 100644 --- a/src/common/meta/src/kv_backend/rds.rs +++ b/src/common/meta/src/kv_backend/rds.rs @@ -575,12 +575,12 @@ macro_rules! record_rds_sql_execute_elapsed { .inspect(|_| { $crate::metrics::RDS_SQL_EXECUTE_ELAPSED .with_label_values(&[$label_store, "success", $label_op, $label_type]) - .observe(timer.elapsed().as_millis_f64()) + .observe(timer.elapsed().as_millis() as f64) }) .inspect_err(|_| { $crate::metrics::RDS_SQL_EXECUTE_ELAPSED .with_label_values(&[$label_store, "error", $label_op, $label_type]) - .observe(timer.elapsed().as_millis_f64()); + .observe(timer.elapsed().as_millis() as f64); }) }}; } diff --git a/src/common/meta/src/lib.rs b/src/common/meta/src/lib.rs index 36aae1026e..ef5109dc03 100644 --- a/src/common/meta/src/lib.rs +++ b/src/common/meta/src/lib.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] -#![feature(duration_millis_float)] - pub mod cache; pub mod cache_invalidator; pub mod cluster; diff --git a/src/common/meta/src/range_stream.rs b/src/common/meta/src/range_stream.rs index 2fc333064e..4bcd5de7db 100644 --- a/src/common/meta/src/range_stream.rs +++ b/src/common/meta/src/range_stream.rs @@ -187,7 +187,7 @@ impl PaginationStream { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::BTreeMap; use std::sync::Arc; diff --git a/src/common/meta/src/reconciliation/reconcile_logical_tables/update_table_infos.rs b/src/common/meta/src/reconciliation/reconcile_logical_tables/update_table_infos.rs index d0de7a06fb..e8597ccd36 100644 --- a/src/common/meta/src/reconciliation/reconcile_logical_tables/update_table_infos.rs +++ b/src/common/meta/src/reconciliation/reconcile_logical_tables/update_table_infos.rs @@ -84,7 +84,7 @@ impl State for UpdateTableInfos { .persistent_ctx .update_table_infos .iter() - .zip(table_info_values.into_iter()) + .zip(table_info_values) { let new_table_info = Self::build_new_table_info( *table_id, diff --git a/src/common/meta/src/reconciliation/utils.rs b/src/common/meta/src/reconciliation/utils.rs index 4debc6de4d..6ddc084596 100644 --- a/src/common/meta/src/reconciliation/utils.rs +++ b/src/common/meta/src/reconciliation/utils.rs @@ -949,7 +949,7 @@ impl Display for ReconcileTableMetrics { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/common/meta/src/sequence.rs b/src/common/meta/src/sequence.rs index d186446fda..cd0a8ebf88 100644 --- a/src/common/meta/src/sequence.rs +++ b/src/common/meta/src/sequence.rs @@ -337,7 +337,7 @@ impl Inner { #[cfg(test)] mod tests { use std::any::Any; - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashSet; use std::sync::Arc; diff --git a/src/common/meta/src/snapshot.rs b/src/common/meta/src/snapshot.rs index 8f4818e33a..e66156b255 100644 --- a/src/common/meta/src/snapshot.rs +++ b/src/common/meta/src/snapshot.rs @@ -355,7 +355,7 @@ impl MetadataSnapshotManager { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use common_test_util::temp_dir::{TempDir, create_temp_dir}; diff --git a/src/common/meta/src/state_store.rs b/src/common/meta/src/state_store.rs index 0ecbd5a8e4..d98a286581 100644 --- a/src/common/meta/src/state_store.rs +++ b/src/common/meta/src/state_store.rs @@ -380,9 +380,8 @@ impl PoisonStore for KvStateStore { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; - use std::env; use std::sync::Arc; + use std::{assert_matches, env}; use common_procedure::store::state_store::KeyValue; use common_telemetry::info; diff --git a/src/common/meta/src/wal_provider.rs b/src/common/meta/src/wal_provider.rs index cd599b58ba..bf2cb9ba32 100644 --- a/src/common/meta/src/wal_provider.rs +++ b/src/common/meta/src/wal_provider.rs @@ -172,7 +172,7 @@ pub fn extract_topic_from_wal_options( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_wal::config::kafka::MetasrvKafkaConfig; use common_wal::config::kafka::common::KafkaTopicConfig; diff --git a/src/common/meta/src/wal_provider/topic_pool.rs b/src/common/meta/src/wal_provider/topic_pool.rs index 919f0b2abe..f9b4863e52 100644 --- a/src/common/meta/src/wal_provider/topic_pool.rs +++ b/src/common/meta/src/wal_provider/topic_pool.rs @@ -136,7 +136,7 @@ impl KafkaTopicPool { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_wal::maybe_skip_kafka_integration_test; use common_wal::test_util::get_kafka_endpoints; diff --git a/src/common/procedure/src/lib.rs b/src/common/procedure/src/lib.rs index e322765558..156a0ab78c 100644 --- a/src/common/procedure/src/lib.rs +++ b/src/common/procedure/src/lib.rs @@ -14,8 +14,6 @@ //! Common traits and structures for the procedure framework. -#![feature(assert_matches)] - pub mod error; pub mod event; pub mod local; diff --git a/src/common/procedure/src/local.rs b/src/common/procedure/src/local.rs index fe86cd7993..9e8536308c 100644 --- a/src/common/procedure/src/local.rs +++ b/src/common/procedure/src/local.rs @@ -920,7 +920,7 @@ pub(crate) mod test_util { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_error::mock::MockError; use common_error::status_code::StatusCode; diff --git a/src/common/procedure/src/local/runner.rs b/src/common/procedure/src/local/runner.rs index 46dcef11d4..2a974de889 100644 --- a/src/common/procedure/src/local/runner.rs +++ b/src/common/procedure/src/local/runner.rs @@ -704,7 +704,7 @@ impl Runner { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use std::sync::atomic::{AtomicU64, Ordering}; diff --git a/src/common/procedure/src/store/util.rs b/src/common/procedure/src/store/util.rs index e6ef5b62ec..c44e5f4712 100644 --- a/src/common/procedure/src/store/util.rs +++ b/src/common/procedure/src/store/util.rs @@ -57,7 +57,7 @@ fn merge_multiple_values( let (key, value) = pairs.into_iter().next().unwrap(); let prefix = KeySet::with_prefix(&key); let mut parsed_segments = parse_segments(segments, &prefix)?; - parsed_segments.sort_unstable_by(|a, b| a.0.cmp(&b.0)); + parsed_segments.sort_unstable_by_key(|a| a.0); // Safety: `parsed_segments` must larger than 0. let segment_num = parsed_segments.last().unwrap().0; @@ -133,7 +133,7 @@ pub fn multiple_value_stream( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use futures::TryStreamExt; use futures::stream::{self}; diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index d84e9e9d26..629efd6d84 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -485,7 +485,7 @@ impl QueryMemoryTracker { "{} requested, {} used globally ({}%), {} used by this stream, hard limit: {}", ReadableSize(additional as u64), ReadableSize(current as u64), - if limit > 0 { current * 100 / limit } else { 0 }, + (current * 100).checked_div(limit).unwrap_or(0), ReadableSize(stream_tracked as u64), ReadableSize(limit as u64) ); @@ -613,7 +613,7 @@ impl StreamMemoryTracker { waited, ReadableSize(additional as u64), ReadableSize(current as u64), - if limit > 0 { current * 100 / limit } else { 0 }, + (current * 100).checked_div(limit).unwrap_or(0), ReadableSize(self.tracked_bytes as u64), ReadableSize(limit as u64) ); diff --git a/src/common/recordbatch/src/recordbatch.rs b/src/common/recordbatch/src/recordbatch.rs index f6e0aeed93..2e92b9e87a 100644 --- a/src/common/recordbatch/src/recordbatch.rs +++ b/src/common/recordbatch/src/recordbatch.rs @@ -437,7 +437,7 @@ fn maybe_align_json_array_with_schema( } let mut aligned = Vec::with_capacity(arrays.len()); - for (field, array) in schema.fields().iter().zip(arrays.into_iter()) { + for (field, array) in schema.fields().iter().zip(arrays) { if !is_json_extension_type(field) { aligned.push(array); continue; diff --git a/src/common/sql/src/default_constraint.rs b/src/common/sql/src/default_constraint.rs index 0084320835..bad9f374fd 100644 --- a/src/common/sql/src/default_constraint.rs +++ b/src/common/sql/src/default_constraint.rs @@ -122,7 +122,7 @@ pub fn parse_column_default_constraint( #[cfg(test)] mod test { - use std::assert_matches::assert_matches; + use std::assert_matches; use datatypes::prelude::{ConcreteDataType, Value}; use datatypes::types::BooleanType; diff --git a/src/common/sql/src/lib.rs b/src/common/sql/src/lib.rs index abca883124..8835cae6b4 100644 --- a/src/common/sql/src/lib.rs +++ b/src/common/sql/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] - pub mod convert; pub mod default_constraint; pub mod error; diff --git a/src/common/telemetry/src/lib.rs b/src/common/telemetry/src/lib.rs index cd60d61645..26bf5d53b3 100644 --- a/src/common/telemetry/src/lib.rs +++ b/src/common/telemetry/src/lib.rs @@ -21,10 +21,12 @@ mod panic_hook; pub mod tracing_context; mod tracing_sampler; +pub use common_error; pub use logging::{ LOG_RELOAD_HANDLE, TRACE_RELOAD_HANDLE, get_or_init_tracer, init_default_ut_logging, init_global_logging, }; pub use metric::dump_metrics; pub use panic_hook::set_panic_hook; -pub use {common_error, tracing, tracing_subscriber}; +pub use tracing; +pub use tracing_subscriber; diff --git a/src/common/wal/src/lib.rs b/src/common/wal/src/lib.rs index 659a045f57..a0b1dc99f9 100644 --- a/src/common/wal/src/lib.rs +++ b/src/common/wal/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] - use std::net::SocketAddr; use error::{EndpointIPV4NotFoundSnafu, ResolveEndpointSnafu, Result}; @@ -59,7 +57,7 @@ async fn resolve_to_ipv4_one>(endpoint: T) -> Result { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_telemetry::warn; use rskafka::client::{Credentials, SaslConfig}; diff --git a/src/datanode/src/datanode.rs b/src/datanode/src/datanode.rs index 859235fa9f..c848215d39 100644 --- a/src/datanode/src/datanode.rs +++ b/src/datanode/src/datanode.rs @@ -551,14 +551,15 @@ impl DatanodeBuilder { if kafka_config.create_index && opts.node_id.is_none() { warn!("The WAL index creation only available in distributed mode.") } - let global_index_collector = if kafka_config.create_index && opts.node_id.is_some() + let global_index_collector = if kafka_config.create_index + && let Some(node_id) = opts.node_id { let operator = new_object_store_without_cache( &opts.storage.store, &opts.storage.data_home, ) .await?; - let path = default_index_file(opts.node_id.unwrap()); + let path = default_index_file(node_id); Some(Self::build_global_index_collector( kafka_config.dump_index_interval, operator, @@ -782,7 +783,7 @@ async fn open_all_regions( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; diff --git a/src/datanode/src/heartbeat/handler.rs b/src/datanode/src/heartbeat/handler.rs index defc910573..10948a3e7c 100644 --- a/src/datanode/src/heartbeat/handler.rs +++ b/src/datanode/src/heartbeat/handler.rs @@ -295,7 +295,7 @@ impl HeartbeatResponseHandler for RegionHeartbeatResponseHandler { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; diff --git a/src/datanode/src/heartbeat/handler/close_region.rs b/src/datanode/src/heartbeat/handler/close_region.rs index 819fcc4880..484edf5ff4 100644 --- a/src/datanode/src/heartbeat/handler/close_region.rs +++ b/src/datanode/src/heartbeat/handler/close_region.rs @@ -47,7 +47,7 @@ impl InstructionHandler for CloseRegionsHandler { let results = join_all(futs).await; let mut errors = vec![]; - for (region_id, result) in region_ids.into_iter().zip(results.into_iter()) { + for (region_id, result) in region_ids.into_iter().zip(results) { match result { Ok(_) => (), Err(error::Error::RegionNotFound { .. }) => { @@ -79,7 +79,6 @@ mod tests { use std::assert_matches; use std::sync::Arc; - use assert_matches::assert_matches; use common_meta::RegionIdent; use common_meta::heartbeat::handler::{HandleControl, HeartbeatResponseHandler}; use common_meta::heartbeat::mailbox::MessageMeta; diff --git a/src/datanode/src/heartbeat/handler/downgrade_region.rs b/src/datanode/src/heartbeat/handler/downgrade_region.rs index 40d9765ca2..f0ec37c844 100644 --- a/src/datanode/src/heartbeat/handler/downgrade_region.rs +++ b/src/datanode/src/heartbeat/handler/downgrade_region.rs @@ -225,7 +225,7 @@ impl HandlerContext { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use std::time::Duration; diff --git a/src/datanode/src/heartbeat/handler/open_region.rs b/src/datanode/src/heartbeat/handler/open_region.rs index 91ba618d9a..56c07a3efe 100644 --- a/src/datanode/src/heartbeat/handler/open_region.rs +++ b/src/datanode/src/heartbeat/handler/open_region.rs @@ -72,7 +72,7 @@ impl InstructionHandler for OpenRegionsHandler { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/datanode/src/heartbeat/handler/upgrade_region.rs b/src/datanode/src/heartbeat/handler/upgrade_region.rs index b2036a6ef4..c06e8aa845 100644 --- a/src/datanode/src/heartbeat/handler/upgrade_region.rs +++ b/src/datanode/src/heartbeat/handler/upgrade_region.rs @@ -183,9 +183,10 @@ impl UpgradeRegionsHandler { .await { Ok(responses) => { - replies.extend( - Self::convert_responses_to_replies(responses, &catchup_regions).into_iter(), - ); + replies.extend(Self::convert_responses_to_replies( + responses, + &catchup_regions, + )); } Err(_) => { replies.extend(catchup_regions.iter().map(|region_id| UpgradeRegionReply { diff --git a/src/datanode/src/lib.rs b/src/datanode/src/lib.rs index 55d2b1796d..7e0db3cabc 100644 --- a/src/datanode/src/lib.rs +++ b/src/datanode/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] - pub mod alive_keeper; pub mod config; pub mod datanode; diff --git a/src/datanode/src/region_server.rs b/src/datanode/src/region_server.rs index 9d675aa276..ec10691bea 100644 --- a/src/datanode/src/region_server.rs +++ b/src/datanode/src/region_server.rs @@ -1667,7 +1667,7 @@ impl RegionAttribute { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use api::v1::SemanticType; use common_error::ext::ErrorExt; diff --git a/src/datatypes/src/json.rs b/src/datatypes/src/json.rs index 3bebbf89aa..db657abbcb 100644 --- a/src/datatypes/src/json.rs +++ b/src/datatypes/src/json.rs @@ -426,7 +426,7 @@ fn decode_struct_with_context<'a>( let (items, fields) = struct_value.into_parts(); - for (field, field_value) in fields.fields().iter().zip(items.into_iter()) { + for (field, field_value) in fields.fields().iter().zip(items) { let field_context = context.with_key(field.name()); let json_value = decode_value_with_context(field_value, &field_context)?; json_object.insert(field.name().to_string(), json_value); @@ -561,7 +561,7 @@ fn decode_struct_with_settings<'a>( // Process each field in the struct value let (struct_data, fields) = struct_value.into_parts(); - for (field, value) in fields.fields().iter().zip(struct_data.into_iter()) { + for (field, value) in fields.fields().iter().zip(struct_data) { let field_context = context.with_key(field.name()); // Check if this field should be treated as unstructured diff --git a/src/datatypes/src/lib.rs b/src/datatypes/src/lib.rs index 2c3d4c23bf..0dcce16857 100644 --- a/src/datatypes/src/lib.rs +++ b/src/datatypes/src/lib.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] -#![feature(box_patterns)] - pub mod arrow_array; pub mod data_type; pub mod duration; diff --git a/src/datatypes/src/types/json_type.rs b/src/datatypes/src/types/json_type.rs index 912bbfca54..13aeffb26c 100644 --- a/src/datatypes/src/types/json_type.rs +++ b/src/datatypes/src/types/json_type.rs @@ -773,7 +773,7 @@ mod tests { r#"Failed to merge JSON datatype: datatypes have conflict, this: {"hello":"","list":[""],"object":{"a":""}}, that: """#, r#"Failed to merge JSON datatype: datatypes have conflict, this: {"hello":"","list":[""],"object":{"a":""}}, that: [""]"#, ]; - for (json, expect) in jsons.into_iter().zip(expects.into_iter()) { + for (json, expect) in jsons.into_iter().zip(expects) { test(json, json_type, Err(expect))?; } diff --git a/src/datatypes/src/value.rs b/src/datatypes/src/value.rs index ab64c801e5..8cfb8da7ad 100644 --- a/src/datatypes/src/value.rs +++ b/src/datatypes/src/value.rs @@ -922,7 +922,7 @@ impl TryFrom for serde_json::Value { let map = struct_type .fields() .iter() - .zip(items.into_iter()) + .zip(items) .map(|(field, value)| { Ok(( field.name().to_string(), @@ -2723,26 +2723,26 @@ pub(crate) mod tests { .unwrap() ); assert_eq!( - ScalarValue::UInt8(Some(u8::MIN + 1)), - Value::UInt8(u8::MIN + 1) + ScalarValue::UInt8(Some(1)), + Value::UInt8(1) .try_to_scalar_value(&ConcreteDataType::uint8_datatype()) .unwrap() ); assert_eq!( - ScalarValue::UInt16(Some(u16::MIN + 2)), - Value::UInt16(u16::MIN + 2) + ScalarValue::UInt16(Some(2)), + Value::UInt16(2) .try_to_scalar_value(&ConcreteDataType::uint16_datatype()) .unwrap() ); assert_eq!( - ScalarValue::UInt32(Some(u32::MIN + 3)), - Value::UInt32(u32::MIN + 3) + ScalarValue::UInt32(Some(3)), + Value::UInt32(3) .try_to_scalar_value(&ConcreteDataType::uint32_datatype()) .unwrap() ); assert_eq!( - ScalarValue::UInt64(Some(u64::MIN + 4)), - Value::UInt64(u64::MIN + 4) + ScalarValue::UInt64(Some(4)), + Value::UInt64(4) .try_to_scalar_value(&ConcreteDataType::uint64_datatype()) .unwrap() ); diff --git a/src/datatypes/src/vectors.rs b/src/datatypes/src/vectors.rs index 5355b35ff4..7c7d2a4ad6 100644 --- a/src/datatypes/src/vectors.rs +++ b/src/datatypes/src/vectors.rs @@ -307,10 +307,11 @@ macro_rules! impl_extend_for_builder { }}; } -pub(crate) use { - impl_extend_for_builder, impl_get_for_vector, impl_get_ref_for_vector, - impl_try_from_arrow_array_for_vector, impl_validity_for_vector, -}; +pub(crate) use impl_extend_for_builder; +pub(crate) use impl_get_for_vector; +pub(crate) use impl_get_ref_for_vector; +pub(crate) use impl_try_from_arrow_array_for_vector; +pub(crate) use impl_validity_for_vector; #[cfg(test)] pub mod tests { diff --git a/src/datatypes/src/vectors/binary.rs b/src/datatypes/src/vectors/binary.rs index 21d188199c..6d9954fb80 100644 --- a/src/datatypes/src/vectors/binary.rs +++ b/src/datatypes/src/vectors/binary.rs @@ -458,7 +458,7 @@ impl BinaryVector { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use arrow::datatypes::DataType as ArrowDataType; use common_base::bytes::Bytes; diff --git a/src/datatypes/src/vectors/json/builder.rs b/src/datatypes/src/vectors/json/builder.rs index ecc19f4bdd..58b4073666 100644 --- a/src/datatypes/src/vectors/json/builder.rs +++ b/src/datatypes/src/vectors/json/builder.rs @@ -328,7 +328,7 @@ mod tests { ), ]; let mut builder = JsonVectorBuilder::new(JsonNativeType::Null, 1); - for (json, result) in jsons.into_iter().zip(results.into_iter()) { + for (json, result) in jsons.into_iter().zip(results) { push(json, &mut builder, result); } let vector = builder.to_vector(); @@ -448,7 +448,7 @@ mod tests { for (builder, (expect_type, expect_vector)) in builder .builders .iter() - .zip(expect_types.into_iter().zip(expect_vectors.into_iter())) + .zip(expect_types.into_iter().zip(expect_vectors)) { assert_eq!(builder.json_type.name(), expect_type); let vector = builder.inner.to_vector_cloned(); diff --git a/src/file-engine/src/lib.rs b/src/file-engine/src/lib.rs index cc9bac8c6f..51d13946cd 100644 --- a/src/file-engine/src/lib.rs +++ b/src/file-engine/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] - pub mod config; pub mod engine; pub mod error; diff --git a/src/file-engine/src/region.rs b/src/file-engine/src/region.rs index c17e797966..3808b33a67 100644 --- a/src/file-engine/src/region.rs +++ b/src/file-engine/src/region.rs @@ -105,7 +105,7 @@ impl FileRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use store_api::region_request::PathType; diff --git a/src/flow/src/adapter/flownode_impl.rs b/src/flow/src/adapter/flownode_impl.rs index d1e4600e23..976df56c9e 100644 --- a/src/flow/src/adapter/flownode_impl.rs +++ b/src/flow/src/adapter/flownode_impl.rs @@ -1060,7 +1060,7 @@ impl StreamingEngine { let fetch_order: Vec = table_col_names .iter() - .zip(default_vals.into_iter()) + .zip(default_vals) .map(|(col_name, col_default_val)| { name_to_col .get(col_name) diff --git a/src/flow/src/lib.rs b/src/flow/src/lib.rs index bd4cbd9f08..fe8a760a07 100644 --- a/src/flow/src/lib.rs +++ b/src/flow/src/lib.rs @@ -17,7 +17,6 @@ //! It also contains definition of expression, adapter and plan, and internal state management. #![allow(dead_code)] -#![warn(clippy::missing_docs_in_private_items)] #![warn(clippy::too_many_lines)] // TODO(discord9): enable this lint to handle out of bound access diff --git a/src/flow/src/utils.rs b/src/flow/src/utils.rs index 8f28ed23c7..1a9879b996 100644 --- a/src/flow/src/utils.rs +++ b/src/flow/src/utils.rs @@ -213,7 +213,7 @@ impl KeyExpiryManager { let mut before = self.event_ts_to_key.split_off(&expire_time); std::mem::swap(&mut before, &mut self.event_ts_to_key); - Some(before.into_iter().flat_map(|(_ts, keys)| keys.into_iter())) + Some(before.into_values().flat_map(|keys| keys.into_iter())) } } @@ -409,8 +409,8 @@ impl Arrangement { // iter over batches that only have updates of `timestamp>now` and find the first non empty batch, then get the minimum timestamp in that batch for (_ts, batch) in self.spine.range((Bound::Excluded(now), Bound::Unbounded)) { let min_ts = batch - .iter() - .flat_map(|(_k, v)| v.iter().map(|(_, ts, _)| *ts).min()) + .values() + .flat_map(|v| v.iter().map(|(_, ts, _)| *ts).min()) .min(); if min_ts.is_some() { diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 16321795b7..c170236073 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] - pub mod error; pub mod events; pub mod frontend; diff --git a/src/index/src/inverted_index/format/reader/footer.rs b/src/index/src/inverted_index/format/reader/footer.rs index 866021c6e6..5d6ac922c9 100644 --- a/src/index/src/inverted_index/format/reader/footer.rs +++ b/src/index/src/inverted_index/format/reader/footer.rs @@ -173,7 +173,7 @@ impl InvertedIndexFooterReader { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use prost::Message; diff --git a/src/index/src/lib.rs b/src/index/src/lib.rs index c469acb8c3..7969ece891 100644 --- a/src/index/src/lib.rs +++ b/src/index/src/lib.rs @@ -13,7 +13,6 @@ // limitations under the License. #![feature(iter_partition_in_place)] -#![feature(assert_matches)] pub mod bitmap; pub mod bloom_filter; diff --git a/src/log-query/src/log_query.rs b/src/log-query/src/log_query.rs index c5b71c6efb..2955b7bded 100644 --- a/src/log-query/src/log_query.rs +++ b/src/log-query/src/log_query.rs @@ -199,57 +199,53 @@ impl TimeFilter { let mut start_dt = None; let mut end_dt = None; - if self.start.is_some() && self.end.is_none() && self.span.is_none() { - // Only 'start' is provided - let s = self.start.as_ref().unwrap(); - let (start, end_opt) = Self::parse_datetime(s)?; - if end_opt.is_none() { + match (&self.start, &self.end, &self.span) { + (Some(start), None, None) => { + let (start, end_opt) = Self::parse_datetime(start)?; + if end_opt.is_none() { + return Err(InvalidTimeFilterSnafu { + filter: self.clone(), + } + .build()); + } + start_dt = Some(start); + end_dt = end_opt; + } + (Some(start), Some(end), _) => { + // Both 'start' and 'end' are provided + let (start, _) = Self::parse_datetime(start)?; + let (end, _) = Self::parse_datetime(end)?; + start_dt = Some(start); + end_dt = Some(end); + } + (Some(start), None, Some(span)) => { + let (start, _) = Self::parse_datetime(start)?; + let span = Self::parse_span(span)?; + let end = start + span; + start_dt = Some(start); + end_dt = Some(end); + } + (None, Some(end), Some(span)) => { + let (end, _) = Self::parse_datetime(end)?; + let span = Self::parse_span(span)?; + let start = end - span; + start_dt = Some(start); + end_dt = Some(end); + } + (None, None, Some(span)) => { + let span = Self::parse_span(span)?; + let end = Utc::now(); + let start = end - span; + start_dt = Some(start); + end_dt = Some(end); + } + _ => { + // Exception return Err(InvalidTimeFilterSnafu { filter: self.clone(), } .build()); } - start_dt = Some(start); - end_dt = end_opt; - } else if self.start.is_some() && self.end.is_some() { - // Both 'start' and 'end' are provided - let (start, _) = Self::parse_datetime(self.start.as_ref().unwrap())?; - let (end, _) = Self::parse_datetime(self.end.as_ref().unwrap())?; - start_dt = Some(start); - end_dt = Some(end); - } else if self.span.is_some() && (self.start.is_some() || self.end.is_some()) { - // 'span' with 'start' or 'end' - let span = Self::parse_span(self.span.as_ref().unwrap())?; - if self.start.is_some() { - let (start, _) = Self::parse_datetime(self.start.as_ref().unwrap())?; - let end = start + span; - start_dt = Some(start); - end_dt = Some(end); - } else { - let (end, _) = Self::parse_datetime(self.end.as_ref().unwrap())?; - let start = end - span; - start_dt = Some(start); - end_dt = Some(end); - } - } else if self.span.is_some() && self.start.is_none() && self.end.is_none() { - // Only 'span' is provided - let span = Self::parse_span(self.span.as_ref().unwrap())?; - let end = Utc::now(); - let start = end - span; - start_dt = Some(start); - end_dt = Some(end); - } else if self.start.is_some() && self.span.is_some() && self.end.is_some() { - // All fields are provided; 'start' and 'end' take priority - let (start, _) = Self::parse_datetime(self.start.as_ref().unwrap())?; - let (end, _) = Self::parse_datetime(self.end.as_ref().unwrap())?; - start_dt = Some(start); - end_dt = Some(end); - } else { - // Exception - return Err(InvalidTimeFilterSnafu { - filter: self.clone(), - } - .build()); } // Validate that end is after start diff --git a/src/log-store/src/kafka/index/iterator.rs b/src/log-store/src/kafka/index/iterator.rs index 9ab350036d..ba8e7273d7 100644 --- a/src/log-store/src/kafka/index/iterator.rs +++ b/src/log-store/src/kafka/index/iterator.rs @@ -61,11 +61,7 @@ impl RegionWalRange { fn next_batch_size(&self) -> Option { if self.current_entry_id < self.end_entry_id { - Some( - self.end_entry_id - .checked_sub(self.current_entry_id) - .unwrap_or_default(), - ) + Some(self.end_entry_id.saturating_sub(self.current_entry_id)) } else { None } diff --git a/src/log-store/src/kafka/log_store.rs b/src/log-store/src/kafka/log_store.rs index 702e5bf319..e7fd06816d 100644 --- a/src/log-store/src/kafka/log_store.rs +++ b/src/log-store/src/kafka/log_store.rs @@ -550,7 +550,7 @@ fn check_termination(offset: i64, end_offset: i64) -> bool { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; diff --git a/src/log-store/src/kafka/util/record.rs b/src/log-store/src/kafka/util/record.rs index 1e291d9776..720f989139 100644 --- a/src/log-store/src/kafka/util/record.rs +++ b/src/log-store/src/kafka/util/record.rs @@ -306,7 +306,7 @@ pub(crate) fn maybe_emit_entry( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use super::*; diff --git a/src/log-store/src/lib.rs b/src/log-store/src/lib.rs index ec8207d5eb..c054fbc1d8 100644 --- a/src/log-store/src/lib.rs +++ b/src/log-store/src/lib.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(io_error_more)] -#![feature(assert_matches)] - pub mod error; pub mod kafka; pub mod metrics; diff --git a/src/meta-client/src/client/cluster.rs b/src/meta-client/src/client/cluster.rs index 8da45aaa7a..976626b3a0 100644 --- a/src/meta-client/src/client/cluster.rs +++ b/src/meta-client/src/client/cluster.rs @@ -212,8 +212,8 @@ impl Inner { } } } - } else if let Err(err) = leader_provider.ask_leader().await { - return Err(err); + } else { + leader_provider.ask_leader().await?; } } diff --git a/src/meta-client/src/client/procedure.rs b/src/meta-client/src/client/procedure.rs index 93e37511d9..63f45c28fb 100644 --- a/src/meta-client/src/client/procedure.rs +++ b/src/meta-client/src/client/procedure.rs @@ -208,8 +208,8 @@ impl Inner { } } } - } else if let Err(err) = leader_provider.ask_leader().await { - return Err(err); + } else { + leader_provider.ask_leader().await?; } } diff --git a/src/meta-srv/src/gc/candidate.rs b/src/meta-srv/src/gc/candidate.rs index 05fc79ac52..2101318cdf 100644 --- a/src/meta-srv/src/gc/candidate.rs +++ b/src/meta-srv/src/gc/candidate.rs @@ -109,7 +109,7 @@ impl GcScheduler { } // Sort candidates by score in descending order and take top N - candidates.sort_by(|a, b| b.score.cmp(&a.score)); + candidates.sort_by_key(|a| std::cmp::Reverse(a.score)); let top_candidates: Vec = candidates .into_iter() .take(self.config.regions_per_table_threshold) diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index 105ddca58c..c62e927f89 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -346,7 +346,7 @@ impl GcScheduler { // Add to need_retry_regions since it failed combined_report .need_retry_regions - .extend(fast_list_regions.clone().into_iter()); + .extend(fast_list_regions.clone()); } } } diff --git a/src/meta-srv/src/handler.rs b/src/meta-srv/src/handler.rs index 58dd1a7c98..4b05db4e4c 100644 --- a/src/meta-srv/src/handler.rs +++ b/src/meta-srv/src/handler.rs @@ -870,7 +870,7 @@ impl HeartbeatHandlerGroupBuilderCustomizer for DefaultHeartbeatHandlerGroupBuil #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use std::time::Duration; @@ -972,7 +972,7 @@ mod tests { "RemapFlowPeerHandler", ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1009,7 +1009,7 @@ mod tests { "RemapFlowPeerHandler", ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1043,7 +1043,7 @@ mod tests { "RemapFlowPeerHandler", ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1077,7 +1077,7 @@ mod tests { "RemapFlowPeerHandler", ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1111,7 +1111,7 @@ mod tests { "RemapFlowPeerHandler", ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1145,7 +1145,7 @@ mod tests { ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1179,7 +1179,7 @@ mod tests { ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } @@ -1212,7 +1212,7 @@ mod tests { "RemapFlowPeerHandler", ]; assert_eq!(names.len(), handlers.len()); - for (handler, name) in handlers.iter().zip(names.into_iter()) { + for (handler, name) in handlers.iter().zip(names) { assert_eq!(handler.name, name); } } diff --git a/src/meta-srv/src/lib.rs b/src/meta-srv/src/lib.rs index 0e87d4421a..70ce449bba 100644 --- a/src/meta-srv/src/lib.rs +++ b/src/meta-srv/src/lib.rs @@ -12,10 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] #![feature(hash_set_entry)] #![feature(duration_constructors)] -#![feature(string_from_utf8_lossy_owned)] pub mod bootstrap; pub mod cache_invalidator; diff --git a/src/meta-srv/src/procedure/region_migration.rs b/src/meta-srv/src/procedure/region_migration.rs index b3797860b3..6563da63bc 100644 --- a/src/meta-srv/src/procedure/region_migration.rs +++ b/src/meta-srv/src/procedure/region_migration.rs @@ -924,7 +924,7 @@ impl Procedure for RegionMigrationProcedure { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use common_meta::distributed_time_constants::default_distributed_time_constants; diff --git a/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs b/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs index 37cc0dfa96..b7e8315a77 100644 --- a/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/downgrade_leader_region.rs @@ -372,7 +372,7 @@ impl DowngradeLeaderRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use common_meta::key::table_route::TableRouteValue; diff --git a/src/meta-srv/src/procedure/region_migration/flush_leader_region.rs b/src/meta-srv/src/procedure/region_migration/flush_leader_region.rs index f3dc0ee661..9dee05373b 100644 --- a/src/meta-srv/src/procedure/region_migration/flush_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/flush_leader_region.rs @@ -93,7 +93,7 @@ impl PreFlushRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use store_api::storage::RegionId; diff --git a/src/meta-srv/src/procedure/region_migration/manager.rs b/src/meta-srv/src/procedure/region_migration/manager.rs index 70cba21b5f..d18458b08f 100644 --- a/src/meta-srv/src/procedure/region_migration/manager.rs +++ b/src/meta-srv/src/procedure/region_migration/manager.rs @@ -620,7 +620,7 @@ impl RegionMigrationManager { #[cfg(test)] mod test { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_meta::key::table_route::LogicalTableRouteValue; use common_meta::key::test_utils::new_test_table_info; diff --git a/src/meta-srv/src/procedure/region_migration/migration_start.rs b/src/meta-srv/src/procedure/region_migration/migration_start.rs index 17b577501e..b9d4372c47 100644 --- a/src/meta-srv/src/procedure/region_migration/migration_start.rs +++ b/src/meta-srv/src/procedure/region_migration/migration_start.rs @@ -187,7 +187,7 @@ impl RegionMigrationStart { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_meta::key::test_utils::new_test_table_info; use common_meta::peer::Peer; diff --git a/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs index 8e2e015669..189ba89449 100644 --- a/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/open_candidate_region.rs @@ -208,7 +208,7 @@ impl OpenCandidateRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use common_catalog::consts::MITO2_ENGINE; diff --git a/src/meta-srv/src/procedure/region_migration/test_util.rs b/src/meta-srv/src/procedure/region_migration/test_util.rs index 4e5624401e..269357bc0a 100644 --- a/src/meta-srv/src/procedure/region_migration/test_util.rs +++ b/src/meta-srv/src/procedure/region_migration/test_util.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs index 9e8545bb43..bab79a96bf 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/downgrade_leader_region.rs @@ -88,7 +88,7 @@ impl UpdateMetadata { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs index b3ee848fbe..2f0ed0fbe9 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/rollback_downgraded_region.rs @@ -72,7 +72,7 @@ impl UpdateMetadata { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; diff --git a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs index db70e3e166..d5aa8699ec 100644 --- a/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/update_metadata/upgrade_candidate_region.rs @@ -224,7 +224,7 @@ impl UpdateMetadata { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_meta::key::test_utils::new_test_table_info; use common_meta::peer::Peer; diff --git a/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs b/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs index 39ff1ed741..4c60215cf7 100644 --- a/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs +++ b/src/meta-srv/src/procedure/region_migration/upgrade_candidate_region.rs @@ -353,7 +353,7 @@ impl UpgradeCandidateRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use common_meta::key::table_route::TableRouteValue; diff --git a/src/meta-srv/src/procedure/region_migration/utils.rs b/src/meta-srv/src/procedure/region_migration/utils.rs index df2e8014e2..91af7ccf17 100644 --- a/src/meta-srv/src/procedure/region_migration/utils.rs +++ b/src/meta-srv/src/procedure/region_migration/utils.rs @@ -242,7 +242,7 @@ pub async fn analyze_region_migration_task( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use std::time::Duration; diff --git a/src/meta-srv/src/procedure/repartition/group.rs b/src/meta-srv/src/procedure/repartition/group.rs index 2ef764b634..f0cb1c4dd0 100644 --- a/src/meta-srv/src/procedure/repartition/group.rs +++ b/src/meta-srv/src/procedure/repartition/group.rs @@ -597,7 +597,7 @@ pub(crate) trait State: Sync + Send + Debug { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use common_meta::key::TableMetadataManager; diff --git a/src/meta-srv/src/procedure/repartition/group/enter_staging_region.rs b/src/meta-srv/src/procedure/repartition/group/enter_staging_region.rs index 18ecfd4bb2..59de569c13 100644 --- a/src/meta-srv/src/procedure/repartition/group/enter_staging_region.rs +++ b/src/meta-srv/src/procedure/repartition/group/enter_staging_region.rs @@ -433,7 +433,7 @@ impl EnterStagingRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::time::Duration; use common_meta::instruction::StagingPartitionDirective; diff --git a/src/meta-srv/src/procedure/repartition/group/repartition_start.rs b/src/meta-srv/src/procedure/repartition/group/repartition_start.rs index 72de2f2934..8b8b5208b4 100644 --- a/src/meta-srv/src/procedure/repartition/group/repartition_start.rs +++ b/src/meta-srv/src/procedure/repartition/group/repartition_start.rs @@ -207,7 +207,7 @@ impl State for RepartitionStart { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_meta::peer::Peer; use common_meta::rpc::router::{Region, RegionRoute}; diff --git a/src/meta-srv/src/procedure/repartition/group/sync_region.rs b/src/meta-srv/src/procedure/repartition/group/sync_region.rs index 5e842da2fb..dcd58c21e9 100644 --- a/src/meta-srv/src/procedure/repartition/group/sync_region.rs +++ b/src/meta-srv/src/procedure/repartition/group/sync_region.rs @@ -338,7 +338,7 @@ impl SyncRegion { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_meta::peer::Peer; use common_meta::rpc::router::{Region, RegionRoute}; diff --git a/src/meta-srv/src/procedure/wal_prune.rs b/src/meta-srv/src/procedure/wal_prune.rs index 0897441647..f5e74ef543 100644 --- a/src/meta-srv/src/procedure/wal_prune.rs +++ b/src/meta-srv/src/procedure/wal_prune.rs @@ -193,7 +193,7 @@ impl Procedure for WalPruneProcedure { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_wal::maybe_skip_kafka_integration_test; use common_wal::test_util::get_kafka_endpoints; diff --git a/src/meta-srv/src/procedure/wal_prune/manager.rs b/src/meta-srv/src/procedure/wal_prune/manager.rs index e5f45d229a..1f1ab2b2f0 100644 --- a/src/meta-srv/src/procedure/wal_prune/manager.rs +++ b/src/meta-srv/src/procedure/wal_prune/manager.rs @@ -287,7 +287,7 @@ impl WalPruneManager { #[cfg(test)] mod test { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::time::Duration; use common_meta::key::topic_name::TopicNameKey; diff --git a/src/meta-srv/src/region/supervisor.rs b/src/meta-srv/src/region/supervisor.rs index 90e08992f6..05716e60c8 100644 --- a/src/meta-srv/src/region/supervisor.rs +++ b/src/meta-srv/src/region/supervisor.rs @@ -847,7 +847,7 @@ impl RegionSupervisor { #[cfg(test)] pub(crate) mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use std::sync::{Arc, Mutex}; use std::time::Duration; diff --git a/src/meta-srv/src/service/mailbox.rs b/src/meta-srv/src/service/mailbox.rs index 5e569e6d4d..8b37eeaad5 100644 --- a/src/meta-srv/src/service/mailbox.rs +++ b/src/meta-srv/src/service/mailbox.rs @@ -214,7 +214,7 @@ pub trait Mailbox: Send + Sync { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_time::util::current_time_millis; use tokio::sync::watch; diff --git a/src/meta-srv/src/state.rs b/src/meta-srv/src/state.rs index e5edc5f169..12eb708b5c 100644 --- a/src/meta-srv/src/state.rs +++ b/src/meta-srv/src/state.rs @@ -117,7 +117,7 @@ pub fn become_follower() -> impl FnOnce(&State) -> State { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::state::{FollowerState, LeaderState, State, become_follower, become_leader}; diff --git a/src/metric-engine/src/engine.rs b/src/metric-engine/src/engine.rs index ba90ca960d..c2c39951cc 100644 --- a/src/metric-engine/src/engine.rs +++ b/src/metric-engine/src/engine.rs @@ -575,7 +575,7 @@ struct MetricEngineInner { #[cfg(test)] mod test { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use common_telemetry::info; diff --git a/src/metric-engine/src/engine/bulk_insert.rs b/src/metric-engine/src/engine/bulk_insert.rs index 2a3c26c80c..8122cdc958 100644 --- a/src/metric-engine/src/engine/bulk_insert.rs +++ b/src/metric-engine/src/engine/bulk_insert.rs @@ -380,7 +380,7 @@ fn record_batch_to_ipc(record_batch: &RecordBatch) -> Result<(Bytes, Bytes, Byte #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::sync::Arc; use api::v1::ArrowIpc; diff --git a/src/metric-engine/src/engine/create/extract_new_columns.rs b/src/metric-engine/src/engine/create/extract_new_columns.rs index b3eabd5706..9d1de9ebb2 100644 --- a/src/metric-engine/src/engine/create/extract_new_columns.rs +++ b/src/metric-engine/src/engine/create/extract_new_columns.rs @@ -52,7 +52,7 @@ pub fn extract_new_columns<'a>( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::{HashMap, HashSet}; use api::v1::SemanticType; diff --git a/src/metric-engine/src/lib.rs b/src/metric-engine/src/lib.rs index b93029f2f4..557baba25a 100644 --- a/src/metric-engine/src/lib.rs +++ b/src/metric-engine/src/lib.rs @@ -50,7 +50,7 @@ //! └─────────────────────┘ //! ``` -#![feature(assert_matches)] +#![recursion_limit = "256"] mod batch_modifier; pub mod config; diff --git a/src/metric-engine/src/repeated_task.rs b/src/metric-engine/src/repeated_task.rs index fa382d7844..4354511039 100644 --- a/src/metric-engine/src/repeated_task.rs +++ b/src/metric-engine/src/repeated_task.rs @@ -85,7 +85,7 @@ impl TaskFunction for FlushMetadataRegionTask { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::time::Duration; use store_api::region_engine::{RegionEngine, RegionManifestInfo}; diff --git a/src/metric-engine/src/test_util.rs b/src/metric-engine/src/test_util.rs index d81240d47f..d3e929cf63 100644 --- a/src/metric-engine/src/test_util.rs +++ b/src/metric-engine/src/test_util.rs @@ -167,7 +167,7 @@ impl TestEnv { primary_key: vec![], options: [(PHYSICAL_TABLE_METADATA_KEY.to_string(), String::new())] .into_iter() - .chain(options.into_iter()) + .chain(options) .collect(), table_dir: table_dir.to_string(), path_type: PathType::Bare, // Use Bare path type for engine regions diff --git a/src/mito-codec/src/key_values.rs b/src/mito-codec/src/key_values.rs index 5afacc3718..d66110bacf 100644 --- a/src/mito-codec/src/key_values.rs +++ b/src/mito-codec/src/key_values.rs @@ -431,11 +431,9 @@ mod tests { values: &[Option], ) { assert_eq!(num_rows, kvs.num_rows()); - let mut expect_seq = START_SEQ; let expect_ts = ValueRef::Int64(ts); - for kv in kvs.iter() { + for (expect_seq, kv) in (START_SEQ..).zip(kvs.iter()) { assert_eq!(expect_seq, kv.sequence()); - expect_seq += 1; assert_eq!(OpType::Put, kv.op_type); assert_eq!(keys.len(), kv.num_primary_keys()); assert_eq!(values.len(), kv.num_fields()); diff --git a/src/mito2/src/cache/index.rs b/src/mito2/src/cache/index.rs index 74f16fdb71..a404978714 100644 --- a/src/mito2/src/cache/index.rs +++ b/src/mito2/src/cache/index.rs @@ -229,14 +229,14 @@ where } if !cache_miss_range.is_empty() { let pages = load(cache_miss_range).await?; - for (i, page) in cache_miss_idx.into_iter().zip(pages.into_iter()) { + for (i, page) in cache_miss_idx.into_iter().zip(pages) { let page_key = page_keys[i]; metrics.page_bytes += page.len() as u64; data[i] = page.clone(); self.put_page(key, page_key, page.clone()); } } - let buffer = Buffer::from_iter(data.into_iter()); + let buffer = Buffer::from_iter(data); Ok(( buffer .slice(PageKey::calculate_range(offset, size, self.page_size)) diff --git a/src/mito2/src/compaction.rs b/src/mito2/src/compaction.rs index ba6957fdae..a43fa8a0a6 100644 --- a/src/mito2/src/compaction.rs +++ b/src/mito2/src/compaction.rs @@ -614,15 +614,12 @@ async fn find_dynamic_options( region_options: &crate::region::options::RegionOptions, schema_metadata_manager: &SchemaMetadataManagerRef, ) -> Result<(crate::region::options::CompactionOptions, TimeToLive)> { - if region_options.compaction_override && region_options.ttl.is_some() { + if let (true, Some(ttl)) = (region_options.compaction_override, region_options.ttl) { debug!( "Use region options directly for table {}: compaction={:?}, ttl={:?}", table_id, region_options.compaction, region_options.ttl ); - return Ok(( - region_options.compaction.clone(), - region_options.ttl.unwrap(), - )); + return Ok((region_options.compaction.clone(), ttl)); } let db_options = tokio::time::timeout( @@ -633,12 +630,12 @@ async fn find_dynamic_options( .context(TimeoutSnafu)? .context(GetSchemaMetadataSnafu)?; - let ttl = if region_options.ttl.is_some() { + let ttl = if let Some(ttl) = region_options.ttl { debug!( "Use region TTL directly for table {}: ttl={:?}", table_id, region_options.ttl ); - region_options.ttl.unwrap() + ttl } else { db_options .as_ref() @@ -980,7 +977,7 @@ struct PendingCompaction { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::time::Duration; use api::v1::region::StrictWindow; diff --git a/src/mito2/src/compaction/run.rs b/src/mito2/src/compaction/run.rs index a7e5ca490c..cf1cedd29a 100644 --- a/src/mito2/src/compaction/run.rs +++ b/src/mito2/src/compaction/run.rs @@ -309,7 +309,7 @@ where pub fn reduce_runs(mut runs: Vec>) -> Vec { assert!(runs.len() > 1); // sort runs by size - runs.sort_unstable_by(|a, b| a.size.cmp(&b.size)); + runs.sort_unstable_by_key(|a| a.size); // limit max probe runs to 100 let probe_end = runs.len().min(100); let mut min_penalty = usize::MAX; diff --git a/src/mito2/src/compaction/twcs.rs b/src/mito2/src/compaction/twcs.rs index 9012457f75..952d8771d8 100644 --- a/src/mito2/src/compaction/twcs.rs +++ b/src/mito2/src/compaction/twcs.rs @@ -588,8 +588,8 @@ mod tests { assert_eq!(*overlapping, actual_window.overlapping); let mut file_ranges = actual_window .files - .iter() - .flat_map(|(_, f)| { + .values() + .flat_map(|f| { f.files().iter().map(|f| { let (s, e) = f.time_range(); (s.value(), e.value()) diff --git a/src/mito2/src/engine/alter_test.rs b/src/mito2/src/engine/alter_test.rs index e710e08688..b8ba06f0b9 100644 --- a/src/mito2/src/engine/alter_test.rs +++ b/src/mito2/src/engine/alter_test.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; diff --git a/src/mito2/src/engine/apply_staging_manifest_test.rs b/src/mito2/src/engine/apply_staging_manifest_test.rs index 6904fbd624..401e6572a2 100644 --- a/src/mito2/src/engine/apply_staging_manifest_test.rs +++ b/src/mito2/src/engine/apply_staging_manifest_test.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; -use std::fs; use std::sync::Arc; +use std::{assert_matches, fs}; use api::v1::Rows; use common_function::utils::partition_expr_version; diff --git a/src/mito2/src/engine/catchup_test.rs b/src/mito2/src/engine/catchup_test.rs index 0c7d058e4d..718462e8a8 100644 --- a/src/mito2/src/engine/catchup_test.rs +++ b/src/mito2/src/engine/catchup_test.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::collections::HashMap; use api::v1::Rows; diff --git a/src/mito2/src/engine/copy_region_from_test.rs b/src/mito2/src/engine/copy_region_from_test.rs index 75580d5c0b..e9f8398302 100644 --- a/src/mito2/src/engine/copy_region_from_test.rs +++ b/src/mito2/src/engine/copy_region_from_test.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; -use std::fs; use std::sync::Arc; +use std::{assert_matches, fs}; use api::v1::Rows; use common_error::ext::ErrorExt; diff --git a/src/mito2/src/engine/remap_manifests_test.rs b/src/mito2/src/engine/remap_manifests_test.rs index e3538401aa..339896450c 100644 --- a/src/mito2/src/engine/remap_manifests_test.rs +++ b/src/mito2/src/engine/remap_manifests_test.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use api::v1::Rows; use datatypes::value::Value; diff --git a/src/mito2/src/engine/staging_test.rs b/src/mito2/src/engine/staging_test.rs index 2e9c5045ff..e47a77bea0 100644 --- a/src/mito2/src/engine/staging_test.rs +++ b/src/mito2/src/engine/staging_test.rs @@ -14,10 +14,9 @@ //! Integration tests for staging state functionality. -use std::assert_matches::assert_matches; -use std::fs; use std::sync::Arc; use std::time::Duration; +use std::{assert_matches, fs}; use api::v1::Rows; use common_error::ext::ErrorExt; diff --git a/src/mito2/src/engine/sync_test.rs b/src/mito2/src/engine/sync_test.rs index 65b997e498..6c3b91c130 100644 --- a/src/mito2/src/engine/sync_test.rs +++ b/src/mito2/src/engine/sync_test.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use api::v1::{Rows, SemanticType}; use common_error::ext::ErrorExt; diff --git a/src/mito2/src/lib.rs b/src/mito2/src/lib.rs index a15711b34a..7d43685ded 100644 --- a/src/mito2/src/lib.rs +++ b/src/mito2/src/lib.rs @@ -16,8 +16,6 @@ //! //! Mito is the a region engine to store timeseries data. -#![feature(assert_matches)] -#![feature(int_roundings)] #![feature(debug_closure_helpers)] #![feature(duration_constructors)] diff --git a/src/mito2/src/manifest/tests/checkpoint.rs b/src/mito2/src/manifest/tests/checkpoint.rs index 718755cd46..64547f45aa 100644 --- a/src/mito2/src/manifest/tests/checkpoint.rs +++ b/src/mito2/src/manifest/tests/checkpoint.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::sync::Arc; use std::time::Duration; diff --git a/src/mito2/src/region.rs b/src/mito2/src/region.rs index 3020c9ecf4..26ab96c779 100644 --- a/src/mito2/src/region.rs +++ b/src/mito2/src/region.rs @@ -656,7 +656,7 @@ impl MitoRegion { .unwrap_or_default(); let files = manifest_files .into_iter() - .chain(staging_files.into_iter()) + .chain(staging_files) .collect::>(); files diff --git a/src/mito2/src/region/opener.rs b/src/mito2/src/region/opener.rs index d089493f81..9aa6454f75 100644 --- a/src/mito2/src/region/opener.rs +++ b/src/mito2/src/region/opener.rs @@ -881,7 +881,7 @@ impl RegionLoadCacheTask { } // Sorts files by max timestamp in descending order to loads latest files first - files_to_download.sort_by(|a, b| b.2.cmp(&a.2)); + files_to_download.sort_by_key(|b| std::cmp::Reverse(b.2)); let total_files = files_to_download.len() as i64; @@ -1011,7 +1011,7 @@ async fn preload_parquet_meta_cache_for_files( let allow_direct_load = matches!(object_store.info().scheme(), object_store::Scheme::Fs); // Sort by time range so we can prefer preloading newer files first. - files.sort_by(|a, b| b.meta_ref().time_range.1.cmp(&a.meta_ref().time_range.1)); + files.sort_by_key(|b| std::cmp::Reverse(b.meta_ref().time_range.1)); let mut loaded = 0usize; for file_handle in files { diff --git a/src/mito2/src/sst/index.rs b/src/mito2/src/sst/index.rs index 88aebfc001..31a96eecea 100644 --- a/src/mito2/src/sst/index.rs +++ b/src/mito2/src/sst/index.rs @@ -1083,8 +1083,8 @@ impl IndexBuildScheduler { /// Find the next task which has the highest priority to run. fn find_next_task(&self) -> Option { self.region_status - .iter() - .filter_map(|(_, status)| status.pending_tasks.peek()) + .values() + .filter_map(|status| status.pending_tasks.peek()) .max() .cloned() } diff --git a/src/mito2/src/sst/parquet/row_group.rs b/src/mito2/src/sst/parquet/row_group.rs index 8822882c5d..d8a44dfff5 100644 --- a/src/mito2/src/sst/parquet/row_group.rs +++ b/src/mito2/src/sst/parquet/row_group.rs @@ -220,7 +220,7 @@ pub(crate) fn compute_total_range_size(ranges: &[Range]) -> (u64, u64) { let gap = MERGE_GAP as u64; let mut sorted_ranges = ranges.to_vec(); - sorted_ranges.sort_unstable_by(|a, b| a.start.cmp(&b.start)); + sorted_ranges.sort_unstable_by_key(|a| a.start); let mut total_size_aligned = 0; let mut total_size_unaligned = 0; diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index 2366db7897..e1f121699e 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -343,8 +343,7 @@ impl Inserter { .convert(request) .await?; - let table_infos = - HashMap::from_iter([(table_info.table_id(), table_info.clone())].into_iter()); + let table_infos = HashMap::from_iter([(table_info.table_id(), table_info.clone())]); self.do_request(inserts, &table_infos, &ctx).await } @@ -360,8 +359,7 @@ impl Inserter { .convert(insert, ctx, statement_executor) .await?; - let table_infos = - HashMap::from_iter([(table_info.table_id(), table_info.clone())].into_iter()); + let table_infos = HashMap::from_iter([(table_info.table_id(), table_info.clone())]); self.do_request(inserts, &table_infos, ctx).await } diff --git a/src/operator/src/lib.rs b/src/operator/src/lib.rs index e31ad87ca1..5e723faeb5 100644 --- a/src/operator/src/lib.rs +++ b/src/operator/src/lib.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] -#![feature(if_let_guard)] - mod bulk_insert; pub mod delete; pub mod error; diff --git a/src/operator/src/statement.rs b/src/operator/src/statement.rs index b87be42ff4..f3931255d0 100644 --- a/src/operator/src/statement.rs +++ b/src/operator/src/statement.rs @@ -933,7 +933,7 @@ impl Inserter for InserterImpl { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use common_time::range::TimestampRange; diff --git a/src/partition/src/lib.rs b/src/partition/src/lib.rs index 647210d1d5..3bfd7297cc 100644 --- a/src/partition/src/lib.rs +++ b/src/partition/src/lib.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] //! Structs and traits for partitioning rule. pub mod cache; diff --git a/src/partition/src/multi_dim.rs b/src/partition/src/multi_dim.rs index 7b1f7aa3dd..8825c6de59 100644 --- a/src/partition/src/multi_dim.rs +++ b/src/partition/src/multi_dim.rs @@ -356,7 +356,7 @@ impl PartitionRule for MultiDimPartitionRule { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use super::*; use crate::error::{self, Error}; diff --git a/src/pipeline/src/etl/ctx_req.rs b/src/pipeline/src/etl/ctx_req.rs index 23873cfdf1..0bfac33dac 100644 --- a/src/pipeline/src/etl/ctx_req.rs +++ b/src/pipeline/src/etl/ctx_req.rs @@ -223,7 +223,7 @@ impl ContextReq { } pub fn all_req(self) -> impl Iterator { - self.req.into_iter().flat_map(|(_, req)| req) + self.req.into_values().flatten() } pub fn ref_all_req(&self) -> impl Iterator { diff --git a/src/pipeline/src/etl/processor/dissect.rs b/src/pipeline/src/etl/processor/dissect.rs index 6694049253..69bdbe0f89 100644 --- a/src/pipeline/src/etl/processor/dissect.rs +++ b/src/pipeline/src/etl/processor/dissect.rs @@ -527,7 +527,7 @@ impl DissectProcessor { }; for (name, mut values) in appends { - values.sort_by(|a, b| a.1.cmp(&b.1)); + values.sort_by_key(|a| a.1); let value = values.into_iter().map(|(a, _)| a).join(sep); map.push((name, VrlValue::Bytes(Bytes::from(value)))); } diff --git a/src/pipeline/src/etl/processor/filter.rs b/src/pipeline/src/etl/processor/filter.rs index a700a8f2d3..d1820ba8ff 100644 --- a/src/pipeline/src/etl/processor/filter.rs +++ b/src/pipeline/src/etl/processor/filter.rs @@ -158,20 +158,18 @@ impl Processor for FilterProcessor { for field in self.fields.iter() { let val = val.as_object_mut().context(ValueMustBeMapSnafu)?; let index = field.input_field(); - match val.get(index) { - Some(VrlValue::Bytes(b)) => { + if let Some(v) = val.get(index) { + if let VrlValue::Bytes(b) = v { if self.match_target(&String::from_utf8_lossy(b)) { return Ok(VrlValue::Null); } - } - Some(v) => { + } else { return ProcessorExpectStringSnafu { processor: self.kind(), v: v.clone(), } .fail(); } - None => {} } } diff --git a/src/promql/src/extension_plan/range_manipulate.rs b/src/promql/src/extension_plan/range_manipulate.rs index 6d57de62c6..ac03ff5e79 100644 --- a/src/promql/src/extension_plan/range_manipulate.rs +++ b/src/promql/src/extension_plan/range_manipulate.rs @@ -753,7 +753,7 @@ impl RangeManipulateStream { if ts <= curr_ts { range_end = range_end.max(cursor); } else { - range_start_index = range_start_index.checked_sub(1usize).unwrap_or_default(); + range_start_index = range_start_index.saturating_sub(1usize); break; } cursor += 1; diff --git a/src/promql/src/extension_plan/series_divide.rs b/src/promql/src/extension_plan/series_divide.rs index 4a0b32f4e3..19385b7a97 100644 --- a/src/promql/src/extension_plan/series_divide.rs +++ b/src/promql/src/extension_plan/series_divide.rs @@ -632,7 +632,7 @@ impl SeriesDivideStream { let tags = TagIdentifier::try_new(batch, &self.tag_indices)?; // check if the first row is the same with last batch's last row - if resumed_batch_index > self.inspect_start.checked_sub(1).unwrap_or_default() { + if resumed_batch_index > self.inspect_start.saturating_sub(1) { let last_batch = &self.buffer[resumed_batch_index - 1]; let last_row = last_batch.num_rows() - 1; let last_tags = TagIdentifier::try_new(last_batch, &self.tag_indices)?; diff --git a/src/promql/src/functions/extrapolate_rate.rs b/src/promql/src/functions/extrapolate_rate.rs index 126a506bc9..8c3ab88776 100644 --- a/src/promql/src/functions/extrapolate_rate.rs +++ b/src/promql/src/functions/extrapolate_rate.rs @@ -96,7 +96,7 @@ impl ExtrapolatedRate() .unwrap() - .value(0) as i64; + .value(0); Ok(Self::new(range_length)) } diff --git a/src/promql/src/functions/quantile.rs b/src/promql/src/functions/quantile.rs index 93fc632d68..35f0c1aa81 100644 --- a/src/promql/src/functions/quantile.rs +++ b/src/promql/src/functions/quantile.rs @@ -216,7 +216,7 @@ fn quantile_with_scratch(values: &[f64], quantile: f64, scratch: &mut Vec) let length = scratch.len(); let rank = quantile * (length - 1) as f64; - let lower_index = 0.max(rank.floor() as usize); + let lower_index = rank.floor() as usize; let upper_index = (length - 1).min(lower_index + 1); let weight = rank - rank.floor(); diff --git a/src/puffin/src/puffin_manager/fs_puffin_manager/reader.rs b/src/puffin/src/puffin_manager/fs_puffin_manager/reader.rs index c660d1e19a..b35b03875e 100644 --- a/src/puffin/src/puffin_manager/fs_puffin_manager/reader.rs +++ b/src/puffin/src/puffin_manager/fs_puffin_manager/reader.rs @@ -262,7 +262,7 @@ where tasks.push(task); } - let size = futures::future::try_join_all(tasks.into_iter()) + let size = futures::future::try_join_all(tasks) .await .into_iter() .flatten() diff --git a/src/query/src/dist_plan/analyzer/utils.rs b/src/query/src/dist_plan/analyzer/utils.rs index e9205e33f4..dd1ad867e5 100644 --- a/src/query/src/dist_plan/analyzer/utils.rs +++ b/src/query/src/dist_plan/analyzer/utils.rs @@ -89,7 +89,7 @@ pub fn patch_batch_timezone( let patched_columns: Vec = expected_schema .fields() .iter() - .zip(columns.into_iter()) + .zip(columns) .map(|(expected_field, column)| { let expected_type = expected_field.data_type(); let actual_type = column.data_type(); diff --git a/src/query/src/optimizer/parallelize_scan.rs b/src/query/src/optimizer/parallelize_scan.rs index 171fc6e919..dd2ba06290 100644 --- a/src/query/src/optimizer/parallelize_scan.rs +++ b/src/query/src/optimizer/parallelize_scan.rs @@ -139,7 +139,7 @@ impl ParallelizeScan { } // Sort ranges by number of rows in descending order. - ranges.sort_by(|a, b| b.num_rows.cmp(&a.num_rows)); + ranges.sort_by_key(|b| std::cmp::Reverse(b.num_rows)); let mut partition_ranges = vec![vec![]; expected_partition_num]; #[derive(Eq, PartialEq)] diff --git a/src/query/src/optimizer/scan_hint.rs b/src/query/src/optimizer/scan_hint.rs index da70813404..89e3afaed0 100644 --- a/src/query/src/optimizer/scan_hint.rs +++ b/src/query/src/optimizer/scan_hint.rs @@ -375,10 +375,10 @@ impl TreeNodeVisitor<'_> for ScanHintVisitor { LogicalPlan::Filter(_) => { self.vector_search.on_filter_exit(); } - LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) => { - if is_branching_for_vector(_node) { - self.vector_search.on_branching_exit(); - } + LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) + if is_branching_for_vector(_node) => + { + self.vector_search.on_branching_exit(); } _ if _node.inputs().len() > 1 => { self.vector_search.on_branching_exit(); diff --git a/src/query/src/promql/planner.rs b/src/query/src/promql/planner.rs index 23d654d2b6..640994dea2 100644 --- a/src/query/src/promql/planner.rs +++ b/src/query/src/promql/planner.rs @@ -523,8 +523,7 @@ impl PromPlanner { .use_tsid .then_some(DfExpr::Column(Column::from_name( DATA_SCHEMA_TSID_COLUMN_NAME, - ))) - .into_iter(), + ))), ) .chain(Some(self.create_time_index_column_expr()?)); @@ -1313,8 +1312,7 @@ impl PromPlanner { .use_tsid .then_some(DfExpr::Column(Column::new_unqualified( DATA_SCHEMA_TSID_COLUMN_NAME, - ))) - .into_iter(), + ))), ) .chain(Some(self.create_time_index_column_expr()?)) .collect::>(); @@ -1828,15 +1826,10 @@ impl PromPlanner { .iter() .map(|tag| DfExpr::Column(Column::from_name(tag))), ) - .chain( - self.ctx - .use_tsid - .then_some(DfExpr::Column(Column::new( - Some(table_ref.clone()), - DATA_SCHEMA_TSID_COLUMN_NAME.to_string(), - ))) - .into_iter(), - ) + .chain(self.ctx.use_tsid.then_some(DfExpr::Column(Column::new( + Some(table_ref.clone()), + DATA_SCHEMA_TSID_COLUMN_NAME.to_string(), + )))) .chain(Some(DfExpr::Alias(Alias { expr: Box::new(DfExpr::Cast(Cast { expr: Box::new(self.create_time_index_column_expr()?), @@ -1874,8 +1867,7 @@ impl PromPlanner { .use_tsid .then_some(DfExpr::Column(Column::from_name( DATA_SCHEMA_TSID_COLUMN_NAME, - ))) - .into_iter(), + ))), ) .chain(Some(self.create_time_index_column_expr()?)) .collect::>(); diff --git a/src/query/src/sql.rs b/src/query/src/sql.rs index 49e26c92ca..74f8b13fea 100644 --- a/src/query/src/sql.rs +++ b/src/query/src/sql.rs @@ -215,10 +215,10 @@ pub async fn show_databases( fn replace_column_in_expr(expr: &mut sqlparser::ast::Expr, from_column: &str, to_column: &str) { let _ = visit_expressions_mut(expr, |e| { match e { - sqlparser::ast::Expr::Identifier(ident) => { - if ident.value.eq_ignore_ascii_case(from_column) { - ident.value = to_column.to_string(); - } + sqlparser::ast::Expr::Identifier(ident) + if ident.value.eq_ignore_ascii_case(from_column) => + { + ident.value = to_column.to_string(); } sqlparser::ast::Expr::CompoundIdentifier(idents) => { if let Some(last) = idents.last_mut() @@ -748,23 +748,17 @@ pub fn show_variable(stmt: ShowVariables, query_ctx: QueryContextRef) -> Result< .pg_intervalstyle_format(); style.to_string() } - "MAX_EXECUTION_TIME" => { - if query_ctx.channel() == Channel::Mysql { + "MAX_EXECUTION_TIME" + if query_ctx.channel() == Channel::Mysql => { query_ctx.query_timeout_as_millis().to_string() - } else { - return UnsupportedVariableSnafu { name: variable }.fail(); } - } - "STATEMENT_TIMEOUT" => { + "STATEMENT_TIMEOUT" // Add time units to postgres query timeout display. - if query_ctx.channel() == Channel::Postgres { + if query_ctx.channel() == Channel::Postgres => { let mut timeout = query_ctx.query_timeout_as_millis().to_string(); timeout.push_str("ms"); timeout - } else { - return UnsupportedVariableSnafu { name: variable }.fail(); } - } _ => return UnsupportedVariableSnafu { name: variable }.fail(), }; let schema = Arc::new(Schema::new(vec![ColumnSchema::new( diff --git a/src/query/src/window_sort.rs b/src/query/src/window_sort.rs index b667b9c3b0..83feee6f30 100644 --- a/src/query/src/window_sort.rs +++ b/src/query/src/window_sort.rs @@ -1198,7 +1198,7 @@ fn split_overlapping_ranges(ranges: &[PartitionRange]) -> BTreeMap (Option<&str>, Option<&str>) #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_base::secrets::ExposeSecret; diff --git a/src/servers/src/http/header.rs b/src/servers/src/http/header.rs index ce58e15e98..785901ad7b 100644 --- a/src/servers/src/http/header.rs +++ b/src/servers/src/http/header.rs @@ -170,13 +170,11 @@ pub fn collect_plan_metrics(plan: &Arc, maps: &mut [&mut Hash MetricValue::Gauge { name, gauge } => { collect_into_maps(name, gauge.value() as u64, maps); } - MetricValue::Time { name, time } => { - if name.starts_with(GREPTIME_EXEC_PREFIX) { - // override - maps.iter_mut().for_each(|map| { - map.insert(name.to_string(), time.value() as u64); - }); - } + MetricValue::Time { name, time } if name.starts_with(GREPTIME_EXEC_PREFIX) => { + // override + maps.iter_mut().for_each(|map| { + map.insert(name.to_string(), time.value() as u64); + }); } _ => {} }); diff --git a/src/servers/src/http/prometheus.rs b/src/servers/src/http/prometheus.rs index 488a49df99..60ad780beb 100644 --- a/src/servers/src/http/prometheus.rs +++ b/src/servers/src/http/prometheus.rs @@ -1118,17 +1118,17 @@ fn collect_metric_names(expr: &PromqlExpr, metric_names: &mut HashSet) { match expr { PromqlExpr::Aggregate(AggregateExpr { modifier, expr, .. }) => { match modifier { - Some(LabelModifier::Include(labels)) => { - if !labels.labels.contains(&METRIC_NAME.to_string()) { - metric_names.clear(); - return; - } + Some(LabelModifier::Include(labels)) + if !labels.labels.contains(&METRIC_NAME.to_string()) => + { + metric_names.clear(); + return; } - Some(LabelModifier::Exclude(labels)) => { - if labels.labels.contains(&METRIC_NAME.to_string()) { - metric_names.clear(); - return; - } + Some(LabelModifier::Exclude(labels)) + if labels.labels.contains(&METRIC_NAME.to_string()) => + { + metric_names.clear(); + return; } _ => {} } diff --git a/src/servers/src/http/result/prometheus_resp.rs b/src/servers/src/http/result/prometheus_resp.rs index 9ecbe671b4..6f2b115686 100644 --- a/src/servers/src/http/result/prometheus_resp.rs +++ b/src/servers/src/http/result/prometheus_resp.rs @@ -204,10 +204,10 @@ impl PrometheusJsonResponse { for (i, column) in batches.schema().column_schemas().iter().enumerate() { match column.data_type { - ConcreteDataType::Timestamp(datatypes::types::TimestampType::Millisecond(_)) => { - if timestamp_column_index.is_none() { - timestamp_column_index = Some(i); - } + ConcreteDataType::Timestamp(datatypes::types::TimestampType::Millisecond(_)) + if timestamp_column_index.is_none() => + { + timestamp_column_index = Some(i); } // Treat all value types as field ConcreteDataType::Float32(_) @@ -219,10 +219,10 @@ impl PrometheusJsonResponse { | ConcreteDataType::UInt8(_) | ConcreteDataType::UInt16(_) | ConcreteDataType::UInt32(_) - | ConcreteDataType::UInt64(_) => { - if first_field_column_index.is_none() { - first_field_column_index = Some(i); - } + | ConcreteDataType::UInt64(_) + if first_field_column_index.is_none() => + { + first_field_column_index = Some(i); } ConcreteDataType::String(_) => { tag_column_indices.push(i); diff --git a/src/servers/src/lib.rs b/src/servers/src/lib.rs index c44c674b9e..41d73b109f 100644 --- a/src/servers/src/lib.rs +++ b/src/servers/src/lib.rs @@ -12,11 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] #![feature(try_blocks)] #![feature(exclusive_wrapper)] -#![feature(if_let_guard)] -#![feature(box_patterns)] use datafusion_expr::LogicalPlan; use datatypes::schema::Schema; diff --git a/src/servers/src/mysql/handler.rs b/src/servers/src/mysql/handler.rs index dd67012a52..2ce229fea4 100644 --- a/src/servers/src/mysql/handler.rs +++ b/src/servers/src/mysql/handler.rs @@ -674,10 +674,8 @@ fn convert_param_value_to_string(param: &ParamValue) -> String { fn replace_params(params: Vec, query: String) -> String { let mut query = query; - let mut index = 1; - for param in params { + for (index, param) in (1..).zip(params) { query = query.replace(&format_placeholder(index), ¶m); - index += 1; } query } diff --git a/src/servers/tests/mysql/mysql_server_test.rs b/src/servers/tests/mysql/mysql_server_test.rs index 3aa7b98f39..c53ff34d45 100644 --- a/src/servers/tests/mysql/mysql_server_test.rs +++ b/src/servers/tests/mysql/mysql_server_test.rs @@ -514,8 +514,7 @@ async fn test_prepare_all_type( connection: &mut Conn, ) { let mut column_index = 0; - let mut stmt_id = 1; - for schema in column_schemas { + for (stmt_id, schema) in (1..).zip(column_schemas) { let query = format!( "SELECT {} FROM all_datatypes WHERE {} = ?", schema.name, schema.name @@ -523,7 +522,6 @@ async fn test_prepare_all_type( let statement = connection.prep(query).await; let statement = statement.unwrap(); assert_eq!(stmt_id, statement.id()); - stmt_id += 1; let vector_ref = columns.get(column_index).unwrap(); for vector_index in 0..vector_ref.len() { diff --git a/src/sql/src/lib.rs b/src/sql/src/lib.rs index b2b151d439..e8c6bdf8ef 100644 --- a/src/sql/src/lib.rs +++ b/src/sql/src/lib.rs @@ -13,8 +13,6 @@ // limitations under the License. #![feature(box_patterns)] -#![feature(assert_matches)] -#![feature(if_let_guard)] pub mod ast; pub mod dialect; diff --git a/src/sql/src/parsers/alter_parser.rs b/src/sql/src/parsers/alter_parser.rs index df89f90922..e5e1575a20 100644 --- a/src/sql/src/parsers/alter_parser.rs +++ b/src/sql/src/parsers/alter_parser.rs @@ -677,7 +677,7 @@ fn parse_string_option_names(parser: &mut Parser) -> std::result::Result { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/sql/src/parsers/copy_parser.rs b/src/sql/src/parsers/copy_parser.rs index d975d884f6..9a2eddcc78 100644 --- a/src/sql/src/parsers/copy_parser.rs +++ b/src/sql/src/parsers/copy_parser.rs @@ -267,7 +267,7 @@ impl ParserContext<'_> { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use sqlparser::ast::{Ident, ObjectName}; diff --git a/src/sql/src/parsers/create_parser.rs b/src/sql/src/parsers/create_parser.rs index d69a1af61d..b83c2032db 100644 --- a/src/sql/src/parsers/create_parser.rs +++ b/src/sql/src/parsers/create_parser.rs @@ -1242,7 +1242,7 @@ fn ensure_partition_columns_defined<'a>( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use std::collections::HashMap; use common_catalog::consts::FILE_ENGINE; diff --git a/src/sql/src/parsers/delete_parser.rs b/src/sql/src/parsers/delete_parser.rs index 3f13d18f97..be1a93b6c6 100644 --- a/src/sql/src/parsers/delete_parser.rs +++ b/src/sql/src/parsers/delete_parser.rs @@ -43,7 +43,7 @@ impl ParserContext<'_> { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use super::*; use crate::dialect::GreptimeDbDialect; diff --git a/src/sql/src/parsers/insert_parser.rs b/src/sql/src/parsers/insert_parser.rs index a7d5f311f6..d121181774 100644 --- a/src/sql/src/parsers/insert_parser.rs +++ b/src/sql/src/parsers/insert_parser.rs @@ -64,7 +64,7 @@ impl ParserContext<'_> { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use super::*; use crate::dialect::GreptimeDbDialect; diff --git a/src/sql/src/parsers/show_parser.rs b/src/sql/src/parsers/show_parser.rs index e2e5fc50ac..d6fc35c675 100644 --- a/src/sql/src/parsers/show_parser.rs +++ b/src/sql/src/parsers/show_parser.rs @@ -599,7 +599,7 @@ impl ParserContext<'_> { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use sqlparser::ast::{Ident, ObjectName}; diff --git a/src/sql/src/statements/alter.rs b/src/sql/src/statements/alter.rs index 5b6a5ab5e6..ab35e5bd34 100644 --- a/src/sql/src/statements/alter.rs +++ b/src/sql/src/statements/alter.rs @@ -390,7 +390,7 @@ impl Display for AlterDatabaseOperation { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/sql/src/statements/copy.rs b/src/sql/src/statements/copy.rs index 7aa099c53c..7a59175538 100644 --- a/src/sql/src/statements/copy.rs +++ b/src/sql/src/statements/copy.rs @@ -164,7 +164,7 @@ impl CopyTableArgument { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/sql/src/statements/create.rs b/src/sql/src/statements/create.rs index 817b31518d..80eb52c406 100644 --- a/src/sql/src/statements/create.rs +++ b/src/sql/src/statements/create.rs @@ -709,7 +709,7 @@ impl Display for CreateView { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::error::Error; diff --git a/src/sql/src/statements/describe.rs b/src/sql/src/statements/describe.rs index 2a87725dfc..ec48f4be81 100644 --- a/src/sql/src/statements/describe.rs +++ b/src/sql/src/statements/describe.rs @@ -44,7 +44,7 @@ impl Display for DescribeTable { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/sql/src/statements/drop.rs b/src/sql/src/statements/drop.rs index 1e97bee25c..26190b48a0 100644 --- a/src/sql/src/statements/drop.rs +++ b/src/sql/src/statements/drop.rs @@ -167,7 +167,7 @@ impl Display for DropView { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/sql/src/statements/set_variables.rs b/src/sql/src/statements/set_variables.rs index 748d077d84..4c6eb16692 100644 --- a/src/sql/src/statements/set_variables.rs +++ b/src/sql/src/statements/set_variables.rs @@ -41,7 +41,7 @@ impl Display for SetVariables { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/sql/src/statements/show.rs b/src/sql/src/statements/show.rs index 0dfdd1de7d..77880e4a50 100644 --- a/src/sql/src/statements/show.rs +++ b/src/sql/src/statements/show.rs @@ -340,7 +340,7 @@ impl Display for ShowProcessList { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use sqlparser::ast::UnaryOperator; diff --git a/src/sql/src/statements/truncate.rs b/src/sql/src/statements/truncate.rs index b9c299601c..307326d9c7 100644 --- a/src/sql/src/statements/truncate.rs +++ b/src/sql/src/statements/truncate.rs @@ -98,7 +98,7 @@ impl Display for TruncateTable { #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use crate::dialect::GreptimeDbDialect; use crate::parser::{ParseOptions, ParserContext}; diff --git a/src/store-api/src/lib.rs b/src/store-api/src/lib.rs index 4df594fc67..cb39875d74 100644 --- a/src/store-api/src/lib.rs +++ b/src/store-api/src/lib.rs @@ -14,8 +14,6 @@ //! Storage related APIs -#![feature(iterator_try_collect)] - pub mod codec; pub mod data_source; pub mod logstore; diff --git a/src/table/src/lib.rs b/src/table/src/lib.rs index 64e72029b8..abc7dc8300 100644 --- a/src/table/src/lib.rs +++ b/src/table/src/lib.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] #![feature(try_blocks)] pub mod dist_table; diff --git a/src/table/src/metadata.rs b/src/table/src/metadata.rs index ca8bc30aa1..42f66044d9 100644 --- a/src/table/src/metadata.rs +++ b/src/table/src/metadata.rs @@ -1388,7 +1388,7 @@ fn unset_column_skipping_index_options( #[cfg(test)] mod tests { - use std::assert_matches::assert_matches; + use std::assert_matches; use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; diff --git a/tests-integration/Cargo.toml b/tests-integration/Cargo.toml index ec35205a55..bee03ae7fe 100644 --- a/tests-integration/Cargo.toml +++ b/tests-integration/Cargo.toml @@ -3,6 +3,11 @@ name = "tests-integration" version.workspace = true edition.workspace = true license.workspace = true +autotests = false + +[[test]] +name = "main" +path = "tests/main.rs" [features] dashboard = ["servers/dashboard"] diff --git a/tests-integration/src/cluster.rs b/tests-integration/src/cluster.rs index daa2f64919..4a73014dbf 100644 --- a/tests-integration/src/cluster.rs +++ b/tests-integration/src/cluster.rs @@ -127,8 +127,8 @@ impl GreptimeDbCluster { .await .into_iter() .flat_map(|e| { - if e.index_file_path.is_some() { - vec![e.file_path, e.index_file_path.unwrap()] + if let Some(index_file_path) = e.index_file_path { + vec![e.file_path, index_file_path] } else { vec![e.file_path] } diff --git a/tests-integration/src/lib.rs b/tests-integration/src/lib.rs index 5def9351d0..5a4dca146f 100644 --- a/tests-integration/src/lib.rs +++ b/tests-integration/src/lib.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(assert_matches)] +#![recursion_limit = "256"] pub mod cluster; mod grpc; diff --git a/tests-integration/src/tests/instance_kafka_wal_test.rs b/tests-integration/src/tests/instance_kafka_wal_test.rs index ed74525aef..57be7e6cd6 100644 --- a/tests-integration/src/tests/instance_kafka_wal_test.rs +++ b/tests-integration/src/tests/instance_kafka_wal_test.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; +use std::assert_matches; use std::sync::Arc; use std::sync::atomic::{AtomicU64, Ordering}; diff --git a/tests-integration/tests/main.rs b/tests-integration/tests/main.rs index 14cf734291..01aed6c8e9 100644 --- a/tests-integration/tests/main.rs +++ b/tests-integration/tests/main.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![recursion_limit = "256"] + #[macro_use] mod grpc; #[macro_use] From fb2dd862d5b29f07b30511bce3e9919d0c789b8a Mon Sep 17 00:00:00 2001 From: Yao Noel Achi <43069141+ynachi@users.noreply.github.com> Date: Tue, 31 Mar 2026 03:51:09 +0200 Subject: [PATCH 10/20] fix: avoid cloning serialized view plans on resolve (#7882) fix: avoid cloning serialized view plan on resolve - Change `ViewInfoValue.view_info` from `Vec` to `common_base::bytes::Bytes` so resolving a view no longer clones the full serialized plan buffer on every decode. - To keep the change narrow, the metadata write boundary still accepts `Vec` and converts once when constructing/updating `ViewInfoValue`. The hot read path now uses a cheap clone of the stored bytes. - The benches introduced revealed up to 82% resolution time improvment. Signed-off-by: Yao ACHI --- src/catalog/src/table_source.rs | 7 +--- src/common/meta/src/key.rs | 8 ++-- src/common/meta/src/key/view_info.rs | 61 ++++++++++++++++++++++++---- 3 files changed, 59 insertions(+), 17 deletions(-) diff --git a/src/catalog/src/table_source.rs b/src/catalog/src/table_source.rs index f7ba51722f..fd78cc2573 100644 --- a/src/catalog/src/table_source.rs +++ b/src/catalog/src/table_source.rs @@ -15,7 +15,6 @@ use std::collections::HashMap; use std::sync::Arc; -use bytes::Bytes; use common_catalog::format_full_table_name; use common_query::logical_plan::{SubstraitPlanDecoderRef, rename_logical_plan_columns}; use datafusion::common::{ResolvedTableReference, TableReference}; @@ -151,11 +150,7 @@ impl DfTableSourceProvider { let catalog_list = Arc::new(DummyCatalogList::new(self.catalog_manager.clone())); let logical_plan = self .plan_decoder - .decode( - Bytes::from(view_info.view_info.clone()), - catalog_list, - false, - ) + .decode(view_info.view_info.clone().into(), catalog_list, false) .await .context(DecodePlanSnafu { name: &table.table_info().name, diff --git a/src/common/meta/src/key.rs b/src/common/meta/src/key.rs index 97b68f9b04..332c60f225 100644 --- a/src/common/meta/src/key.rs +++ b/src/common/meta/src/key.rs @@ -708,7 +708,7 @@ impl TableMetadataManager { // Creates view info let view_info_value = ViewInfoValue::new( - raw_logical_plan, + raw_logical_plan.into(), table_names, columns, plan_columns, @@ -1184,7 +1184,7 @@ impl TableMetadataManager { definition: String, ) -> Result<()> { let new_view_info_value = current_view_info_value.update( - new_view_info, + new_view_info.into(), table_names, columns, plan_columns, @@ -2752,7 +2752,7 @@ mod tests { let new_definition = "CREATE VIEW test AS SELECT * FROM b_table join c_table"; let current_view_info_value = DeserializedValueWithBytes::from_inner(ViewInfoValue::new( - logical_plan.clone(), + logical_plan.clone().into(), table_names, columns, plan_columns, @@ -2803,7 +2803,7 @@ mod tests { let wrong_definition = "wrong_definition"; let wrong_view_info_value = DeserializedValueWithBytes::from_inner(current_view_info_value.update( - wrong_view_info, + wrong_view_info.into(), new_table_names.clone(), new_columns.clone(), new_plan_columns.clone(), diff --git a/src/common/meta/src/key/view_info.rs b/src/common/meta/src/key/view_info.rs index 82be00b26d..5efc2c9bb7 100644 --- a/src/common/meta/src/key/view_info.rs +++ b/src/common/meta/src/key/view_info.rs @@ -16,6 +16,7 @@ use std::collections::{HashMap, HashSet}; use std::fmt::Display; use std::sync::Arc; +use common_base::bytes::Bytes; use serde::{Deserialize, Serialize}; use snafu::OptionExt; use table::metadata::TableId; @@ -31,9 +32,6 @@ use crate::kv_backend::KvBackendRef; use crate::kv_backend::txn::Txn; use crate::rpc::store::BatchGetRequest; -/// The VIEW logical plan encoded bytes -type RawViewLogicalPlan = Vec; - /// The key stores the metadata of the view. /// /// The layout: `__view_info/{view_id}`. @@ -86,7 +84,7 @@ impl MetadataKey<'_, ViewInfoKey> for ViewInfoKey { #[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct ViewInfoValue { // The encoded logical plan - pub view_info: RawViewLogicalPlan, + pub view_info: Bytes, // The resolved fully table names in logical plan pub table_names: HashSet, // The view columns @@ -100,7 +98,7 @@ pub struct ViewInfoValue { impl ViewInfoValue { pub fn new( - view_info: RawViewLogicalPlan, + view_info: Bytes, table_names: HashSet, columns: Vec, plan_columns: Vec, @@ -118,7 +116,7 @@ impl ViewInfoValue { pub(crate) fn update( &self, - new_view_info: RawViewLogicalPlan, + new_view_info: Bytes, table_names: HashSet, columns: Vec, plan_columns: Vec, @@ -305,7 +303,7 @@ mod tests { }; let value = ViewInfoValue { - view_info: vec![1, 2, 3], + view_info: Bytes::from([1, 2, 3].as_ref()), version: 1, table_names, columns: vec!["a".to_string()], @@ -316,4 +314,53 @@ mod tests { let deserialized = ViewInfoValue::try_from_raw_value(&serialized).unwrap(); assert_eq!(value, deserialized); } + + #[test] + fn test_deserialize_view_info_value_with_vec_u8() { + #[derive(Serialize)] + struct OldViewInfoValue { + view_info: Vec, + table_names: HashSet, + columns: Vec, + plan_columns: Vec, + definition: String, + version: u64, + } + + let table_names = { + let mut set = HashSet::new(); + set.insert(TableName { + catalog_name: "greptime".to_string(), + schema_name: "public".to_string(), + table_name: "a_table".to_string(), + }); + set.insert(TableName { + catalog_name: "greptime".to_string(), + schema_name: "public".to_string(), + table_name: "b_table".to_string(), + }); + set + }; + + let old_value = OldViewInfoValue { + view_info: vec![1, 2, 3], + table_names: table_names.clone(), + columns: vec!["a".to_string()], + plan_columns: vec!["number".to_string()], + definition: "CREATE VIEW test AS SELECT * FROM numbers".to_string(), + version: 1, + }; + + let serialized = serde_json::to_vec(&old_value).unwrap(); + let deserialized = ViewInfoValue::try_from_raw_value(&serialized).unwrap(); + + assert_eq!(deserialized.view_info, vec![1, 2, 3]); + assert_eq!(deserialized.table_names, table_names); + assert_eq!(deserialized.columns, vec!["a".to_string()]); + assert_eq!(deserialized.plan_columns, vec!["number".to_string()]); + assert_eq!( + deserialized.definition, + "CREATE VIEW test AS SELECT * FROM numbers" + ); + } } From dde1edcdb4ca32bbeed0c4e1d1ae1a5cfcbcea0d Mon Sep 17 00:00:00 2001 From: Yingwen Date: Tue, 31 Mar 2026 10:45:07 +0800 Subject: [PATCH 11/20] fix: incorrect prefilter check (#7886) Signed-off-by: evenyag --- src/mito2/src/sst/parquet/flat_format.rs | 6 +++--- src/mito2/src/sst/parquet/prefilter.rs | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/mito2/src/sst/parquet/flat_format.rs b/src/mito2/src/sst/parquet/flat_format.rs index ca39cac7e1..d4d6c11a45 100644 --- a/src/mito2/src/sst/parquet/flat_format.rs +++ b/src/mito2/src/sst/parquet/flat_format.rs @@ -282,10 +282,10 @@ impl FlatReadFormat { } } - /// Returns `true` if raw batches from parquet use the flat layout with a - /// dictionary-encoded `__primary_key` column (i.e., [`ParquetAdapter::Flat`]). + /// Returns `true` if raw batches from parquet use the flat layout and + /// stores primary key columns as raw columns. /// Returns `false` for the legacy primary-key-to-flat conversion path. - pub(crate) fn raw_batch_has_primary_key_dictionary(&self) -> bool { + pub(crate) fn batch_has_raw_pk_columns(&self) -> bool { matches!(&self.parquet_adapter, ParquetAdapter::Flat(_)) } diff --git a/src/mito2/src/sst/parquet/prefilter.rs b/src/mito2/src/sst/parquet/prefilter.rs index 07efbd052f..88df56e401 100644 --- a/src/mito2/src/sst/parquet/prefilter.rs +++ b/src/mito2/src/sst/parquet/prefilter.rs @@ -217,9 +217,9 @@ impl PrefilterContextBuilder { return None; } - // Only flat format with dictionary-encoded PKs supports PK prefiltering. + // Only perform PK prefiltering for primary-key-to-flat conversion path. let flat_format = read_format.as_flat()?; - if !flat_format.raw_batch_has_primary_key_dictionary() { + if flat_format.batch_has_raw_pk_columns() { return None; } From 358524566a7d300d8be712ebd2b430c0bb0a5585 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Tue, 31 Mar 2026 18:46:30 +0800 Subject: [PATCH 12/20] chore: update ignore list for AI related (#7896) Signed-off-by: shuiyisong --- .gitignore | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.gitignore b/.gitignore index 87412d570c..1e6194369f 100644 --- a/.gitignore +++ b/.gitignore @@ -65,11 +65,12 @@ greptimedb_data # github !/.github -# Claude code +# AI related CLAUDE.md - -# AGENTS.md AGENTS.md +.codex +.gemini +.opencode # local design docs docs/specs/ From ab106966574bea30296931b69050400e8b73a3aa Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Tue, 31 Mar 2026 15:43:40 -0700 Subject: [PATCH 13/20] feat: implement export-v2 chunked data export flow (#7841) * feat: implement export-v2 chunked data export flow Signed-off-by: jeremyhi * fix: by codex comment Signed-off-by: jeremyhi * fix: by gemini comment Signed-off-by: jeremyhi * fix: clippy Signed-off-by: jeremyhi * fix: by comment Signed-off-by: jeremyhi * fix: handle empty export ranges consistently Signed-off-by: jeremyhi * fix: validate resume config Signed-off-by: jeremyhi * fix: file-uri paths Signed-off-by: jeremyhi * feat: check args on schema-only mode Signed-off-by: jeremyhi --------- Signed-off-by: jeremyhi --- src/cli/src/data/export_v2.rs | 5 +- src/cli/src/data/export_v2/chunker.rs | 103 +++++ src/cli/src/data/export_v2/command.rs | 467 ++++++++++++++++++++-- src/cli/src/data/export_v2/coordinator.rs | 166 ++++++++ src/cli/src/data/export_v2/data.rs | 440 ++++++++++++++++++++ src/cli/src/data/export_v2/error.rs | 56 ++- src/cli/src/data/export_v2/manifest.rs | 190 ++++++++- src/cli/src/data/import_v2/command.rs | 131 +++++- src/cli/src/data/import_v2/error.rs | 18 +- src/cli/src/data/path.rs | 13 + src/cli/src/data/snapshot_storage.rs | 57 ++- 11 files changed, 1577 insertions(+), 69 deletions(-) create mode 100644 src/cli/src/data/export_v2/chunker.rs create mode 100644 src/cli/src/data/export_v2/coordinator.rs create mode 100644 src/cli/src/data/export_v2/data.rs diff --git a/src/cli/src/data/export_v2.rs b/src/cli/src/data/export_v2.rs index 91020d2f2e..1921ffe4b4 100644 --- a/src/cli/src/data/export_v2.rs +++ b/src/cli/src/data/export_v2.rs @@ -30,7 +30,7 @@ //! --to file:///tmp/snapshot \ //! --schema-only //! -//! # Export with time range (M2) +//! # Export with time range //! greptime cli data export-v2 create \ //! --addr 127.0.0.1:4000 \ //! --to s3://bucket/snapshots/prod-20250101 \ @@ -38,7 +38,10 @@ //! --end-time 2025-01-31T23:59:59Z //! ``` +mod chunker; mod command; +mod coordinator; +mod data; pub mod error; pub mod extractor; pub mod manifest; diff --git a/src/cli/src/data/export_v2/chunker.rs b/src/cli/src/data/export_v2/chunker.rs new file mode 100644 index 0000000000..260d95fae9 --- /dev/null +++ b/src/cli/src/data/export_v2/chunker.rs @@ -0,0 +1,103 @@ +// 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::time::Duration; + +use chrono::Duration as ChronoDuration; + +use crate::data::export_v2::manifest::{ChunkMeta, TimeRange}; + +pub fn generate_chunks(time_range: &TimeRange, window: Duration) -> Vec { + let (Some(start), Some(end)) = (time_range.start, time_range.end) else { + return vec![ChunkMeta::new(1, time_range.clone())]; + }; + + if start == end { + return vec![ChunkMeta::skipped(1, time_range.clone())]; + } + + if start > end { + return Vec::new(); + } + + let window = match ChronoDuration::from_std(window) { + Ok(window) if window > ChronoDuration::zero() => window, + _ => return vec![ChunkMeta::new(1, time_range.clone())], + }; + + let mut chunks = Vec::new(); + let mut cursor = start; + let mut id = 1; + + while cursor < end { + let next = cursor + .checked_add_signed(window) + .map_or(end, |timestamp| timestamp.min(end)); + chunks.push(ChunkMeta::new(id, TimeRange::new(Some(cursor), Some(next)))); + id += 1; + cursor = next; + } + + chunks +} + +#[cfg(test)] +mod tests { + use chrono::{TimeZone, Utc}; + + use super::*; + use crate::data::export_v2::manifest::ChunkStatus; + + #[test] + fn test_generate_chunks_unbounded() { + let range = TimeRange::unbounded(); + let chunks = generate_chunks(&range, Duration::from_secs(3600)); + assert_eq!(chunks.len(), 1); + assert_eq!(chunks[0].time_range, range); + } + + #[test] + fn test_generate_chunks_split() { + let start = Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let end = Utc.with_ymd_and_hms(2025, 1, 1, 3, 0, 0).unwrap(); + let range = TimeRange::new(Some(start), Some(end)); + + let chunks = generate_chunks(&range, Duration::from_secs(3600)); + assert_eq!(chunks.len(), 3); + assert_eq!(chunks[0].time_range.start, Some(start)); + assert_eq!( + chunks[2].time_range.end, + Some(Utc.with_ymd_and_hms(2025, 1, 1, 3, 0, 0).unwrap()) + ); + } + + #[test] + fn test_generate_chunks_empty_range() { + let start = Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let range = TimeRange::new(Some(start), Some(start)); + let chunks = generate_chunks(&range, Duration::from_secs(3600)); + assert_eq!(chunks.len(), 1); + assert_eq!(chunks[0].status, ChunkStatus::Skipped); + assert_eq!(chunks[0].time_range, range); + } + + #[test] + fn test_generate_chunks_invalid_range_is_empty() { + let start = Utc.with_ymd_and_hms(2025, 1, 1, 1, 0, 0).unwrap(); + let end = Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let range = TimeRange::new(Some(start), Some(end)); + let chunks = generate_chunks(&range, Duration::from_secs(3600)); + assert!(chunks.is_empty()); + } +} diff --git a/src/cli/src/data/export_v2/command.rs b/src/cli/src/data/export_v2/command.rs index 341436fe0f..ddcb323fef 100644 --- a/src/cli/src/data/export_v2/command.rs +++ b/src/cli/src/data/export_v2/command.rs @@ -26,12 +26,16 @@ use snafu::{OptionExt, ResultExt}; use crate::Tool; use crate::common::ObjectStoreConfig; +use crate::data::export_v2::coordinator::export_data; use crate::data::export_v2::error::{ - CannotResumeSchemaOnlySnafu, DataExportNotImplementedSnafu, DatabaseSnafu, EmptyResultSnafu, - ManifestVersionMismatchSnafu, Result, UnexpectedValueTypeSnafu, + ChunkTimeWindowRequiresBoundsSnafu, DatabaseSnafu, EmptyResultSnafu, + ManifestVersionMismatchSnafu, Result, ResumeConfigMismatchSnafu, SchemaOnlyArgsNotAllowedSnafu, + SchemaOnlyModeMismatchSnafu, UnexpectedValueTypeSnafu, }; use crate::data::export_v2::extractor::SchemaExtractor; -use crate::data::export_v2::manifest::{DataFormat, MANIFEST_VERSION, Manifest}; +use crate::data::export_v2::manifest::{ + ChunkMeta, DataFormat, MANIFEST_VERSION, Manifest, TimeRange, +}; use crate::data::path::ddl_path_for_schema; use crate::data::snapshot_storage::{OpenDalStorage, SnapshotStorage, validate_uri}; use crate::data::sql::{escape_sql_identifier, escape_sql_literal}; @@ -84,6 +88,11 @@ pub struct ExportCreateCommand { #[clap(long)] end_time: Option, + /// Chunk time window (e.g., 1h, 6h, 1d, 7d). + /// Requires both --start-time and --end-time when specified. + #[clap(long, value_parser = humantime::parse_duration)] + chunk_time_window: Option, + /// Data format: parquet, csv, json. #[clap(long, value_enum, default_value = "parquet")] format: DataFormat, @@ -92,7 +101,7 @@ pub struct ExportCreateCommand { #[clap(long)] force: bool, - /// Concurrency level (for future use). + /// Parallelism for COPY DATABASE execution (server-side, per schema per chunk). #[clap(long, default_value = "1")] parallelism: usize, @@ -127,11 +136,38 @@ impl ExportCreateCommand { // Validate URI format validate_uri(&self.to).map_err(BoxedError::new)?; - if !self.schema_only { - return DataExportNotImplementedSnafu + let time_range = TimeRange::parse(self.start_time.as_deref(), self.end_time.as_deref()) + .map_err(BoxedError::new)?; + if self.chunk_time_window.is_some() && !time_range.is_bounded() { + return ChunkTimeWindowRequiresBoundsSnafu .fail() .map_err(BoxedError::new); } + if self.schema_only { + let mut invalid_args = Vec::new(); + if self.start_time.is_some() { + invalid_args.push("--start-time"); + } + if self.end_time.is_some() { + invalid_args.push("--end-time"); + } + if self.chunk_time_window.is_some() { + invalid_args.push("--chunk-time-window"); + } + if self.format != DataFormat::Parquet { + invalid_args.push("--format"); + } + if self.parallelism != 1 { + invalid_args.push("--parallelism"); + } + if !invalid_args.is_empty() { + return SchemaOnlyArgsNotAllowedSnafu { + args: invalid_args.join(", "), + } + .fail() + .map_err(BoxedError::new); + } + } // Parse schemas (empty vec means all schemas) let schemas = if self.schemas.is_empty() { @@ -155,12 +191,18 @@ impl ExportCreateCommand { ); Ok(Box::new(ExportCreate { - catalog: self.catalog.clone(), - schemas, - schema_only: self.schema_only, - _format: self.format, - force: self.force, - _parallelism: self.parallelism, + config: ExportConfig { + catalog: self.catalog.clone(), + schemas, + schema_only: self.schema_only, + format: self.format, + force: self.force, + time_range, + chunk_time_window: self.chunk_time_window, + parallelism: self.parallelism, + snapshot_uri: self.to.clone(), + storage_config: self.storage.clone(), + }, storage: Box::new(storage), database_client, })) @@ -169,14 +211,22 @@ impl ExportCreateCommand { /// Export tool implementation. pub struct ExportCreate { + config: ExportConfig, + storage: Box, + database_client: DatabaseClient, +} + +struct ExportConfig { catalog: String, schemas: Option>, schema_only: bool, - _format: DataFormat, + format: DataFormat, force: bool, - _parallelism: usize, - storage: Box, - database_client: DatabaseClient, + time_range: TimeRange, + chunk_time_window: Option, + parallelism: usize, + snapshot_uri: String, + storage_config: ObjectStoreConfig, } #[async_trait] @@ -192,12 +242,12 @@ impl ExportCreate { let exists = self.storage.exists().await?; if exists { - if self.force { + if self.config.force { info!("Deleting existing snapshot (--force)"); self.storage.delete_snapshot().await?; } else { // Resume mode - read existing manifest - let manifest = self.storage.read_manifest().await?; + let mut manifest = self.storage.read_manifest().await?; // Check version compatibility if manifest.version != MANIFEST_VERSION { @@ -208,10 +258,7 @@ impl ExportCreate { .fail(); } - // Cannot resume schema-only with data export - if manifest.schema_only && !self.schema_only { - return CannotResumeSchemaOnlySnafu.fail(); - } + validate_resume_config(&manifest, &self.config)?; info!( "Resuming existing snapshot: {} (completed: {}/{} chunks)", @@ -220,22 +267,31 @@ impl ExportCreate { manifest.chunks.len() ); - // For M1, we only handle schema-only exports - // M2 will add chunk resume logic if manifest.is_complete() { info!("Snapshot is already complete"); return Ok(()); } - // TODO: Resume data export in M2 - info!("Data export resume not yet implemented (M2)"); + if manifest.schema_only { + return Ok(()); + } + + export_data( + self.storage.as_ref(), + &self.database_client, + &self.config.snapshot_uri, + &self.config.storage_config, + &mut manifest, + self.config.parallelism, + ) + .await?; return Ok(()); } } // 2. Get schema list - let extractor = SchemaExtractor::new(&self.database_client, &self.catalog); - let schema_snapshot = extractor.extract(self.schemas.as_deref()).await?; + let extractor = SchemaExtractor::new(&self.database_client, &self.config.catalog); + let schema_snapshot = extractor.extract(self.config.schemas.as_deref()).await?; let schema_names: Vec = schema_snapshot .schemas @@ -245,7 +301,14 @@ impl ExportCreate { info!("Exporting schemas: {:?}", schema_names); // 3. Create manifest - let manifest = Manifest::new_schema_only(self.catalog.clone(), schema_names.clone()); + let mut manifest = Manifest::new_for_export( + self.config.catalog.clone(), + schema_names.clone(), + self.config.schema_only, + self.config.time_range.clone(), + self.config.format, + self.config.chunk_time_window, + )?; // 4. Write schema files self.storage.write_schema(&schema_snapshot).await?; @@ -259,14 +322,28 @@ impl ExportCreate { info!("Exported DDL for schema {} to {}", schema, ddl_path); } - // 6. Write manifest last. + // 6. Write manifest after schema artifacts and before any data export. // // The manifest is the snapshot commit point: only write it after the schema // index and all DDL files are durable, so a crash cannot leave a "valid" - // snapshot that is missing required schema artifacts. + // snapshot that is missing required schema artifacts. For full exports we + // still need the manifest before data copy starts, because chunk resume is + // tracked by updating this manifest in place. self.storage.write_manifest(&manifest).await?; info!("Snapshot created: {}", manifest.snapshot_id); + if !self.config.schema_only { + export_data( + self.storage.as_ref(), + &self.database_client, + &self.config.snapshot_uri, + &self.config.storage_config, + &mut manifest, + self.config.parallelism, + ) + .await?; + } + Ok(()) } @@ -321,7 +398,7 @@ impl ExportCreate { "SELECT table_name, table_type FROM information_schema.tables \ WHERE table_catalog = '{}' AND table_schema = '{}' \ AND (table_type = 'BASE TABLE' OR table_type = 'VIEW')", - escape_sql_literal(&self.catalog), + escape_sql_literal(&self.config.catalog), escape_sql_literal(schema) ); let records: Option>> = self @@ -359,7 +436,7 @@ impl ExportCreate { let sql = format!( "SELECT DISTINCT table_name FROM information_schema.columns \ WHERE table_catalog = '{}' AND table_schema = '{}' AND column_name = '__tsid'", - escape_sql_literal(&self.catalog), + escape_sql_literal(&self.config.catalog), escape_sql_literal(schema) ); let records: Option>> = self @@ -392,14 +469,14 @@ impl ExportCreate { Some(table) => format!( r#"SHOW CREATE {} "{}"."{}"."{}""#, show_type, - escape_sql_identifier(&self.catalog), + escape_sql_identifier(&self.config.catalog), escape_sql_identifier(schema), escape_sql_identifier(table) ), None => format!( r#"SHOW CREATE {} "{}"."{}""#, show_type, - escape_sql_identifier(&self.catalog), + escape_sql_identifier(&self.config.catalog), escape_sql_identifier(schema) ), }; @@ -442,8 +519,118 @@ fn build_schema_ddl( ddl } +fn validate_resume_config(manifest: &Manifest, config: &ExportConfig) -> Result<()> { + if manifest.schema_only != config.schema_only { + return SchemaOnlyModeMismatchSnafu { + existing_schema_only: manifest.schema_only, + requested_schema_only: config.schema_only, + } + .fail(); + } + + if manifest.catalog != config.catalog { + return ResumeConfigMismatchSnafu { + field: "catalog", + existing: manifest.catalog.clone(), + requested: config.catalog.clone(), + } + .fail(); + } + + // If no schema filter is provided on resume, inherit the existing snapshot + // selection instead of reinterpreting the request as "all schemas". + if let Some(requested_schemas) = &config.schemas + && !schema_selection_matches(&manifest.schemas, requested_schemas) + { + return ResumeConfigMismatchSnafu { + field: "schemas", + existing: format_schema_selection(&manifest.schemas), + requested: format_schema_selection(requested_schemas), + } + .fail(); + } + + if manifest.time_range != config.time_range { + return ResumeConfigMismatchSnafu { + field: "time_range", + existing: format!("{:?}", manifest.time_range), + requested: format!("{:?}", config.time_range), + } + .fail(); + } + + if manifest.format != config.format { + return ResumeConfigMismatchSnafu { + field: "format", + existing: manifest.format.to_string(), + requested: config.format.to_string(), + } + .fail(); + } + + let expected_plan = Manifest::new_for_export( + manifest.catalog.clone(), + manifest.schemas.clone(), + config.schema_only, + config.time_range.clone(), + config.format, + config.chunk_time_window, + )?; + if !chunk_plan_matches(manifest, &expected_plan) { + return ResumeConfigMismatchSnafu { + field: "chunk plan", + existing: format_chunk_plan(&manifest.chunks), + requested: format_chunk_plan(&expected_plan.chunks), + } + .fail(); + } + + Ok(()) +} + +fn schema_selection_matches(existing: &[String], requested: &[String]) -> bool { + canonical_schema_selection(existing) == canonical_schema_selection(requested) +} + +fn canonical_schema_selection(schemas: &[String]) -> Vec { + let mut canonicalized = Vec::new(); + let mut seen = HashSet::new(); + + for schema in schemas { + let normalized = schema.to_ascii_lowercase(); + if seen.insert(normalized.clone()) { + canonicalized.push(normalized); + } + } + + canonicalized.sort(); + canonicalized +} + +fn format_schema_selection(schemas: &[String]) -> String { + format!("[{}]", schemas.join(", ")) +} + +fn chunk_plan_matches(existing: &Manifest, expected: &Manifest) -> bool { + existing.chunks.len() == expected.chunks.len() + && existing + .chunks + .iter() + .zip(&expected.chunks) + .all(|(left, right)| left.id == right.id && left.time_range == right.time_range) +} + +fn format_chunk_plan(chunks: &[ChunkMeta]) -> String { + let items = chunks + .iter() + .map(|chunk| format!("#{}:{:?}", chunk.id, chunk.time_range)) + .collect::>(); + format!("[{}]", items.join(", ")) +} + #[cfg(test)] mod tests { + use chrono::TimeZone; use clap::Parser; use super::*; @@ -478,19 +665,225 @@ mod tests { } #[tokio::test] - async fn test_build_rejects_non_schema_only_export() { + async fn test_build_rejects_chunk_window_without_bounds() { let cmd = ExportCreateCommand::parse_from([ "export-v2-create", "--addr", "127.0.0.1:4000", "--to", "file:///tmp/export-v2-test", + "--chunk-time-window", + "1h", ]); let result = cmd.build().await; assert!(result.is_err()); let error = result.err().unwrap().to_string(); - assert!(error.contains("Data export is not implemented yet")); + assert!(error.contains("chunk_time_window requires both --start-time and --end-time")); + } + + #[tokio::test] + async fn test_build_rejects_data_export_args_in_schema_only_mode() { + let cmd = ExportCreateCommand::parse_from([ + "export-v2-create", + "--addr", + "127.0.0.1:4000", + "--to", + "file:///tmp/export-v2-test", + "--schema-only", + "--start-time", + "2024-01-01T00:00:00Z", + "--end-time", + "2024-01-02T00:00:00Z", + "--chunk-time-window", + "1h", + "--format", + "csv", + "--parallelism", + "2", + ]); + + let error = cmd.build().await.err().unwrap().to_string(); + + assert!(error.contains("--schema-only cannot be used with data export arguments")); + assert!(error.contains("--start-time")); + assert!(error.contains("--end-time")); + assert!(error.contains("--chunk-time-window")); + assert!(error.contains("--format")); + assert!(error.contains("--parallelism")); + } + + #[test] + fn test_schema_only_mode_mismatch_error_message() { + let error = crate::data::export_v2::error::SchemaOnlyModeMismatchSnafu { + existing_schema_only: false, + requested_schema_only: true, + } + .build() + .to_string(); + + assert!(error.contains("existing: false")); + assert!(error.contains("requested: true")); + } + + #[test] + fn test_validate_resume_config_rejects_catalog_mismatch() { + let manifest = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string()], + false, + TimeRange::unbounded(), + DataFormat::Parquet, + None, + ) + .unwrap(); + let config = ExportConfig { + catalog: "other".to_string(), + schemas: None, + schema_only: false, + format: DataFormat::Parquet, + force: false, + time_range: TimeRange::unbounded(), + chunk_time_window: None, + parallelism: 1, + snapshot_uri: "file:///tmp/snapshot".to_string(), + storage_config: ObjectStoreConfig::default(), + }; + + let error = validate_resume_config(&manifest, &config) + .err() + .unwrap() + .to_string(); + assert!(error.contains("catalog")); + } + + #[test] + fn test_validate_resume_config_accepts_schema_selection_with_different_case_and_order() { + let manifest = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string(), "analytics".to_string()], + false, + TimeRange::unbounded(), + DataFormat::Parquet, + None, + ) + .unwrap(); + let config = ExportConfig { + catalog: "greptime".to_string(), + schemas: Some(vec![ + "ANALYTICS".to_string(), + "PUBLIC".to_string(), + "public".to_string(), + ]), + schema_only: false, + format: DataFormat::Parquet, + force: false, + time_range: TimeRange::unbounded(), + chunk_time_window: None, + parallelism: 1, + snapshot_uri: "file:///tmp/snapshot".to_string(), + storage_config: ObjectStoreConfig::default(), + }; + + assert!(validate_resume_config(&manifest, &config).is_ok()); + } + + #[test] + fn test_validate_resume_config_rejects_chunk_plan_mismatch() { + let start = chrono::Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let end = chrono::Utc.with_ymd_and_hms(2025, 1, 1, 2, 0, 0).unwrap(); + let time_range = TimeRange::new(Some(start), Some(end)); + let manifest = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string()], + false, + time_range.clone(), + DataFormat::Parquet, + None, + ) + .unwrap(); + let config = ExportConfig { + catalog: "greptime".to_string(), + schemas: None, + schema_only: false, + format: DataFormat::Parquet, + force: false, + time_range, + chunk_time_window: Some(Duration::from_secs(3600)), + parallelism: 1, + snapshot_uri: "file:///tmp/snapshot".to_string(), + storage_config: ObjectStoreConfig::default(), + }; + + let error = validate_resume_config(&manifest, &config) + .err() + .unwrap() + .to_string(); + assert!(error.contains("chunk plan")); + } + + #[test] + fn test_validate_resume_config_rejects_format_mismatch() { + let manifest = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string()], + false, + TimeRange::unbounded(), + DataFormat::Parquet, + None, + ) + .unwrap(); + let config = ExportConfig { + catalog: "greptime".to_string(), + schemas: None, + schema_only: false, + format: DataFormat::Csv, + force: false, + time_range: TimeRange::unbounded(), + chunk_time_window: None, + parallelism: 1, + snapshot_uri: "file:///tmp/snapshot".to_string(), + storage_config: ObjectStoreConfig::default(), + }; + + let error = validate_resume_config(&manifest, &config) + .err() + .unwrap() + .to_string(); + assert!(error.contains("format")); + } + + #[test] + fn test_validate_resume_config_rejects_time_range_mismatch() { + let start = chrono::Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let end = chrono::Utc.with_ymd_and_hms(2025, 1, 1, 1, 0, 0).unwrap(); + let manifest = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string()], + false, + TimeRange::new(Some(start), Some(end)), + DataFormat::Parquet, + None, + ) + .unwrap(); + let config = ExportConfig { + catalog: "greptime".to_string(), + schemas: None, + schema_only: false, + format: DataFormat::Parquet, + force: false, + time_range: TimeRange::new(Some(start), Some(start)), + chunk_time_window: None, + parallelism: 1, + snapshot_uri: "file:///tmp/snapshot".to_string(), + storage_config: ObjectStoreConfig::default(), + }; + + let error = validate_resume_config(&manifest, &config) + .err() + .unwrap() + .to_string(); + assert!(error.contains("time_range")); } } diff --git a/src/cli/src/data/export_v2/coordinator.rs b/src/cli/src/data/export_v2/coordinator.rs new file mode 100644 index 0000000000..d96c01d693 --- /dev/null +++ b/src/cli/src/data/export_v2/coordinator.rs @@ -0,0 +1,166 @@ +// 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_telemetry::info; + +use crate::common::ObjectStoreConfig; +use crate::data::export_v2::data::{CopyOptions, build_copy_target, execute_copy_database}; +use crate::data::export_v2::error::Result; +use crate::data::export_v2::manifest::{ChunkStatus, DataFormat, Manifest, TimeRange}; +use crate::data::path::data_dir_for_schema_chunk; +use crate::data::snapshot_storage::{SnapshotStorage, StorageScheme}; +use crate::database::DatabaseClient; + +struct ExportContext<'a> { + storage: &'a dyn SnapshotStorage, + database_client: &'a DatabaseClient, + snapshot_uri: &'a str, + storage_config: &'a ObjectStoreConfig, + catalog: &'a str, + schemas: &'a [String], + format: DataFormat, + parallelism: usize, +} + +pub async fn export_data( + storage: &dyn SnapshotStorage, + database_client: &DatabaseClient, + snapshot_uri: &str, + storage_config: &ObjectStoreConfig, + manifest: &mut Manifest, + parallelism: usize, +) -> Result<()> { + if manifest.chunks.is_empty() { + return Ok(()); + } + + for idx in 0..manifest.chunks.len() { + if matches!( + manifest.chunks[idx].status, + ChunkStatus::Completed | ChunkStatus::Skipped + ) { + continue; + } + + let (chunk_id, time_range) = mark_chunk_in_progress(manifest, idx); + manifest.touch(); + storage.write_manifest(manifest).await?; + + let context = ExportContext { + storage, + database_client, + snapshot_uri, + storage_config, + catalog: &manifest.catalog, + schemas: &manifest.schemas, + format: manifest.format, + parallelism, + }; + let export_result = export_chunk(&context, chunk_id, time_range).await; + + let result = match export_result { + Ok(files) => { + mark_chunk_completed(manifest, idx, files); + Ok(()) + } + Err(err) => { + mark_chunk_failed(manifest, idx, err.to_string()); + Err(err) + } + }; + + manifest.touch(); + storage.write_manifest(manifest).await?; + + result?; + } + + Ok(()) +} + +fn mark_chunk_in_progress(manifest: &mut Manifest, idx: usize) -> (u32, TimeRange) { + let chunk = &mut manifest.chunks[idx]; + chunk.mark_in_progress(); + (chunk.id, chunk.time_range.clone()) +} + +fn mark_chunk_completed(manifest: &mut Manifest, idx: usize, files: Vec) { + let chunk = &mut manifest.chunks[idx]; + if files.is_empty() { + chunk.mark_skipped(); + } else { + chunk.mark_completed(files, None); + } +} + +fn mark_chunk_failed(manifest: &mut Manifest, idx: usize, error: String) { + let chunk = &mut manifest.chunks[idx]; + chunk.mark_failed(error); +} + +async fn export_chunk( + context: &ExportContext<'_>, + chunk_id: u32, + time_range: TimeRange, +) -> Result> { + let scheme = StorageScheme::from_uri(context.snapshot_uri)?; + let needs_dir = matches!(scheme, StorageScheme::File); + let copy_options = CopyOptions { + format: context.format, + time_range, + parallelism: context.parallelism, + }; + + for schema in context.schemas { + let prefix = data_dir_for_schema_chunk(schema, chunk_id); + if needs_dir { + context.storage.create_dir_all(&prefix).await?; + } + + let target = build_copy_target( + context.snapshot_uri, + context.storage_config, + schema, + chunk_id, + )?; + execute_copy_database( + context.database_client, + context.catalog, + schema, + &target, + ©_options, + ) + .await?; + } + + let files = list_chunk_files(context.storage, context.schemas, chunk_id).await?; + info!("Collected {} files for chunk {}", files.len(), chunk_id); + Ok(files) +} + +async fn list_chunk_files( + storage: &dyn SnapshotStorage, + schemas: &[String], + chunk_id: u32, +) -> Result> { + let mut files = Vec::new(); + + for schema in schemas { + let prefix = data_dir_for_schema_chunk(schema, chunk_id); + files.extend(storage.list_files_recursive(&prefix).await?); + } + + files.sort(); + Ok(files) +} diff --git a/src/cli/src/data/export_v2/data.rs b/src/cli/src/data/export_v2/data.rs new file mode 100644 index 0000000000..fe2ec7c051 --- /dev/null +++ b/src/cli/src/data/export_v2/data.rs @@ -0,0 +1,440 @@ +// 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_base::secrets::{ExposeSecret, SecretString}; +use common_telemetry::info; +use object_store::util::{join_path, normalize_path}; +use snafu::ResultExt; +use url::Url; + +use crate::common::ObjectStoreConfig; +use crate::data::export_v2::error::{DatabaseSnafu, InvalidUriSnafu, Result, UrlParseSnafu}; +use crate::data::export_v2::manifest::{DataFormat, TimeRange}; +use crate::data::path::data_dir_for_schema_chunk; +use crate::data::snapshot_storage::StorageScheme; +use crate::data::sql::{escape_sql_identifier, escape_sql_literal}; +use crate::database::DatabaseClient; + +pub(super) struct CopyOptions { + pub(super) format: DataFormat, + pub(super) time_range: TimeRange, + pub(super) parallelism: usize, +} + +pub(super) struct CopyTarget { + pub(super) location: String, + pub(super) connection: String, + secrets: Vec>, +} + +impl CopyTarget { + fn mask_sql(&self, sql: &str) -> String { + mask_secrets(sql, &self.secrets) + } +} + +pub(super) fn build_copy_target( + snapshot_uri: &str, + storage: &ObjectStoreConfig, + schema: &str, + chunk_id: u32, +) -> Result { + let url = Url::parse(snapshot_uri).context(UrlParseSnafu)?; + let scheme = StorageScheme::from_uri(snapshot_uri)?; + let suffix = data_dir_for_schema_chunk(schema, chunk_id); + + match scheme { + StorageScheme::File => { + let root = url.to_file_path().map_err(|_| { + InvalidUriSnafu { + uri: snapshot_uri, + reason: "file:// URI must use an absolute path like file:///tmp/backup", + } + .build() + })?; + let location = normalize_path(&format!("{}/{}", root.to_string_lossy(), suffix)); + Ok(CopyTarget { + location, + connection: String::new(), + secrets: Vec::new(), + }) + } + StorageScheme::S3 => { + let (bucket, root) = extract_bucket_root(&url, snapshot_uri)?; + let location = format!("s3://{}/{}", bucket, join_root(&root, &suffix)); + let (connection, secrets) = build_s3_connection(storage); + Ok(CopyTarget { + location, + connection, + secrets, + }) + } + StorageScheme::Oss => { + let (bucket, root) = extract_bucket_root(&url, snapshot_uri)?; + let location = format!("oss://{}/{}", bucket, join_root(&root, &suffix)); + let (connection, secrets) = build_oss_connection(storage); + Ok(CopyTarget { + location, + connection, + secrets, + }) + } + StorageScheme::Gcs => { + let (bucket, root) = extract_bucket_root(&url, snapshot_uri)?; + let location = format!("gcs://{}/{}", bucket, join_root(&root, &suffix)); + let (connection, secrets) = build_gcs_connection(storage, snapshot_uri)?; + Ok(CopyTarget { + location, + connection, + secrets, + }) + } + StorageScheme::Azblob => { + let (bucket, root) = extract_bucket_root(&url, snapshot_uri)?; + let location = format!("azblob://{}/{}", bucket, join_root(&root, &suffix)); + let (connection, secrets) = build_azblob_connection(storage); + Ok(CopyTarget { + location, + connection, + secrets, + }) + } + } +} + +pub(super) async fn execute_copy_database( + database_client: &DatabaseClient, + catalog: &str, + schema: &str, + target: &CopyTarget, + options: &CopyOptions, +) -> Result<()> { + let with_options = build_with_options(options); + let sql = format!( + r#"COPY DATABASE "{}"."{}" TO '{}' WITH ({}){};"#, + escape_sql_identifier(catalog), + escape_sql_identifier(schema), + escape_sql_literal(&target.location), + with_options, + target.connection + ); + let safe_sql = target.mask_sql(&sql); + info!("Executing sql: {}", safe_sql); + database_client + .sql_in_public(&sql) + .await + .context(DatabaseSnafu)?; + Ok(()) +} + +fn build_with_options(options: &CopyOptions) -> String { + let mut parts = vec![format!("FORMAT='{}'", options.format)]; + if let Some(start) = options.time_range.start { + parts.push(format!( + "START_TIME='{}'", + escape_sql_literal(&start.to_rfc3339()) + )); + } + if let Some(end) = options.time_range.end { + parts.push(format!( + "END_TIME='{}'", + escape_sql_literal(&end.to_rfc3339()) + )); + } + parts.push(format!("PARALLELISM={}", options.parallelism)); + parts.join(", ") +} + +fn extract_bucket_root(url: &Url, snapshot_uri: &str) -> Result<(String, String)> { + let bucket = url.host_str().unwrap_or("").to_string(); + if bucket.is_empty() { + return InvalidUriSnafu { + uri: snapshot_uri, + reason: "URI must include bucket/container in host", + } + .fail(); + } + let root = url + .path() + .trim_start_matches('/') + .trim_end_matches('/') + .to_string(); + Ok((bucket, root)) +} + +fn join_root(root: &str, suffix: &str) -> String { + join_path(root, suffix).trim_start_matches('/').to_string() +} + +fn build_s3_connection(storage: &ObjectStoreConfig) -> (String, Vec>) { + let access_key_id = expose_optional_secret(&storage.s3.s3_access_key_id); + let secret_access_key = expose_optional_secret(&storage.s3.s3_secret_access_key); + + let mut options = Vec::new(); + if let Some(access_key_id) = &access_key_id { + options.push(format!( + "ACCESS_KEY_ID='{}'", + escape_sql_literal(access_key_id) + )); + } + if let Some(secret_access_key) = &secret_access_key { + options.push(format!( + "SECRET_ACCESS_KEY='{}'", + escape_sql_literal(secret_access_key) + )); + } + if let Some(region) = &storage.s3.s3_region { + options.push(format!("REGION='{}'", escape_sql_literal(region))); + } + if let Some(endpoint) = &storage.s3.s3_endpoint { + options.push(format!("ENDPOINT='{}'", escape_sql_literal(endpoint))); + } + + let secrets = vec![access_key_id, secret_access_key]; + let connection = if options.is_empty() { + String::new() + } else { + format!(" CONNECTION ({})", options.join(", ")) + }; + (connection, secrets) +} + +fn build_oss_connection(storage: &ObjectStoreConfig) -> (String, Vec>) { + let access_key_id = expose_optional_secret(&storage.oss.oss_access_key_id); + let access_key_secret = expose_optional_secret(&storage.oss.oss_access_key_secret); + + let mut options = Vec::new(); + if let Some(access_key_id) = &access_key_id { + options.push(format!( + "ACCESS_KEY_ID='{}'", + escape_sql_literal(access_key_id) + )); + } + if let Some(access_key_secret) = &access_key_secret { + options.push(format!( + "ACCESS_KEY_SECRET='{}'", + escape_sql_literal(access_key_secret) + )); + } + if !storage.oss.oss_endpoint.is_empty() { + options.push(format!( + "ENDPOINT='{}'", + escape_sql_literal(&storage.oss.oss_endpoint) + )); + } + + let secrets = vec![access_key_id, access_key_secret]; + let connection = if options.is_empty() { + String::new() + } else { + format!(" CONNECTION ({})", options.join(", ")) + }; + (connection, secrets) +} + +fn build_gcs_connection( + storage: &ObjectStoreConfig, + snapshot_uri: &str, +) -> Result<(String, Vec>)> { + let credential_path = expose_optional_secret(&storage.gcs.gcs_credential_path); + let credential = expose_optional_secret(&storage.gcs.gcs_credential); + + if credential.is_none() && credential_path.is_some() { + return InvalidUriSnafu { + uri: snapshot_uri, + reason: "gcs_credential_path is not supported for server-side COPY; provide gcs_credential or rely on server-side ADC", + } + .fail(); + } + + let mut options = Vec::new(); + if let Some(credential) = &credential { + options.push(format!("CREDENTIAL='{}'", escape_sql_literal(credential))); + } + if !storage.gcs.gcs_scope.is_empty() { + options.push(format!( + "SCOPE='{}'", + escape_sql_literal(&storage.gcs.gcs_scope) + )); + } + if !storage.gcs.gcs_endpoint.is_empty() { + options.push(format!( + "ENDPOINT='{}'", + escape_sql_literal(&storage.gcs.gcs_endpoint) + )); + } + + let connection = if options.is_empty() { + String::new() + } else { + format!(" CONNECTION ({})", options.join(", ")) + }; + let secrets = vec![credential_path, credential]; + Ok((connection, secrets)) +} + +fn build_azblob_connection(storage: &ObjectStoreConfig) -> (String, Vec>) { + let account_name = expose_optional_secret(&storage.azblob.azblob_account_name); + let account_key = expose_optional_secret(&storage.azblob.azblob_account_key); + let sas_token = storage.azblob.azblob_sas_token.clone(); + + let mut options = Vec::new(); + if let Some(account_name) = &account_name { + options.push(format!( + "ACCOUNT_NAME='{}'", + escape_sql_literal(account_name) + )); + } + if let Some(account_key) = &account_key { + options.push(format!("ACCOUNT_KEY='{}'", escape_sql_literal(account_key))); + } + if let Some(sas_token) = &sas_token { + options.push(format!("SAS_TOKEN='{}'", escape_sql_literal(sas_token))); + } + if !storage.azblob.azblob_endpoint.is_empty() { + options.push(format!( + "ENDPOINT='{}'", + escape_sql_literal(&storage.azblob.azblob_endpoint) + )); + } + + let secrets = vec![account_name, account_key, sas_token]; + let connection = if options.is_empty() { + String::new() + } else { + format!(" CONNECTION ({})", options.join(", ")) + }; + (connection, secrets) +} + +fn expose_optional_secret(secret: &Option) -> Option { + secret.as_ref().map(|s| s.expose_secret().to_owned()) +} + +fn mask_secrets(sql: &str, secrets: &[Option]) -> String { + let mut masked = sql.to_string(); + for secret in secrets { + if let Some(secret) = secret + && !secret.is_empty() + { + masked = masked.replace(secret, "[REDACTED]"); + } + } + masked +} + +#[cfg(test)] +mod tests { + use common_base::secrets::SecretString; + + use super::*; + use crate::common::{PrefixedAzblobConnection, PrefixedGcsConnection, PrefixedOssConnection}; + + #[test] + fn test_build_oss_connection_includes_endpoint() { + let storage = ObjectStoreConfig { + oss: PrefixedOssConnection { + oss_endpoint: "https://oss.example.com".to_string(), + oss_access_key_id: Some(SecretString::from("key_id".to_string())), + oss_access_key_secret: Some(SecretString::from("key_secret".to_string())), + ..Default::default() + }, + ..Default::default() + }; + + let (connection, _) = build_oss_connection(&storage); + assert!(connection.contains("ENDPOINT='https://oss.example.com'")); + } + + #[test] + fn test_build_gcs_connection_uses_scope_and_inline_credential() { + let storage = ObjectStoreConfig { + gcs: PrefixedGcsConnection { + gcs_scope: "scope-a".to_string(), + gcs_endpoint: "https://storage.googleapis.com".to_string(), + gcs_credential: Some(SecretString::from("credential-json".to_string())), + ..Default::default() + }, + ..Default::default() + }; + + let (connection, _) = build_gcs_connection(&storage, "gcs://bucket/root").unwrap(); + assert!(connection.contains("CREDENTIAL='credential-json'")); + assert!(connection.contains("SCOPE='scope-a'")); + assert!(connection.contains("ENDPOINT='https://storage.googleapis.com'")); + assert!(!connection.contains("CREDENTIAL_PATH")); + } + + #[test] + fn test_build_gcs_connection_rejects_credential_path_only() { + let storage = ObjectStoreConfig { + gcs: PrefixedGcsConnection { + gcs_scope: "scope-a".to_string(), + gcs_credential_path: Some(SecretString::from("/tmp/creds.json".to_string())), + ..Default::default() + }, + ..Default::default() + }; + + let error = build_gcs_connection(&storage, "gcs://bucket/root") + .expect_err("credential_path-only should be rejected") + .to_string(); + assert!(error.contains("gcs_credential_path is not supported")); + } + + #[test] + fn test_build_azblob_connection_includes_endpoint() { + let storage = ObjectStoreConfig { + azblob: PrefixedAzblobConnection { + azblob_account_name: Some(SecretString::from("account".to_string())), + azblob_account_key: Some(SecretString::from("key".to_string())), + azblob_endpoint: "https://blob.example.com".to_string(), + ..Default::default() + }, + ..Default::default() + }; + + let (connection, _) = build_azblob_connection(&storage); + assert!(connection.contains("ENDPOINT='https://blob.example.com'")); + } + + #[test] + fn test_build_azblob_connection_redacts_sas_token() { + let storage = ObjectStoreConfig { + azblob: PrefixedAzblobConnection { + azblob_account_name: Some(SecretString::from("account".to_string())), + azblob_account_key: Some(SecretString::from("key".to_string())), + azblob_sas_token: Some("sig=secret-token".to_string()), + ..Default::default() + }, + ..Default::default() + }; + + let (connection, secrets) = build_azblob_connection(&storage); + let masked = mask_secrets(&connection, &secrets); + + assert!(connection.contains("SAS_TOKEN='sig=secret-token'")); + assert!(masked.contains("SAS_TOKEN='[REDACTED]'")); + assert!(!masked.contains("sig=secret-token")); + } + + #[test] + fn test_build_copy_target_decodes_file_uri_path() { + let storage = ObjectStoreConfig::default(); + let target = build_copy_target("file:///tmp/my%20backup", &storage, "public", 7) + .expect("file:// copy target should be built"); + + assert_eq!(target.location, "/tmp/my backup/data/public/7/"); + } +} diff --git a/src/cli/src/data/export_v2/error.rs b/src/cli/src/data/export_v2/error.rs index 2db71d5326..ec860fecfa 100644 --- a/src/cli/src/data/export_v2/error.rs +++ b/src/cli/src/data/export_v2/error.rs @@ -72,17 +72,55 @@ pub enum Error { }, #[snafu(display( - "Cannot resume schema-only snapshot with data export. Use --force to recreate." + "Cannot resume snapshot with a different schema_only mode (existing: {}, requested: {}). Use --force to recreate.", + existing_schema_only, + requested_schema_only ))] - CannotResumeSchemaOnly { + SchemaOnlyModeMismatch { + existing_schema_only: bool, + requested_schema_only: bool, #[snafu(implicit)] location: Location, }, #[snafu(display( - "Data export is not implemented yet. Use --schema-only to create a schema snapshot." + "Cannot resume snapshot with different {} (existing: {}, requested: {}). Use --force to recreate.", + field, + existing, + requested ))] - DataExportNotImplemented { + ResumeConfigMismatch { + field: String, + existing: String, + requested: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to parse time: invalid format: {}", input))] + TimeParseInvalidFormat { + input: String, + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Failed to parse time: end_time is before start_time"))] + TimeParseEndBeforeStart { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display( + "chunk_time_window requires both --start-time and --end-time to be specified" + ))] + ChunkTimeWindowRequiresBounds { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("--schema-only cannot be used with data export arguments: {}", args))] + SchemaOnlyArgsNotAllowed { + args: String, #[snafu(implicit)] location: Location, }, @@ -154,9 +192,13 @@ impl ErrorExt for Error { match self { Error::InvalidUri { .. } | Error::UnsupportedScheme { .. } - | Error::CannotResumeSchemaOnly { .. } - | Error::DataExportNotImplemented { .. } - | Error::ManifestVersionMismatch { .. } => StatusCode::InvalidArguments, + | Error::SchemaOnlyModeMismatch { .. } + | Error::ResumeConfigMismatch { .. } + | Error::ManifestVersionMismatch { .. } + | Error::SchemaOnlyArgsNotAllowed { .. } => StatusCode::InvalidArguments, + Error::TimeParseInvalidFormat { .. } + | Error::TimeParseEndBeforeStart { .. } + | Error::ChunkTimeWindowRequiresBounds { .. } => StatusCode::InvalidArguments, Error::StorageOperation { .. } | Error::ManifestParse { .. } diff --git a/src/cli/src/data/export_v2/manifest.rs b/src/cli/src/data/export_v2/manifest.rs index 0ebf753fa4..918288bb51 100644 --- a/src/cli/src/data/export_v2/manifest.rs +++ b/src/cli/src/data/export_v2/manifest.rs @@ -14,12 +14,19 @@ //! Manifest data structures for Export/Import V2. +use std::time::Duration; use std::{fmt, str}; use chrono::{DateTime, Utc}; use serde::{Deserialize, Serialize}; use uuid::Uuid; +use crate::data::export_v2::chunker::generate_chunks; +use crate::data::export_v2::error::{ + ChunkTimeWindowRequiresBoundsSnafu, Result as ExportResult, TimeParseEndBeforeStartSnafu, + TimeParseInvalidFormatSnafu, +}; + /// Current manifest format version. pub const MANIFEST_VERSION: u32 = 1; @@ -55,6 +62,31 @@ impl TimeRange { pub fn is_unbounded(&self) -> bool { self.start.is_none() && self.end.is_none() } + + /// Returns true if both bounds are specified. + pub fn is_bounded(&self) -> bool { + self.start.is_some() && self.end.is_some() + } + + /// Parses a time range from optional RFC3339 strings. + pub fn parse(start: Option<&str>, end: Option<&str>) -> ExportResult { + let start = start.map(parse_time).transpose()?; + let end = end.map(parse_time).transpose()?; + + if let (Some(start), Some(end)) = (start, end) + && end < start + { + return TimeParseEndBeforeStartSnafu.fail(); + } + + Ok(Self::new(start, end)) + } +} + +fn parse_time(input: &str) -> ExportResult> { + DateTime::parse_from_rfc3339(input) + .map(|dt| dt.with_timezone(&Utc)) + .map_err(|_| TimeParseInvalidFormatSnafu { input }.build()) } impl Default for TimeRange { @@ -74,6 +106,8 @@ pub enum ChunkStatus { InProgress, /// Chunk export completed successfully. Completed, + /// Chunk had no data to export. + Skipped, /// Chunk export failed. Failed, } @@ -111,6 +145,13 @@ impl ChunkMeta { } } + /// Creates a skipped chunk with the given id and time range. + pub fn skipped(id: u32, time_range: TimeRange) -> Self { + let mut chunk = Self::new(id, time_range); + chunk.mark_skipped(); + chunk + } + /// Marks this chunk as in progress. pub fn mark_in_progress(&mut self) { self.status = ChunkStatus::InProgress; @@ -125,6 +166,14 @@ impl ChunkMeta { self.error = None; } + /// Marks this chunk as skipped because no data files were produced. + pub fn mark_skipped(&mut self) { + self.status = ChunkStatus::Skipped; + self.files.clear(); + self.checksum = None; + self.error = None; + } + /// Marks this chunk as failed with the given error message. pub fn mark_failed(&mut self, error: String) { self.status = ChunkStatus::Failed; @@ -210,6 +259,35 @@ pub struct Manifest { } impl Manifest { + pub fn new_for_export( + catalog: String, + schemas: Vec, + schema_only: bool, + time_range: TimeRange, + format: DataFormat, + chunk_time_window: Option, + ) -> ExportResult { + if chunk_time_window.is_some() && !time_range.is_bounded() { + return ChunkTimeWindowRequiresBoundsSnafu.fail(); + } + + let mut manifest = if schema_only { + Self::new_schema_only(catalog, schemas) + } else { + Self::new_full(catalog, schemas, time_range, format) + }; + + if !schema_only { + manifest.chunks = match chunk_time_window { + Some(window) => generate_chunks(&manifest.time_range, window), + None => generate_single_chunk(&manifest.time_range), + }; + manifest.touch(); + } + + Ok(manifest) + } + /// Creates a new manifest for schema-only export. pub fn new_schema_only(catalog: String, schemas: Vec) -> Self { let now = Utc::now(); @@ -258,7 +336,7 @@ impl Manifest { && self .chunks .iter() - .all(|c| c.status == ChunkStatus::Completed)) + .all(|c| matches!(c.status, ChunkStatus::Completed | ChunkStatus::Skipped))) } /// Returns the number of pending chunks. @@ -285,6 +363,14 @@ impl Manifest { .count() } + /// Returns the number of skipped chunks. + pub fn skipped_count(&self) -> usize { + self.chunks + .iter() + .filter(|c| c.status == ChunkStatus::Skipped) + .count() + } + /// Returns the number of failed chunks. pub fn failed_count(&self) -> usize { self.chunks @@ -313,8 +399,24 @@ impl Manifest { } } +fn generate_single_chunk(time_range: &TimeRange) -> Vec { + if let (Some(start), Some(end)) = (time_range.start, time_range.end) { + if start == end { + return vec![ChunkMeta::skipped(1, time_range.clone())]; + } + if start > end { + return Vec::new(); + } + } + vec![ChunkMeta::new(1, time_range.clone())] +} + #[cfg(test)] mod tests { + use std::time::Duration; + + use chrono::{TimeZone, Utc}; + use super::*; #[test] @@ -338,6 +440,26 @@ mod tests { assert!(manifest.is_complete()); } + #[test] + fn test_generate_single_chunk_zero_width_range_is_skipped() { + let ts = Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let chunks = generate_single_chunk(&TimeRange::new(Some(ts), Some(ts))); + + assert_eq!(chunks.len(), 1); + assert_eq!(chunks[0].status, ChunkStatus::Skipped); + assert_eq!(chunks[0].time_range.start, Some(ts)); + assert_eq!(chunks[0].time_range.end, Some(ts)); + } + + #[test] + fn test_generate_single_chunk_invalid_range_is_empty() { + let start = Utc.with_ymd_and_hms(2025, 1, 1, 1, 0, 0).unwrap(); + let end = Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let chunks = generate_single_chunk(&TimeRange::new(Some(start), Some(end))); + + assert!(chunks.is_empty()); + } + #[test] fn test_manifest_full() { let manifest = Manifest::new_full( @@ -377,5 +499,71 @@ mod tests { ); assert_eq!(chunk.status, ChunkStatus::Completed); assert_eq!(chunk.files.len(), 1); + + chunk.mark_skipped(); + assert_eq!(chunk.status, ChunkStatus::Skipped); + assert!(chunk.files.is_empty()); + } + + #[test] + fn test_manifest_is_complete_when_chunks_are_completed_or_skipped() { + let mut manifest = Manifest::new_full( + "greptime".to_string(), + vec!["public".to_string()], + TimeRange::unbounded(), + DataFormat::Parquet, + ); + manifest.add_chunk(ChunkMeta::new(1, TimeRange::unbounded())); + manifest.add_chunk(ChunkMeta::new(2, TimeRange::unbounded())); + + manifest.update_chunk(1, |chunk| { + chunk.mark_completed(vec!["a.parquet".to_string()], None) + }); + manifest.update_chunk(2, |chunk| chunk.mark_skipped()); + + assert!(manifest.is_complete()); + assert_eq!(manifest.completed_count(), 1); + assert_eq!(manifest.skipped_count(), 1); + } + + #[test] + fn test_manifest_chunk_time_window_none_single_chunk() { + let start = Utc.with_ymd_and_hms(2025, 1, 1, 0, 0, 0).unwrap(); + let end = Utc.with_ymd_and_hms(2025, 1, 2, 0, 0, 0).unwrap(); + let range = TimeRange::new(Some(start), Some(end)); + let manifest = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string()], + false, + range.clone(), + DataFormat::Parquet, + None, + ) + .unwrap(); + + assert_eq!(manifest.chunks.len(), 1); + assert_eq!(manifest.chunks[0].time_range, range); + } + + #[test] + fn test_time_range_parse_requires_order() { + let result = TimeRange::parse(Some("2025-01-02T00:00:00Z"), Some("2025-01-01T00:00:00Z")); + assert!(result.is_err()); + } + + #[test] + fn test_new_for_export_with_chunk_window_requires_bounded_range() { + let result = Manifest::new_for_export( + "greptime".to_string(), + vec!["public".to_string()], + false, + TimeRange::new( + None, + Some(Utc.with_ymd_and_hms(2025, 1, 2, 0, 0, 0).unwrap()), + ), + DataFormat::Parquet, + Some(Duration::from_secs(3600)), + ); + assert!(result.is_err()); } } diff --git a/src/cli/src/data/import_v2/command.rs b/src/cli/src/data/import_v2/command.rs index 544763d92b..6a9d440071 100644 --- a/src/cli/src/data/import_v2/command.rs +++ b/src/cli/src/data/import_v2/command.rs @@ -27,7 +27,8 @@ use crate::Tool; use crate::common::ObjectStoreConfig; use crate::data::export_v2::manifest::MANIFEST_VERSION; use crate::data::import_v2::error::{ - ManifestVersionMismatchSnafu, Result, SchemaNotInSnapshotSnafu, SnapshotStorageSnafu, + FullSnapshotImportNotSupportedSnafu, ManifestVersionMismatchSnafu, Result, + SchemaNotInSnapshotSnafu, SnapshotStorageSnafu, }; use crate::data::import_v2::executor::{DdlExecutor, DdlStatement}; use crate::data::path::ddl_path_for_schema; @@ -58,10 +59,6 @@ pub struct ImportV2Command { #[clap(long)] dry_run: bool, - /// Concurrency level (for future use). - #[clap(long, default_value = "1")] - parallelism: usize, - /// Basic authentication (user:password). #[clap(long)] auth_basic: Option, @@ -121,7 +118,6 @@ impl ImportV2Command { Ok(Box::new(Import { schemas, dry_run: self.dry_run, - _parallelism: self.parallelism, storage: Box::new(storage), database_client, })) @@ -132,7 +128,6 @@ impl ImportV2Command { pub struct Import { schemas: Option>, dry_run: bool, - _parallelism: usize, storage: Box, database_client: DatabaseClient, } @@ -169,6 +164,13 @@ impl Import { info!("Snapshot contains {} schema(s)", manifest.schemas.len()); + if !manifest.schema_only && !manifest.chunks.is_empty() { + return FullSnapshotImportNotSupportedSnafu { + chunk_count: manifest.chunks.len(), + } + .fail(); + } + // 2. Determine schemas to import let schemas_to_import = match &self.schemas { Some(filter) => canonicalize_schema_filter(filter, &manifest.schemas)?, @@ -203,14 +205,6 @@ impl Import { ddl_statements.len() ); - // 6. Data import would happen here for non-schema-only snapshots (M2/M3) - if !manifest.schema_only && !manifest.chunks.is_empty() { - info!( - "Data import not yet implemented (M3). {} chunks pending.", - manifest.chunks.len() - ); - } - Ok(()) } @@ -403,7 +397,114 @@ fn canonicalize_schema_filter( #[cfg(test)] mod tests { + use std::time::Duration; + + use async_trait::async_trait; + use super::*; + use crate::Tool; + use crate::data::export_v2::manifest::{ChunkMeta, DataFormat, Manifest, TimeRange}; + use crate::data::export_v2::schema::SchemaSnapshot; + use crate::data::snapshot_storage::SnapshotStorage; + use crate::database::DatabaseClient; + + struct StubStorage { + manifest: Manifest, + } + + #[async_trait] + impl SnapshotStorage for StubStorage { + async fn exists(&self) -> crate::data::export_v2::error::Result { + Ok(true) + } + + async fn read_manifest(&self) -> crate::data::export_v2::error::Result { + Ok(self.manifest.clone()) + } + + async fn write_manifest( + &self, + _manifest: &Manifest, + ) -> crate::data::export_v2::error::Result<()> { + unimplemented!("not needed in import_v2::command tests") + } + + async fn read_text(&self, _path: &str) -> crate::data::export_v2::error::Result { + unimplemented!("not needed in import_v2::command tests") + } + + async fn write_text( + &self, + _path: &str, + _content: &str, + ) -> crate::data::export_v2::error::Result<()> { + unimplemented!("not needed in import_v2::command tests") + } + + async fn write_schema( + &self, + _snapshot: &SchemaSnapshot, + ) -> crate::data::export_v2::error::Result<()> { + unimplemented!("not needed in import_v2::command tests") + } + + async fn create_dir_all(&self, _path: &str) -> crate::data::export_v2::error::Result<()> { + unimplemented!("not needed in import_v2::command tests") + } + + async fn list_files_recursive( + &self, + _prefix: &str, + ) -> crate::data::export_v2::error::Result> { + unimplemented!("not needed in import_v2::command tests") + } + + async fn delete_snapshot(&self) -> crate::data::export_v2::error::Result<()> { + unimplemented!("not needed in import_v2::command tests") + } + } + + fn test_database_client() -> DatabaseClient { + DatabaseClient::new( + "127.0.0.1:4000".to_string(), + "greptime".to_string(), + None, + Duration::from_secs(1), + None, + false, + ) + } + + #[tokio::test] + async fn test_import_rejects_full_snapshot_before_schema_execution() { + let mut manifest = Manifest::new_full( + "greptime".to_string(), + vec!["public".to_string()], + TimeRange::unbounded(), + DataFormat::Parquet, + ); + manifest + .chunks + .push(ChunkMeta::new(1, TimeRange::unbounded())); + + let import = Import { + schemas: None, + dry_run: false, + storage: Box::new(StubStorage { manifest }), + database_client: test_database_client(), + }; + + let error = import + .do_work() + .await + .expect_err("full snapshot import should fail"); + + assert!( + error + .to_string() + .contains("Importing data from full snapshots is not implemented yet") + ); + } #[test] fn test_parse_ddl_statements() { diff --git a/src/cli/src/data/import_v2/error.rs b/src/cli/src/data/import_v2/error.rs index 5ae3db1583..169f11c0fa 100644 --- a/src/cli/src/data/import_v2/error.rs +++ b/src/cli/src/data/import_v2/error.rs @@ -45,6 +45,16 @@ pub enum Error { location: Location, }, + #[snafu(display( + "Importing data from full snapshots is not implemented yet (snapshot has {} chunk(s))", + chunk_count + ))] + FullSnapshotImportNotSupported { + chunk_count: usize, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Snapshot storage error"))] SnapshotStorage { #[snafu(source)] @@ -67,10 +77,10 @@ pub type Result = std::result::Result; impl ErrorExt for Error { fn status_code(&self) -> StatusCode { match self { - Error::SnapshotNotFound { .. } | Error::SchemaNotInSnapshot { .. } => { - StatusCode::InvalidArguments - } - Error::ManifestVersionMismatch { .. } => StatusCode::InvalidArguments, + Error::SnapshotNotFound { .. } + | Error::SchemaNotInSnapshot { .. } + | Error::ManifestVersionMismatch { .. } + | Error::FullSnapshotImportNotSupported { .. } => StatusCode::InvalidArguments, Error::Database { error, .. } => error.status_code(), Error::SnapshotStorage { error, .. } => error.status_code(), } diff --git a/src/cli/src/data/path.rs b/src/cli/src/data/path.rs index 2e0f5d3f1a..2df81f62c8 100644 --- a/src/cli/src/data/path.rs +++ b/src/cli/src/data/path.rs @@ -25,6 +25,10 @@ pub(crate) fn ddl_path_for_schema(schema: &str) -> String { ) } +pub(crate) fn data_dir_for_schema_chunk(schema: &str, chunk_id: u32) -> String { + format!("data/{}/{}/", encode_path_segment(schema), chunk_id) +} + pub(crate) fn encode_path_segment(value: &str) -> String { let mut encoded = String::with_capacity(value.len()); for byte in value.bytes() { @@ -73,4 +77,13 @@ mod tests { "schema/ddl/%2E%2E%2Fevil.sql" ); } + + #[test] + fn test_data_dir_for_schema_chunk_encodes_schema_segment() { + assert_eq!(data_dir_for_schema_chunk("public", 1), "data/public/1/"); + assert_eq!( + data_dir_for_schema_chunk("../evil", 7), + "data/%2E%2E%2Fevil/7/" + ); + } } diff --git a/src/cli/src/data/snapshot_storage.rs b/src/cli/src/data/snapshot_storage.rs index 50c8734a67..6bc71153df 100644 --- a/src/cli/src/data/snapshot_storage.rs +++ b/src/cli/src/data/snapshot_storage.rs @@ -18,9 +18,12 @@ //! to various storage backends (S3, OSS, GCS, Azure Blob, local filesystem). use async_trait::async_trait; +use futures::TryStreamExt; use object_store::services::{Azblob, Fs, Gcs, Oss, S3}; use object_store::util::{with_instrument_layers, with_retry_layers}; -use object_store::{AzblobConnection, GcsConnection, ObjectStore, OssConnection, S3Connection}; +use object_store::{ + AzblobConnection, ErrorKind, GcsConnection, ObjectStore, OssConnection, S3Connection, +}; use snafu::ResultExt; use url::Url; @@ -139,14 +142,14 @@ fn extract_file_path_from_uri(uri: &str) -> Result { .fail(), _ => url .to_file_path() - .map(|path| path.to_string_lossy().into_owned()) .map_err(|_| { InvalidUriSnafu { uri, - reason: "file:// URI must use a valid absolute filesystem path", + reason: "file:// URI must use an absolute path like file:///tmp/backup", } .build() - }), + }) + .map(|path| path.to_string_lossy().into_owned()), } } @@ -184,6 +187,12 @@ pub trait SnapshotStorage: Send + Sync { /// Reads a text file from a relative path under the snapshot root. async fn read_text(&self, path: &str) -> Result; + /// Creates a directory-like prefix under the snapshot root when needed by the backend. + async fn create_dir_all(&self, path: &str) -> Result<()>; + + /// Lists files recursively under a relative prefix. + async fn list_files_recursive(&self, prefix: &str) -> Result>; + /// Deletes the entire snapshot (for --force). async fn delete_snapshot(&self) -> Result<()>; } @@ -443,6 +452,38 @@ impl SnapshotStorage for OpenDalStorage { String::from_utf8(data).context(TextDecodeSnafu) } + async fn create_dir_all(&self, path: &str) -> Result<()> { + self.object_store + .create_dir(path) + .await + .context(StorageOperationSnafu { + operation: format!("create dir {}", path), + }) + } + + async fn list_files_recursive(&self, prefix: &str) -> Result> { + let mut lister = match self.object_store.lister_with(prefix).recursive(true).await { + Ok(lister) => lister, + Err(error) if error.kind() == ErrorKind::NotFound => return Ok(Vec::new()), + Err(error) => { + return Err(error).context(StorageOperationSnafu { + operation: format!("list {}", prefix), + }); + } + }; + + let mut files = Vec::new(); + while let Some(entry) = lister.try_next().await.context(StorageOperationSnafu { + operation: format!("list {}", prefix), + })? { + if entry.metadata().is_dir() { + continue; + } + files.push(entry.path().to_string()); + } + Ok(files) + } + async fn delete_snapshot(&self) -> Result<()> { self.object_store .remove_all("/") @@ -533,6 +574,14 @@ mod tests { extract_file_path_from_uri("file://localhost/tmp/backup").unwrap(), "/tmp/backup" ); + assert_eq!( + extract_file_path_from_uri("file:///tmp/my%20backup").unwrap(), + "/tmp/my backup" + ); + assert_eq!( + extract_file_path_from_uri("file://localhost/tmp/my%20backup").unwrap(), + "/tmp/my backup" + ); } #[test] From 58ac889818a17f2d7350088582c973ba1561fd2b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Apr 2026 09:44:13 +0800 Subject: [PATCH 14/20] chore(deps): bump rustls-webpki from 0.103.3 to 0.103.10 (#7891) Bumps [rustls-webpki](https://github.com/rustls/webpki) from 0.103.3 to 0.103.10. - [Release notes](https://github.com/rustls/webpki/releases) - [Commits](https://github.com/rustls/webpki/compare/v/0.103.3...v/0.103.10) --- updated-dependencies: - dependency-name: rustls-webpki dependency-version: 0.103.10 dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cfec1c5f54..525ac56f92 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7295,7 +7295,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" dependencies = [ "cfg-if", - "windows-targets 0.52.6", + "windows-targets 0.48.5", ] [[package]] @@ -11595,9 +11595,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.3" +version = "0.103.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4a72fe2bcf7a6ac6fd7d0b9e5cb68aeb7d4c0a0271730218b3e92d43b4eb435" +checksum = "df33b2b81ac578cabaf06b89b0631153a3f416b0a886e8a7a1707fb51abbd1ef" dependencies = [ "ring", "rustls-pki-types", From 03b2f94821e54b6f03f4b78cf249ed601dc12fea Mon Sep 17 00:00:00 2001 From: liyang Date: Wed, 1 Apr 2026 09:59:37 +0800 Subject: [PATCH 15/20] chore: Update Dockerfile (#7893) * chore: Update Dockerfile * Update update-dev-builder-version.sh --- .github/scripts/update-dev-builder-version.sh | 7 +++++-- docker/dev-builder/centos/Dockerfile | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/.github/scripts/update-dev-builder-version.sh b/.github/scripts/update-dev-builder-version.sh index 38466760a4..a593f385fd 100755 --- a/.github/scripts/update-dev-builder-version.sh +++ b/.github/scripts/update-dev-builder-version.sh @@ -30,8 +30,11 @@ update_dev_builder_version() { --body "This PR updates the dev-builder image tag" \ --base main \ --head $BRANCH_NAME \ - --reviewer zyy17 \ - --reviewer daviderli614 + --reviewer sunng87 \ + --reviewer daviderli614 \ + --reviewer killme2008 \ + --reviewer evenyag \ + --reviewer fengjiachun } update_dev_builder_version diff --git a/docker/dev-builder/centos/Dockerfile b/docker/dev-builder/centos/Dockerfile index 8b29bb3065..344bc1ec2a 100644 --- a/docker/dev-builder/centos/Dockerfile +++ b/docker/dev-builder/centos/Dockerfile @@ -7,7 +7,7 @@ RUN sed -i s/mirror.centos.org/vault.centos.org/g /etc/yum.repos.d/*.repo RUN sed -i s/^#.*baseurl=http/baseurl=http/g /etc/yum.repos.d/*.repo # Install dependencies -RUN ulimit -n 1024000 && yum groupinstall -y 'Development Tools' +RUN yum groupinstall -y 'Development Tools' RUN yum install -y epel-release \ openssl \ openssl-devel \ From 0bd0df0e88d6f6da78bd25d195ccf4640f2b6d1e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 1 Apr 2026 02:20:05 +0000 Subject: [PATCH 16/20] chore(deps): bump tar from 0.4.44 to 0.4.45 (#7890) Bumps [tar](https://github.com/alexcrichton/tar-rs) from 0.4.44 to 0.4.45. - [Commits](https://github.com/alexcrichton/tar-rs/compare/0.4.44...0.4.45) --- updated-dependencies: - dependency-name: tar dependency-version: 0.4.45 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 525ac56f92..401ac3b1ca 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6277,7 +6277,7 @@ dependencies = [ "js-sys", "log", "wasm-bindgen", - "windows-core 0.57.0", + "windows-core 0.61.2", ] [[package]] @@ -13365,9 +13365,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "tar" -version = "0.4.44" +version = "0.4.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d863878d212c87a19c1a610eb53bb01fe12951c0501cf5a0d65f724914a667a" +checksum = "22692a6476a21fa75fdfc11d452fda482af402c008cdbaf3476414e122040973" dependencies = [ "filetime", "libc", From b4492ee39d8fdaa824a2f396fbc1508afd484042 Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Wed, 1 Apr 2026 10:43:25 +0800 Subject: [PATCH 17/20] ci: update dev-builder image tag (#7894) * chore: Update Dockerfile * ci: update dev-builder image tag Signed-off-by: greptimedb-ci --------- Signed-off-by: greptimedb-ci Co-authored-by: liyang Co-authored-by: greptimedb-ci Co-authored-by: Ning Sun --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 60ea01a3ce..3fd09ad4ea 100644 --- a/Makefile +++ b/Makefile @@ -8,7 +8,7 @@ CARGO_BUILD_OPTS := --locked IMAGE_REGISTRY ?= docker.io IMAGE_NAMESPACE ?= greptime IMAGE_TAG ?= latest -DEV_BUILDER_IMAGE_TAG ?= 2025-10-01-8fe17d43-20251011080129 +DEV_BUILDER_IMAGE_TAG ?= 2026-03-21-9c9d9e9e-20260331090344 BUILDX_MULTI_PLATFORM_BUILD ?= false BUILDX_BUILDER_NAME ?= gtbuilder BASE_IMAGE ?= ubuntu From 2b4e12c358818ef0829cc524aa56bbe38cc37980 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Wed, 1 Apr 2026 10:45:26 +0800 Subject: [PATCH 18/20] feat: auto-align Prometheus schemas in pending rows batching (#7877) * feat/auto-schema-align: - **Error Handling Improvements**: - Removed `CatalogSnafu` context from various `.await` calls in `dashboard.rs`, `influxdb.rs`, `jaeger.rs`, `prometheus.rs`, `event.rs`, and `pipeline.rs` to streamline error handling. - **Prometheus Store Enhancements**: - Added support for auto-creating tables and adding missing Prometheus tag columns in `prom_store.rs` and `pending_rows_batcher.rs`. - Introduced `PendingRowsSchemaAlterer` trait for schema alterations in `pending_rows_batcher.rs`. - **Test Additions**: - Added tests for new Prometheus store functionalities in `prom_store.rs` and `pending_rows_batcher.rs`. - **Error Message Improvements**: - Enhanced error messages for catalog access in `error.rs`. - **Server Configuration Updates**: - Updated server configuration to include Prometheus store options in `server.rs`. Signed-off-by: Lei, HUANG * reformat Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Add DataTypes Error Handling and Column Renaming Logic - **`error.rs`**: Introduced a new `DataTypes` error variant to handle errors from `datatypes::error::Error`. Updated `ErrorExt` implementation to include `DataTypes`. - **`pending_rows_batcher.rs`**: Added functions `find_prom_special_column_names` and `rename_prom_special_columns_for_existing_schema` to handle renaming of special Prometheus columns. Updated `build_prom_create_table_schema` to simplify error handling with `ConcreteDataType`. - **Tests**: Added a test case `test_rename_prom_special_columns_for_existing_schema` to verify the renaming logic for Prometheus special columns. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - Refactored `PendingRowsBatcher` to accommodate Prometheus record batches: - Introduced `accommodate_record_batch_for_target_schema` to normalize incoming record batches against existing table schemas. - Removed `collect_missing_prom_tag_columns` and `rename_prom_special_columns_for_existing_schema` in favor of the new function. - Added `unzip_logical_region_schema` to extract schema components. - Updated tests in `pending_rows_batcher.rs`: - Added tests for `accommodate_record_batch_for_target_schema` to verify handling of missing tag columns and renaming of special columns. - Ensured error handling for missing timestamp and field columns in target schema. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Commit Summary - **Enhancement in Table Creation Logic**: Updated `prom_store.rs` to modify the handling of `table_options` during table creation. Specifically, `table_options` are now extended differently based on the `AutoCreateTableType`. For `Physical` tables, enforced `sst_format=flat` to optimize pending-rows writes by leveraging bulk memtables. Signed-off-by: Lei, HUANG * feat/auto-schema-align: Enhance Performance Monitoring in `pending_rows_batcher.rs` - Added performance monitoring timers to various stages of the `PendingRowsBatcher` process, including schema cache checks, table resolution, schema creation, and record batch alignment. - Improved schema handling by adding timers around schema alteration and missing column addition processes. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - **Enhance Concurrent Write Handling**: Introduced `FlushRegionWrite` and `FlushWriteResult` structs to manage region writes and their results. Added `flush_region_writes_concurrently` function to handle concurrent flushing of region writes based on `should_dispatch_concurrently` logic in `pending_rows_batcher.rs`. - **Testing Enhancements**: Added tests for concurrent dispatching of region writes and the logic for determining concurrent dispatch in `pending_rows_batcher.rs`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Add Histogram for Flush Stage Elapsed Time - **`metrics.rs`**: Introduced a new `HistogramVec` named `PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED` to track the elapsed time of pending rows batch flush stages. - **`pending_rows_batcher.rs`**: Replaced instances of `PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED` with `PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED` to measure the elapsed time for various flush stages, including `flush_write_region`, `flush_concat_table_batches`, `flush_resolve_table`, `flush_fetch_partition_rule`, `flush_split_record_batch`, `flush_filter_record_batch`, `flush_resolve_region_leader`, and `flush_encode_ipc`. Signed-off-by: Lei, HUANG * Add design doc for physical table batching in PendingRowsBatcher Signed-off-by: Lei, HUANG * Add implementation plan for physical table batching in PendingRowsBatcher * feat/auto-schema-align: ### Commit Message **Enhance Metric Engine with Physical Batch Processing** - **Add `metric-engine` Dependency**: Updated `Cargo.lock` and `Cargo.toml` to include `metric-engine` as a workspace dependency. - **Expose Batch Modifier Functions**: Changed visibility of `TagColumnInfo`, `compute_tsid_array`, and `modify_batch_sparse` in `batch_modifier.rs` to public, and made `batch_modifier` a public module in `lib.rs`. - **Implement Physical Batch Processing**: - Added functions `bulk_insert_physical_region` and `bulk_insert_logical_region` in `bulk_insert.rs` to handle physical and logical batch insertions. - Updated `pending_rows_batcher.rs` to attempt physical batch processing before falling back to logical processing, including new functions `flush_batch_physical` and `flush_batch_per_logical_table`. - **Enhance Testing**: - Added tests for physical region passthrough and empty batch handling in `bulk_insert.rs`. - Introduced `with_mito_config` in `test_util.rs` for customized test environments. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Enhance Batch Processing for Table Creation and Alteration - **`prom_store.rs`**: - Added `create_tables_if_missing_batch` and `add_missing_prom_tag_columns_batch` methods to handle batch creation of tables and batch alteration to add missing tag columns. - Implemented logic to determine missing tables and columns, and perform batch operations accordingly. - **`pending_rows_batcher.rs`**: - Updated `PendingRowsBatcher` to utilize batch methods for creating tables an adding missing columns. - Enhanced logic to resolve table schemas and accommodate record batches after batch operations. Signed-off-by: Lei, HUANG * perf: concurrent catalog lookups and eliminate redundant concat_batches on ingest path Replace sequential catalog_manager.table() calls with concurrent futures::future::join_all in align_table_batches_to_region_schema. This affects all three lookup loops: initial table resolution, post-create resolution, and post-alter schema refresh. Reduces O(N) sequential RPC latency to O(1) wall-clock time for requests with many distinct logical tables (e.g. Prometheus remote_write). Remove the per-logical-table concat_batches in flush_batch_physical. Instead of merging all chunks of a table into one RecordBatch before calling modify_batch_sparse, apply modify_batch_sparse directly to each chunk and collect all modified chunks for a single final concat. This eliminates one full data copy per logical table on the flush path. * refactor: extract Prometheus schema alignment helpers into prom_row_builder module Move six functions and their eight unit tests from pending_rows_batcher.rs (~2386 lines) into a new prom_row_builder.rs module (~776 lines), leaving the batcher at ~1665 lines focused on flush/worker machinery. Extracted functions: - accommodate_record_batch_for_target_schema (normalize incoming batch against existing table schema) - unzip_logical_region_schema (extract ts/field/tag columns) - build_prom_create_table_schema (build ColumnSchema vec for table creation) - align_record_batch_to_schema (reorder/fill/cast columns to target schema) - rows_to_record_batch (convert proto Rows to Arrow RecordBatch) - build_arrow_array (build Arrow arrays from proto values) Cleaned up 12 now-unused imports from pending_rows_batcher.rs. * feat/auto-schema-align: ### Enhance `PendingRowsBatcher` and `prom_row_builder` for Efficient Schema Handling - **`pending_rows_batcher.rs`:** - Refactored `submit` method to integrate table batch building and alignment into a single method `build_and_align_table_batches`. - Removed intermediate `RecordBatch` creation, optimizing the process by directly converting proto `RowInsertRequests` into aligned `RecordBatch`es. - Enhanced schema handling by identifying missing columns directly from proto schemas. - **`prom_row_builder.rs`:** - Introduced `rows_to_aligned_record_batch` for direct conversion of proto `Rows` into aligned `RecordBatch`es. - Added `identify_missing_columns_from_proto` to detect absent tag columns without intermediate `RecordBatch`. - Implemented `build_prom_create_table_schema_from_proto` to construct table schemas directly from proto schemas. Signed-off-by: Lei, HUANG * feat/auto-schema-align: Add elapsed time metrics for bulk insert operations - Updated `bulk_insert` method in `bulk_insert.rs` to record elapsed time metrics using `MITO_OPERATION_ELAPSED` for both physical and logical regions. - Added a new test `test_bulk_insert_records_elapsed_metric` to verify that the elapsed time metric is recorded correctly during bulk insert operations. Signed-off-by: Lei, HUANG * remove flush per logical region Signed-off-by: Lei, HUANG * feat/auto-schema-align: **Refactor `flush_batch` and `flush_batch_physical` functions** - Removed unused `catalog` and `schema` variables from `flush_batch` in `pending_rows_batcher.rs`. - Updated `flush_batch_physical` to directly use `ctx.current_catalog()` and `ctx.current_schema()` for resolving table names. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Remove Unused Function and Associated Test - **File:** `src/servers/src/prom_row_builder.rs` - Removed the unused function `build_prom_create_table_schema` which was responsible for building a `Vec` from an Arrow schema. - Deleted the associated test `test_build_prom_create_table_schema_from_request_schema` that validated the removed function. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - **Remove Test**: Deleted the `test_bulk_insert_records_elapsed_metric` test from `bulk_insert.rs`. - **Refactor Table Resolution**: Introduced `TableResolutionPlan` struct and refactored table resolution logic in `pending_rows_batcher.rs`. - **Enhance Table Handling**: Added functions for collecting non-empty table rows, unique table schemas, and handling table creation and alteration in `pending_rows_batcher.rs`. - **Add Tests**: Implemented tests for `collect_non_empty_table_rows` and `collect_unique_table_schemas` in `pending_rows_batcher.rs`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - **Refactor Error Handling**: Updated error handling in `pending_rows_batcher.rs` and `prom_row_builder.rs` to use `Snafu` error context for more descriptive error messages. - **Remove Unused Functionality**: Eliminated the `rows_to_record_batch` function and related test in `prom_row_builder.rs` as it was redundant. - **Simplify Function Return Types**: Modified `rows_to_aligned_record_batch` in `prom_row_builder.rs` to return only `RecordBatch` without missing columns, simplifying the function's interface and related tests. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Add Helper Function for Table Options in `prom_store.rs` - Introduced `fill_metric_physical_table_options` function to encapsulate logic for setting table options, ensuring the use of flat SST format and physical table metadata. - Updated `Instance` implementation to utilize the new helper function for setting table options. - Added a unit test `test_metric_physical_table_options_forces_flat_sst_format` to verify the correct application of table options. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - **Refactor `PendingRowsBatcher`**: Simplified worker retrieval logic in `get_or_spawn_worker` method by using a more concise conditional check. - **Metrics Update**: Added `PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED` metric in `pending_rows_batcher.rs`. - **Remove Unused Code**: Deleted multiple test functions related to record batch alignment and schema preparation in `pending_rows_batcher.rs` and `prom_row_builder.rs`. - **Function Visibility Change**: Made `build_prom_create_table_schema_from_proto` public in `prom_row_builder.rs`. Signed-off-by: Lei, HUANG * chore: remove plan Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Refactor and Simplify Schema Alteration Logic - **Removed Unused Methods**: Deleted `create_table_if_missing` and `add_missing_prom_tag_columns` methods from `PendingRowsSchemaAlterer` trait in `prom_store.rs` and `pending_rows_batcher.rs`. - **Error Handling Improvement**: Enhanced error handling in `create_tables_if_missing_batch` method to return a specific error message for unsupported `AutoCreateTableType` in `prom_store.rs`. - **Visibility Change**: Made `as_str` method public in `AutoCreateTableType` enum in `insert.rs` to support external access. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Commit Message Improve safety in `prom_row_builder.rs` - Updated `unzip_logical_region_schema` to use `saturating_sub` for safer capacity calculation of `tag_columns`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: Add TODO comments for future improvements in `pending_rows_batcher.rs` - Added a TODO comment to consider bounding the `flush_region_writes_concurrently` function. - Added a TODO comment to potentially limit the maximum rows to concatenate in the `flush_batch_physical` function. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Commit Message Enhance error handling in `pending_rows_batcher.rs` - Updated `collect_unique_table_schemas` to return a `Result` type, enabling error handling for duplicate table names. - Modified the function to return an error when duplicate table names are found in `table_rows`. - Adjusted test cases to handle the new `Result` return type in `collect_unique_table_schemas`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - **Refactor `partition_columns` Method**: Updated the `partition_columns` method in `multi_dim.rs`, `partition.rs`, and `splitter.rs` to return a slice reference instead of a cloned vector, improving performance by avoiding unnecessary cloning. - **Enhance Partition Handling**: Added functions `collect_tag_columns_and_non_tag_indices` and `strip_partition_columns_from_batch` in `pending_rows_batcher.rs` to manage partition columns more efficiently, including stripping partition columns from record batches. - **Update Tests**: Modified existing tests and added new ones in `pending_rows_batcher.rs` to verify the functionality of partition column handling, ensuring correct behavior of the new methods. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Enhance Schema Handling and Validation in `pending_rows_batcher.rs` - **Schema Validation Enhancements**: - Added checks for essential columns (`timestamp`, `value`) in `collect_tag_columns_and_non_tag_indices`. - Introduced `PHYSICAL_REGION_ESSENTIAL_COLUMN_COUNT` to ensure minimum column count in `strip_partition_columns_from_batch`. - Improved error handling for unexpected data types and duplicated columns. - **Function Modifications**: - Updated `strip_partition_columns_from_batch` to project essential columns without lookup. - Modified `flush_batch_physical` to use `essential_col_indices` instead of `non_tag_indices`. - **Test Enhancements**: - Added tests for schema validation, including checks for unexpected data types and duplicated columns. - Verified correct projection of essential columns in `strip_partition_columns_from_batch`. Files affected: `pending_rows_batcher.rs`, `tests`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: - **Add `smallvec` Dependency**: Updated `Cargo.lock` and `Cargo.toml` to include `smallvec` as a workspace dependency. - **Refactor Function**: Renamed `collect_tag_columns_and_non_tag_indices` to `columns_taxonomy` in `pending_rows_batcher.rs` and updated its return type to use `SmallVec`. - **Update Tests**: Modified test cases in `pending_rows_batcher.rs` to reflect changes in function name and return type. Signed-off-by: Lei, HUANG * feat/auto-schema-align: **Refactor `pending_rows_batcher.rs` to Simplify Table ID Handling** - Updated `TableBatch` struct to use `TableId` directly instead of `Option` for `table_id`. - Simplified logic in `flush_batch_physical` by removing the check for `None` in `table_id`. - Adjusted related logic in `start_worker` to accommodate the change in `table_id` handling. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Enhance Batch Processing Logic - **`pending_rows_batcher.rs`**: - Moved column taxonomy resolution inside the loop to handle schema variations across batches. - Added checks to skip processing if both tag columns and essential column indices are empty. - **Tests**: - Added `test_modify_batch_sparse_with_taxonomy_per_batch` to verify batch modification logic with varying schemas. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Remove Primary Key Column Check in `pending_rows_batcher.rs` - Removed the check for the primary key column and other essential column names in the function `strip_partition_columns_from_batch` within `pending_rows_batcher.rs`. - Simplified the logic by eliminating the validation of column order against expected essential names. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Refactor error handling and iteration in `otlp.rs` and `pending_rows_batcher.rs` - **`otlp.rs`**: Simplified error handling by removing `CatalogSnafu` context when awaiting table retrieval. - **`pending_rows_batcher.rs`**: Streamlined iteration over tables by removing unnecessary `into_iter()` calls, improving code readability and efficiency. Signed-off-by: Lei, HUANG * chore/metrics-for-bulk: Add timing metrics for batch processing in `pending_rows_batcher.rs` - Introduced `modify_elapsed` and `columns_taxonomy_elapsed` to measure time spent in `modify_batch_sparse` and `columns_taxonomy` functions. - Updated `flush_batch_physical` to record these metrics using `PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Commit Summary - **Remove Unused Code**: Eliminated the `#[allow(dead_code)]` attribute from the `compute_tsid_array` function in `batch_modifier.rs`. - **Error Handling Improvement**: Enhanced error handling in `flush_batch_physical` function by adjusting the `match` block in `pending_rows_batcher.rs`. - **Simplify Logic**: Streamlined the logic in `rows_to_aligned_record_batch` by removing unnecessary type casting in `prom_row_builder.rs`. Signed-off-by: Lei, HUANG * feat/auto-schema-align: **Refactor `flush_batch_physical` in `pending_rows_batcher.rs`:** - Moved partition column stripping logic to a single location before processing region batches. - Updated the use of `combined_batch` to `stripped_batch` for consistency in batch processing. - Removed redundant partition column stripping logic within the region batch loop. Signed-off-by: Lei, HUANG * feat/auto-schema-align: ### Update `batch_modifier.rs` Documentation and Parameter Naming - Enhanced documentation for `compute_tsid_array` and `modify_batch_sparse` functions to clarify their logic and parameters. - Renamed parameter `non_tag_column_indices` to `extra_column_indices` in `modify_batch_sparse` for better clarity. Signed-off-by: Lei, HUANG --------- Signed-off-by: Lei, HUANG --- Cargo.lock | 2 + src/frontend/src/instance/dashboard.rs | 11 +- src/frontend/src/instance/influxdb.rs | 7 +- src/frontend/src/instance/jaeger.rs | 9 +- src/frontend/src/instance/otlp.rs | 5 +- src/frontend/src/instance/prom_store.rs | 317 ++- src/frontend/src/server.rs | 2 + src/metric-engine/src/batch_modifier.rs | 41 +- src/metric-engine/src/engine/bulk_insert.rs | 139 +- src/metric-engine/src/lib.rs | 2 +- src/metric-engine/src/test_util.rs | 16 + src/operator/src/insert.rs | 2 +- src/partition/src/multi_dim.rs | 4 +- src/partition/src/partition.rs | 2 +- src/partition/src/splitter.rs | 42 +- src/servers/Cargo.toml | 2 + src/servers/src/error.rs | 10 +- src/servers/src/http/event.rs | 9 +- src/servers/src/http/prometheus.rs | 21 +- src/servers/src/lib.rs | 1 + src/servers/src/metrics.rs | 7 + src/servers/src/pending_rows_batcher.rs | 2006 +++++++++++++------ src/servers/src/pipeline.rs | 12 +- src/servers/src/prom_row_builder.rs | 557 +++++ 24 files changed, 2551 insertions(+), 675 deletions(-) create mode 100644 src/servers/src/prom_row_builder.rs diff --git a/Cargo.lock b/Cargo.lock index 401ac3b1ca..54be9bbdcb 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12057,6 +12057,7 @@ dependencies = [ "local-ip-address", "log-query", "loki-proto", + "metric-engine", "mime_guess", "mysql_async", "notify", @@ -12093,6 +12094,7 @@ dependencies = [ "session", "simd-json", "simdutf8", + "smallvec", "snafu 0.8.6", "snap", "socket2 0.5.10", diff --git a/src/frontend/src/instance/dashboard.rs b/src/frontend/src/instance/dashboard.rs index 373961dbfa..5b83a31f20 100644 --- a/src/frontend/src/instance/dashboard.rs +++ b/src/frontend/src/instance/dashboard.rs @@ -33,7 +33,7 @@ use datafusion::sql::TableReference; use datafusion_expr::{DmlStatement, LogicalPlan, lit}; use datatypes::arrow::array::{Array, AsArray}; use servers::error::{ - CatalogSnafu, CollectRecordbatchSnafu, DataFusionSnafu, ExecuteQuerySnafu, NotSupportedSnafu, + CollectRecordbatchSnafu, DataFusionSnafu, ExecuteQuerySnafu, NotSupportedSnafu, TableNotFoundSnafu, }; use servers::query_handler::DashboardDefinition; @@ -139,8 +139,7 @@ impl Instance { DASHBOARD_TABLE_NAME, Some(&ctx), ) - .await - .context(CatalogSnafu)? + .await? { return Ok(table); } @@ -178,8 +177,7 @@ impl Instance { DASHBOARD_TABLE_NAME, Some(&ctx), ) - .await - .context(CatalogSnafu)? + .await? .context(TableNotFoundSnafu { catalog: catalog.to_string(), schema: DEFAULT_PRIVATE_SCHEMA_NAME.to_string(), @@ -255,8 +253,7 @@ impl Instance { DASHBOARD_TABLE_NAME, Some(&query_ctx), ) - .await - .context(CatalogSnafu)? + .await? { table } else { diff --git a/src/frontend/src/instance/influxdb.rs b/src/frontend/src/instance/influxdb.rs index 0c63688262..fe5fdeac77 100644 --- a/src/frontend/src/instance/influxdb.rs +++ b/src/frontend/src/instance/influxdb.rs @@ -21,9 +21,7 @@ use client::Output; use common_error::ext::BoxedError; use common_time::Timestamp; use common_time::timestamp::TimeUnit; -use servers::error::{ - AuthSnafu, CatalogSnafu, Error, TimestampOverflowSnafu, UnexpectedResultSnafu, -}; +use servers::error::{AuthSnafu, Error, TimestampOverflowSnafu, UnexpectedResultSnafu}; use servers::influxdb::InfluxdbRequest; use servers::interceptor::{LineProtocolInterceptor, LineProtocolInterceptorRef}; use servers::query_handler::InfluxdbLineProtocolHandler; @@ -92,8 +90,7 @@ impl InfluxdbLineTimestampAligner<'_> { &insert.table_name, Some(query_context), ) - .await - .context(CatalogSnafu)? + .await? .map(|x| x.schema()) .and_then(|schema| { schema.timestamp_column().map(|col| { diff --git a/src/frontend/src/instance/jaeger.rs b/src/frontend/src/instance/jaeger.rs index 607ed80098..e500de7a85 100644 --- a/src/frontend/src/instance/jaeger.rs +++ b/src/frontend/src/instance/jaeger.rs @@ -38,8 +38,7 @@ use datafusion_expr::{Expr, ExprFunctionExt, SortExpr, col, lit, lit_timestamp_n use query::QueryEngineRef; use serde_json::Value as JsonValue; use servers::error::{ - CatalogSnafu, CollectRecordbatchSnafu, DataFusionSnafu, Result as ServerResult, - TableNotFoundSnafu, + CollectRecordbatchSnafu, DataFusionSnafu, Result as ServerResult, TableNotFoundSnafu, }; use servers::http::jaeger::{JAEGER_QUERY_TABLE_NAME_KEY, QueryTraceParams, TraceUserAgent}; use servers::otlp::trace::{ @@ -336,8 +335,7 @@ async fn query_trace_table( table_name, Some(&ctx), ) - .await - .context(CatalogSnafu)? + .await? .with_context(|| TableNotFoundSnafu { table: table_name, catalog: ctx.current_catalog(), @@ -425,8 +423,7 @@ async fn get_table( table_name, Some(&ctx), ) - .await - .context(CatalogSnafu)? + .await? .with_context(|| TableNotFoundSnafu { table: table_name, catalog: ctx.current_catalog(), diff --git a/src/frontend/src/instance/otlp.rs b/src/frontend/src/instance/otlp.rs index 9b21f9924f..59174aa89a 100644 --- a/src/frontend/src/instance/otlp.rs +++ b/src/frontend/src/instance/otlp.rs @@ -26,7 +26,7 @@ use opentelemetry_proto::tonic::collector::logs::v1::ExportLogsServiceRequest; use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; use otel_arrow_rust::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; use pipeline::{GreptimePipelineParams, PipelineWay}; -use servers::error::{self, AuthSnafu, CatalogSnafu, Result as ServerResult}; +use servers::error::{self, AuthSnafu, Result as ServerResult}; use servers::http::prom_store::PHYSICAL_TABLE_PARAM; use servers::interceptor::{OpenTelemetryProtocolInterceptor, OpenTelemetryProtocolInterceptorRef}; use servers::otlp; @@ -255,8 +255,7 @@ impl Instance { let table = self .catalog_manager .table(catalog, &schema, &req.table_name, None) - .await - .context(CatalogSnafu)?; + .await?; let Some(rows) = req.rows.as_mut() else { continue; diff --git a/src/frontend/src/instance/prom_store.rs b/src/frontend/src/instance/prom_store.rs index c8f76753af..10fc2f1790 100644 --- a/src/frontend/src/instance/prom_store.rs +++ b/src/frontend/src/instance/prom_store.rs @@ -17,7 +17,11 @@ use std::sync::Arc; use api::prom_store::remote::read_request::ResponseType; use api::prom_store::remote::{Query, QueryResult, ReadRequest, ReadResponse}; -use api::v1::RowInsertRequests; +use api::v1::alter_table_expr::Kind; +use api::v1::{ + AddColumn, AddColumns, AlterTableExpr, ColumnDataType, ColumnDef, CreateTableExpr, + RowInsertRequests, SemanticType, +}; use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::OutputData; @@ -27,19 +31,25 @@ use common_query::Output; use common_query::prelude::GREPTIME_PHYSICAL_TABLE; use common_recordbatch::RecordBatches; use common_telemetry::{debug, tracing}; -use operator::insert::InserterRef; +use operator::insert::{ + AutoCreateTableType, InserterRef, build_create_table_expr, fill_table_options_for_create, +}; use operator::statement::StatementExecutor; use prost::Message; use servers::error::{self, AuthSnafu, Result as ServerResult}; use servers::http::header::{CONTENT_ENCODING_SNAPPY, CONTENT_TYPE_PROTOBUF, collect_plan_metrics}; use servers::http::prom_store::PHYSICAL_TABLE_PARAM; use servers::interceptor::{PromStoreProtocolInterceptor, PromStoreProtocolInterceptorRef}; +use servers::pending_rows_batcher::PendingRowsSchemaAlterer; use servers::prom_store::{self, Metrics}; use servers::query_handler::{ PromStoreProtocolHandler, PromStoreProtocolHandlerRef, PromStoreResponse, }; use session::context::QueryContextRef; use snafu::{OptionExt, ResultExt}; +use store_api::metric_engine_consts::{METRIC_ENGINE_NAME, PHYSICAL_TABLE_METADATA_KEY}; +use store_api::mito_engine_options::SST_FORMAT_KEY; +use table::table_reference::TableReference; use tracing::instrument; use crate::error::{ @@ -50,6 +60,34 @@ use crate::instance::Instance; const SAMPLES_RESPONSE_TYPE: i32 = ResponseType::Samples as i32; +fn auto_create_table_type_for_prom_remote_write( + ctx: &QueryContextRef, + with_metric_engine: bool, +) -> AutoCreateTableType { + if with_metric_engine { + let physical_table = ctx + .extension(PHYSICAL_TABLE_PARAM) + .unwrap_or(GREPTIME_PHYSICAL_TABLE) + .to_string(); + AutoCreateTableType::Logical(physical_table) + } else { + AutoCreateTableType::Physical + } +} + +fn required_physical_table_for_create_type(create_type: &AutoCreateTableType) -> Option<&str> { + match create_type { + AutoCreateTableType::Logical(physical_table) => Some(physical_table.as_str()), + _ => None, + } +} + +fn fill_metric_physical_table_options(table_options: &mut HashMap) { + // We always enforce flat format in this ingestion path. + table_options.insert(SST_FORMAT_KEY.to_string(), "flat".to_string()); + table_options.insert(PHYSICAL_TABLE_METADATA_KEY.to_string(), "true".to_string()); +} + #[inline] fn is_supported(response_type: i32) -> bool { // Only supports samples response right now @@ -159,6 +197,157 @@ impl Instance { } } +#[async_trait] +impl PendingRowsSchemaAlterer for Instance { + async fn create_tables_if_missing_batch( + &self, + catalog: &str, + schema: &str, + tables: &[(&str, &[api::v1::ColumnSchema])], + with_metric_engine: bool, + ctx: QueryContextRef, + ) -> ServerResult<()> { + if tables.is_empty() { + return Ok(()); + } + + let create_type = auto_create_table_type_for_prom_remote_write(&ctx, with_metric_engine); + if let Some(physical_table) = required_physical_table_for_create_type(&create_type) { + self.create_metric_physical_table_if_missing( + catalog, + schema, + physical_table, + ctx.clone(), + ) + .await?; + } + + let engine = if matches!(create_type, AutoCreateTableType::Logical(_)) { + METRIC_ENGINE_NAME + } else { + common_catalog::consts::default_engine() + }; + + // Check which tables actually still need to be created (may have been + // concurrently created by another request). + let mut create_exprs: Vec = Vec::with_capacity(tables.len()); + for &(table_name, request_schema) in tables { + let existing = self + .catalog_manager() + .table(catalog, schema, table_name, Some(ctx.as_ref())) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + if existing.is_some() { + continue; + } + + let table_ref = TableReference::full(catalog, schema, table_name); + let mut create_table_expr = build_create_table_expr(&table_ref, request_schema, engine) + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + + let mut table_options = std::collections::HashMap::with_capacity(4); + fill_table_options_for_create(&mut table_options, &create_type, &ctx); + create_table_expr.table_options.extend(table_options); + create_exprs.push(create_table_expr); + } + + if create_exprs.is_empty() { + return Ok(()); + } + + match create_type { + AutoCreateTableType::Logical(_) => { + // Use the batch API for logical tables. + self.statement_executor + .create_logical_tables(&create_exprs, ctx) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + } + AutoCreateTableType::Physical => { + // Physical tables don't have a batch DDL path; create one at a time. + for mut expr in create_exprs { + expr.table_options + .insert(SST_FORMAT_KEY.to_string(), "flat".to_string()); + self.statement_executor + .create_table_inner(&mut expr, None, ctx.clone()) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + } + } + create_type => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!( + "prom remote write only supports logical or physical auto-create: {}", + create_type.as_str() + ), + } + .fail(); + } + } + + Ok(()) + } + + async fn add_missing_prom_tag_columns_batch( + &self, + catalog: &str, + schema: &str, + tables: &[(&str, &[String])], + ctx: QueryContextRef, + ) -> ServerResult<()> { + if tables.is_empty() { + return Ok(()); + } + + let alter_exprs: Vec = tables + .iter() + .filter(|(_, columns)| !columns.is_empty()) + .map(|&(table_name, columns)| { + let add_columns = AddColumns { + add_columns: columns + .iter() + .map(|column_name| AddColumn { + column_def: Some(ColumnDef { + name: column_name.clone(), + data_type: ColumnDataType::String as i32, + is_nullable: true, + semantic_type: SemanticType::Tag as i32, + comment: String::new(), + ..Default::default() + }), + location: None, + add_if_not_exists: true, + }) + .collect(), + }; + + AlterTableExpr { + catalog_name: catalog.to_string(), + schema_name: schema.to_string(), + table_name: table_name.to_string(), + kind: Some(Kind::AddColumns(add_columns)), + } + }) + .collect(); + + if alter_exprs.is_empty() { + return Ok(()); + } + + self.statement_executor + .alter_logical_tables(alter_exprs, ctx) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + + Ok(()) + } +} + #[async_trait] impl PromStoreProtocolHandler for Instance { async fn pre_write( @@ -267,6 +456,61 @@ impl PromStoreProtocolHandler for Instance { } } +impl Instance { + async fn create_metric_physical_table_if_missing( + &self, + catalog: &str, + schema: &str, + physical_table: &str, + ctx: QueryContextRef, + ) -> ServerResult<()> { + let table = self + .catalog_manager() + .table(catalog, schema, physical_table, Some(ctx.as_ref())) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + if table.is_some() { + return Ok(()); + } + + let table_ref = TableReference::full(catalog, schema, physical_table); + let default_schema = vec![ + api::v1::ColumnSchema { + column_name: common_query::prelude::greptime_timestamp().to_string(), + datatype: api::v1::ColumnDataType::TimestampMillisecond as i32, + semantic_type: api::v1::SemanticType::Timestamp as i32, + datatype_extension: None, + options: None, + }, + api::v1::ColumnSchema { + column_name: common_query::prelude::greptime_value().to_string(), + datatype: api::v1::ColumnDataType::Float64 as i32, + semantic_type: api::v1::SemanticType::Field as i32, + datatype_extension: None, + options: None, + }, + ]; + let mut create_table_expr = build_create_table_expr( + &table_ref, + &default_schema, + common_catalog::consts::default_engine(), + ) + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + create_table_expr.engine = METRIC_ENGINE_NAME.to_string(); + fill_metric_physical_table_options(&mut create_table_expr.table_options); + + self.statement_executor + .create_table_inner(&mut create_table_expr, None, ctx) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu)?; + + Ok(()) + } +} + /// This handler is mainly used for `frontend` or `standalone` to directly import /// the metrics collected by itself, thereby avoiding importing metrics through the network, /// thus reducing compression and network transmission overhead, @@ -320,3 +564,72 @@ impl PromStoreProtocolHandler for ExportMetricHandler { unreachable!(); } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use session::context::QueryContext; + + use super::*; + + #[test] + fn test_auto_create_table_type_for_prom_remote_write_metric_engine() { + let mut query_ctx = QueryContext::with( + common_catalog::consts::DEFAULT_CATALOG_NAME, + common_catalog::consts::DEFAULT_SCHEMA_NAME, + ); + query_ctx.set_extension(PHYSICAL_TABLE_PARAM, "metric_physical".to_string()); + let ctx = Arc::new(query_ctx); + + let create_type = auto_create_table_type_for_prom_remote_write(&ctx, true); + match create_type { + AutoCreateTableType::Logical(physical) => assert_eq!(physical, "metric_physical"), + _ => panic!("expected logical table create type"), + } + } + + #[test] + fn test_auto_create_table_type_for_prom_remote_write_without_metric_engine() { + let ctx = Arc::new(QueryContext::with( + common_catalog::consts::DEFAULT_CATALOG_NAME, + common_catalog::consts::DEFAULT_SCHEMA_NAME, + )); + + let create_type = auto_create_table_type_for_prom_remote_write(&ctx, false); + match create_type { + AutoCreateTableType::Physical => {} + _ => panic!("expected physical table create type"), + } + } + + #[test] + fn test_required_physical_table_for_create_type() { + let logical = AutoCreateTableType::Logical("phy_table".to_string()); + assert_eq!( + Some("phy_table"), + required_physical_table_for_create_type(&logical) + ); + + let physical = AutoCreateTableType::Physical; + assert_eq!(None, required_physical_table_for_create_type(&physical)); + } + + #[test] + fn test_metric_physical_table_options_forces_flat_sst_format() { + let mut table_options = HashMap::new(); + + fill_metric_physical_table_options(&mut table_options); + + assert_eq!( + Some("flat"), + table_options.get(SST_FORMAT_KEY).map(String::as_str) + ); + assert_eq!( + Some("true"), + table_options + .get(PHYSICAL_TABLE_METADATA_KEY) + .map(String::as_str) + ); + } +} diff --git a/src/frontend/src/server.rs b/src/frontend/src/server.rs index 4d0db700d1..e66ae718ba 100644 --- a/src/frontend/src/server.rs +++ b/src/frontend/src/server.rs @@ -130,6 +130,8 @@ where self.instance.partition_manager().clone(), self.instance.node_manager().clone(), self.instance.catalog_manager().clone(), + opts.prom_store.with_metric_engine, + self.instance.clone(), opts.prom_store.pending_rows_flush_interval, opts.prom_store.max_batch_rows, opts.prom_store.max_concurrent_flushes, diff --git a/src/metric-engine/src/batch_modifier.rs b/src/metric-engine/src/batch_modifier.rs index d06eaa976b..76d9bb418a 100644 --- a/src/metric-engine/src/batch_modifier.rs +++ b/src/metric-engine/src/batch_modifier.rs @@ -28,7 +28,7 @@ use crate::error::{EncodePrimaryKeySnafu, Result, UnexpectedRequestSnafu}; /// Info about a tag column for TSID computation and sparse primary key encoding. #[allow(dead_code)] -pub(crate) struct TagColumnInfo { +pub struct TagColumnInfo { /// Column name (used for label-name hash). pub name: String, /// Column index in the RecordBatch. @@ -37,9 +37,16 @@ pub(crate) struct TagColumnInfo { pub column_id: ColumnId, } -/// Computes `__tsid` values for each row. -#[allow(dead_code)] -pub(crate) fn compute_tsid_array( +/// Computes the TSID for each row in a [RecordBatch]. +/// +/// The TSID is a stable hash of the set of labels (tags) present in each row. +/// It accounts for both the names and values of all non-null tag columns. +/// +/// # Logic +/// - If a row has no nulls across all `sorted_tag_columns`, it uses a precomputed hash of all label names. +/// - If a row has nulls, it dynamically computes a hash of the names of labels that are present (non-null). +/// - In both cases, it then hashes the values of those present labels in the order specified by `sorted_tag_columns`. +pub fn compute_tsid_array( batch: &RecordBatch, sorted_tag_columns: &[TagColumnInfo], tag_arrays: &[&StringArray], @@ -110,12 +117,30 @@ fn build_tag_arrays<'a>( .collect() } -/// Modifies a RecordBatch for sparse primary key encoding. -pub(crate) fn modify_batch_sparse( +/// Modifies a [RecordBatch] to include a sparse primary key column. +/// +/// This function transforms the input `batch` into a new `RecordBatch` where the first column +/// is the generated primary key (named [PRIMARY_KEY_COLUMN_NAME]), followed by columns +/// indicated by `extra_column_indices`. +/// +/// The primary key uses a "sparse" encoding, which compactly represents the row's identity +/// by only including non-null tag values. The encoding, handled by [SparsePrimaryKeyCodec], +/// consists of: +/// 1. The `table_id`. +/// 2. A `tsid` (Time Series ID), which is a hash of the present tags. +/// 3. The actual non-null tag values paired with their `column_id`. +/// +/// # Parameters +/// - `batch`: The source [RecordBatch]. +/// - `table_id`: The ID of the table. +/// - `sorted_tag_columns`: Metadata for tag columns, used for both TSID computation and PK encoding. +/// - `extra_column_indices`: Indices of columns from the original batch to keep in the output +/// (typically the timestamp and value fields). +pub fn modify_batch_sparse( batch: RecordBatch, table_id: u32, sorted_tag_columns: &[TagColumnInfo], - non_tag_column_indices: &[usize], + extra_column_indices: &[usize], ) -> Result { let num_rows = batch.num_rows(); let codec = SparsePrimaryKeyCodec::schemaless(); @@ -151,7 +176,7 @@ pub(crate) fn modify_batch_sparse( ))]; let mut columns: Vec> = vec![Arc::new(pk_array)]; - for &idx in non_tag_column_indices { + for &idx in extra_column_indices { fields.push(batch.schema().fields()[idx].clone()); columns.push(batch.column(idx).clone()); } diff --git a/src/metric-engine/src/engine/bulk_insert.rs b/src/metric-engine/src/engine/bulk_insert.rs index 8122cdc958..300bd34647 100644 --- a/src/metric-engine/src/engine/bulk_insert.rs +++ b/src/metric-engine/src/engine/bulk_insert.rs @@ -34,18 +34,20 @@ use crate::batch_modifier::{TagColumnInfo, modify_batch_sparse}; use crate::engine::MetricEngineInner; use crate::error; use crate::error::Result; +use crate::metrics::MITO_OPERATION_ELAPSED; impl MetricEngineInner { - /// Bulk-inserts logical rows into a metric region. + /// Bulk-inserts rows into a metric region. /// - /// This method accepts a `RegionBulkInsertsRequest` whose payload is a logical - /// `RecordBatch` (timestamp, value and tag columns) for the given logical `region_id`. + /// **Logical region path:** The request payload is a logical `RecordBatch` + /// (timestamp, value and tag columns). It is transformed to physical format + /// via `modify_batch_sparse`, encoded to Arrow IPC, and forwarded as a + /// `BulkInserts` request to the data region. If mito reports + /// `StatusCode::Unsupported`, the request is transparently retried as a `Put`. /// - /// The transformed batch is encoded to Arrow IPC and forwarded as a `BulkInserts` - /// request to the data region, along with the original `partition_expr_version`. - /// If the data region reports `StatusCode::Unsupported` for bulk inserts, the request - /// is transparently retried as a `Put` by converting the original logical batch into - /// `api::v1::Rows`, so callers observe the same semantics as `put_region`. + /// **Physical region path:** The request payload is already in physical format + /// (produced by the batcher's `flush_batch_physical`). It is forwarded directly + /// to the data region with no transformation. /// /// Returns the number of affected rows, or `0` if the input batch is empty. pub async fn bulk_insert_region( @@ -53,13 +55,42 @@ impl MetricEngineInner { region_id: RegionId, request: RegionBulkInsertsRequest, ) -> Result { - ensure!( - !self.is_physical_region(region_id), - error::UnsupportedRegionRequestSnafu { - request: RegionRequest::BulkInserts(request), - } - ); + if request.payload.num_rows() == 0 { + return Ok(0); + } + if self.is_physical_region(region_id) { + let _timer = MITO_OPERATION_ELAPSED + .with_label_values(&["bulk_insert_physical"]) + .start_timer(); + return self.bulk_insert_physical_region(region_id, request).await; + } + let _timer = MITO_OPERATION_ELAPSED + .with_label_values(&["bulk_insert_logical"]) + .start_timer(); + self.bulk_insert_logical_region(region_id, request).await + } + + /// Passthrough for bulk inserts targeting a physical data region. + /// + /// The batch is already in physical format (with `__primary_key`, timestamp, + /// value columns), so no logical-to-physical transformation is needed. + async fn bulk_insert_physical_region( + &self, + region_id: RegionId, + request: RegionBulkInsertsRequest, + ) -> Result { + self.data_region + .write_data(region_id, RegionRequest::BulkInserts(request)) + .await + } + + /// Bulk-inserts logical rows, transforming them to physical format first. + async fn bulk_insert_logical_region( + &self, + region_id: RegionId, + request: RegionBulkInsertsRequest, + ) -> Result { let (physical_region_id, data_region_id, primary_key_encoding) = self.find_data_region_meta(region_id)?; @@ -390,6 +421,7 @@ mod tests { use datatypes::arrow::array::{Float64Array, StringArray, TimestampMillisecondArray}; use datatypes::arrow::datatypes::{DataType, Field, Schema as ArrowSchema, TimeUnit}; use datatypes::arrow::record_batch::RecordBatch; + use mito2::config::MitoConfig; use store_api::metric_engine_consts::MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING; use store_api::path_utils::table_dir; use store_api::region_engine::RegionEngine; @@ -397,6 +429,7 @@ mod tests { use store_api::storage::{RegionId, ScanRequest}; use super::record_batch_to_ipc; + use crate::batch_modifier::{TagColumnInfo, modify_batch_sparse}; use crate::error::Error; use crate::test_util::{self, TestEnv}; @@ -492,23 +525,81 @@ mod tests { } #[tokio::test] - async fn test_bulk_insert_physical_region_rejected() { - let env = TestEnv::new().await; + async fn test_bulk_insert_physical_region_passthrough() { + // Use flat format so that BulkMemtable is used (supports write_bulk). + let mito_config = MitoConfig { + default_experimental_flat_format: true, + ..Default::default() + }; + let env = TestEnv::with_mito_config("", mito_config, Default::default()).await; env.init_metric_region().await; - let physical_region_id = env.default_physical_region_id(); - let batch = build_logical_batch(0, 2); - let request = build_bulk_request(physical_region_id, batch); + let logical_region_id = env.default_logical_region_id(); - let err = env + // First, do a normal logical bulk insert so we can compare results. + let logical_batch = build_logical_batch(0, 3); + let logical_request = build_bulk_request(logical_region_id, logical_batch.clone()); + let response = env + .metric() + .handle_request(logical_region_id, logical_request) + .await + .unwrap(); + assert_eq!(response.affected_rows, 3); + + // Now build a physical-format batch using modify_batch_sparse (simulating + // what the batcher's flush_batch_physical does) and send it directly to + // the physical region. + let tag_columns = vec![TagColumnInfo { + name: "job".to_string(), + index: 2, + column_id: 2, // column_id for "job" in the physical table + }]; + let non_tag_indices = vec![0, 1]; // timestamp, value + let second_batch = build_logical_batch(3, 3); + let physical_batch = modify_batch_sparse( + second_batch, + logical_region_id.table_id(), + &tag_columns, + &non_tag_indices, + ) + .unwrap(); + let request = build_bulk_request(physical_region_id, physical_batch); + let response = env .metric() .handle_request(physical_region_id, request) .await - .unwrap_err(); - let Some(err) = err.as_any().downcast_ref::() else { - panic!("unexpected error type"); + .unwrap(); + assert_eq!(response.affected_rows, 3); + + // Verify all 6 rows are readable from the logical region. + let stream = env + .metric() + .scan_to_stream(logical_region_id, ScanRequest::default()) + .await + .unwrap(); + let batches = RecordBatches::try_collect(stream).await.unwrap(); + assert_eq!(batches.iter().map(|b| b.num_rows()).sum::(), 6); + } + + #[tokio::test] + async fn test_bulk_insert_physical_region_empty_batch() { + // Use flat format so that BulkMemtable is used (supports write_bulk). + let mito_config = MitoConfig { + default_experimental_flat_format: true, + ..Default::default() }; - assert_matches!(err, Error::UnsupportedRegionRequest { .. }); + let env = TestEnv::with_mito_config("", mito_config, Default::default()).await; + env.init_metric_region().await; + let physical_region_id = env.default_physical_region_id(); + + let batch = build_logical_batch(0, 0); + let request = build_bulk_request(physical_region_id, batch); + let response = env + .metric() + .handle_request(physical_region_id, request) + .await + .unwrap(); + assert_eq!(response.affected_rows, 0); } #[tokio::test] diff --git a/src/metric-engine/src/lib.rs b/src/metric-engine/src/lib.rs index 557baba25a..d209eb7588 100644 --- a/src/metric-engine/src/lib.rs +++ b/src/metric-engine/src/lib.rs @@ -52,7 +52,7 @@ #![recursion_limit = "256"] -mod batch_modifier; +pub mod batch_modifier; pub mod config; mod data_region; pub mod engine; diff --git a/src/metric-engine/src/test_util.rs b/src/metric-engine/src/test_util.rs index d3e929cf63..ec55a01903 100644 --- a/src/metric-engine/src/test_util.rs +++ b/src/metric-engine/src/test_util.rs @@ -76,6 +76,22 @@ impl TestEnv { } } + /// Returns a new env with specific `prefix`, `mito_config`, and `config` for test. + pub async fn with_mito_config( + prefix: &str, + mito_config: MitoConfig, + config: EngineConfig, + ) -> Self { + let mut mito_env = MitoTestEnv::with_prefix(prefix).await; + let mito = mito_env.create_engine(mito_config).await; + let metric = MetricEngine::try_new(mito.clone(), config).unwrap(); + Self { + mito_env, + mito, + metric, + } + } + /// Returns a new env with specific `prefix` and `mito_env` for test. pub async fn with_mito_env(mut mito_env: MitoTestEnv) -> Self { let mito = mito_env.create_engine(MitoConfig::default()).await; diff --git a/src/operator/src/insert.rs b/src/operator/src/insert.rs index e1f121699e..aecca50e09 100644 --- a/src/operator/src/insert.rs +++ b/src/operator/src/insert.rs @@ -103,7 +103,7 @@ pub enum AutoCreateTableType { } impl AutoCreateTableType { - fn as_str(&self) -> &'static str { + pub fn as_str(&self) -> &'static str { match self { AutoCreateTableType::Logical(_) => "logical", AutoCreateTableType::Physical => "physical", diff --git a/src/partition/src/multi_dim.rs b/src/partition/src/multi_dim.rs index 8825c6de59..a68479888b 100644 --- a/src/partition/src/multi_dim.rs +++ b/src/partition/src/multi_dim.rs @@ -338,8 +338,8 @@ impl PartitionRule for MultiDimPartitionRule { self } - fn partition_columns(&self) -> Vec { - self.partition_columns.clone() + fn partition_columns(&self) -> &[String] { + &self.partition_columns } fn find_region(&self, values: &[Value]) -> Result { diff --git a/src/partition/src/partition.rs b/src/partition/src/partition.rs index 110f61a39e..f4c585e404 100644 --- a/src/partition/src/partition.rs +++ b/src/partition/src/partition.rs @@ -29,7 +29,7 @@ pub type PartitionRuleRef = Arc; pub trait PartitionRule: Sync + Send { fn as_any(&self) -> &dyn Any; - fn partition_columns(&self) -> Vec; + fn partition_columns(&self) -> &[String]; /// Finds the target region by the partition values. /// diff --git a/src/partition/src/splitter.rs b/src/partition/src/splitter.rs index fa19b74ad3..176422a173 100644 --- a/src/partition/src/splitter.rs +++ b/src/partition/src/splitter.rs @@ -66,7 +66,7 @@ impl<'a> SplitReadRowHelper<'a> { .collect::>(); let partition_cols = partition_rule.partition_columns(); let partition_cols_indexes = partition_cols - .into_iter() + .iter() .map(|col_name| col_name_to_idx.get(&col_name).cloned()) .collect::>(); @@ -176,15 +176,25 @@ mod tests { } #[derive(Debug, Serialize, Deserialize)] - struct MockPartitionRule; + struct MockPartitionRule { + partition_columns: Vec, + } + + impl Default for MockPartitionRule { + fn default() -> Self { + Self { + partition_columns: vec!["id".to_string()], + } + } + } impl PartitionRule for MockPartitionRule { fn as_any(&self) -> &dyn Any { self } - fn partition_columns(&self) -> Vec { - vec!["id".to_string()] + fn partition_columns(&self) -> &[String] { + &self.partition_columns } fn find_region(&self, values: &[Value]) -> Result { @@ -206,15 +216,25 @@ mod tests { } #[derive(Debug, Serialize, Deserialize)] - struct MockMissedColPartitionRule; + struct MockMissedColPartitionRule { + partition_columns: Vec, + } + + impl Default for MockMissedColPartitionRule { + fn default() -> Self { + Self { + partition_columns: vec!["missed_col".to_string()], + } + } + } impl PartitionRule for MockMissedColPartitionRule { fn as_any(&self) -> &dyn Any { self } - fn partition_columns(&self) -> Vec { - vec!["missed_col".to_string()] + fn partition_columns(&self) -> &[String] { + &self.partition_columns } fn find_region(&self, values: &[Value]) -> Result { @@ -243,8 +263,8 @@ mod tests { self } - fn partition_columns(&self) -> Vec { - vec![] + fn partition_columns(&self) -> &[String] { + &[] } fn find_region(&self, _values: &[Value]) -> Result { @@ -261,7 +281,7 @@ mod tests { #[test] fn test_writer_splitter() { let rows = mock_rows(); - let rule = Arc::new(MockPartitionRule) as PartitionRuleRef; + let rule = Arc::new(MockPartitionRule::default()) as PartitionRuleRef; let splitter = RowSplitter::new(rule); let mut splits = splitter.split(rows).unwrap(); @@ -276,7 +296,7 @@ mod tests { #[test] fn test_missed_col_writer_splitter() { let rows = mock_rows(); - let rule = Arc::new(MockMissedColPartitionRule) as PartitionRuleRef; + let rule = Arc::new(MockMissedColPartitionRule::default()) as PartitionRuleRef; let splitter = RowSplitter::new(rule); let mut splits = splitter.split(rows).unwrap(); diff --git a/src/servers/Cargo.toml b/src/servers/Cargo.toml index 6531390ca3..55bb41ee51 100644 --- a/src/servers/Cargo.toml +++ b/src/servers/Cargo.toml @@ -78,6 +78,7 @@ jsonb.workspace = true lazy_static.workspace = true log-query.workspace = true loki-proto.workspace = true +metric-engine.workspace = true mime_guess = "2.0" notify.workspace = true object-pool = "0.5" @@ -114,6 +115,7 @@ serde_json.workspace = true session.workspace = true simd-json.workspace = true simdutf8 = "0.1" +smallvec.workspace = true snafu.workspace = true snap = "1" socket2 = "0.5" diff --git a/src/servers/src/error.rs b/src/servers/src/error.rs index 5fae7a82db..682288b271 100644 --- a/src/servers/src/error.rs +++ b/src/servers/src/error.rs @@ -392,7 +392,7 @@ pub enum Error { location: Location, }, - #[snafu(display("Error accessing catalog"))] + #[snafu(transparent)] Catalog { source: catalog::error::Error, #[snafu(implicit)] @@ -678,6 +678,13 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(transparent)] + DataTypes { + source: datatypes::error::Error, + #[snafu(implicit)] + location: Location, + }, } pub type Result = std::result::Result; @@ -756,6 +763,7 @@ impl ErrorExt for Error { Catalog { source, .. } => source.status_code(), RowWriter { source, .. } => source.status_code(), + DataTypes { source, .. } => source.status_code(), TlsRequired { .. } => StatusCode::Unknown, Auth { source, .. } => source.status_code(), diff --git a/src/servers/src/http/event.rs b/src/servers/src/http/event.rs index 24bb844dc7..dc468a9b75 100644 --- a/src/servers/src/http/event.rs +++ b/src/servers/src/http/event.rs @@ -49,7 +49,7 @@ use table::table_reference::TableReference; use vrl::value::{KeyString, Value as VrlValue}; use crate::error::{ - CatalogSnafu, Error, InvalidParameterSnafu, OtherSnafu, ParseJsonSnafu, PipelineSnafu, Result, + Error, InvalidParameterSnafu, OtherSnafu, ParseJsonSnafu, PipelineSnafu, Result, status_code_to_http_status, }; use crate::http::HttpResponse; @@ -265,12 +265,7 @@ pub async fn query_pipeline_ddl( .map_err(BoxedError::new) .context(OtherSnafu)?; - let message = if handler - .get_table(&table_name, &query_ctx) - .await - .context(CatalogSnafu)? - .is_some() - { + let message = if handler.get_table(&table_name, &query_ctx).await?.is_some() { Some(CREATE_TABLE_SQL_TABLE_EXISTS.to_string()) } else if pipeline.is_variant_table_name() { Some(CREATE_TABLE_SQL_SUFFIX_EXISTS.to_string()) diff --git a/src/servers/src/http/prometheus.rs b/src/servers/src/http/prometheus.rs index 60ad780beb..63149948a5 100644 --- a/src/servers/src/http/prometheus.rs +++ b/src/servers/src/http/prometheus.rs @@ -65,9 +65,8 @@ use store_api::metric_engine_consts::{ pub use super::result::prometheus_resp::PrometheusJsonResponse; use crate::error::{ - CatalogSnafu, CollectRecordbatchSnafu, ConvertScalarValueSnafu, DataFusionSnafu, Error, - InvalidQuerySnafu, NotSupportedSnafu, ParseTimestampSnafu, Result, TableNotFoundSnafu, - UnexpectedResultSnafu, + CollectRecordbatchSnafu, ConvertScalarValueSnafu, DataFusionSnafu, Error, InvalidQuerySnafu, + NotSupportedSnafu, ParseTimestampSnafu, Result, TableNotFoundSnafu, UnexpectedResultSnafu, }; use crate::http::header::collect_plan_metrics; use crate::prom_store::{FIELD_NAME_LABEL, METRIC_NAME_LABEL, is_database_selection_label}; @@ -662,8 +661,7 @@ async fn retrieve_series_from_query_result( table_name, Some(query_ctx), ) - .await - .context(CatalogSnafu)? + .await? .with_context(|| TableNotFoundSnafu { catalog: query_ctx.current_catalog(), schema: query_ctx.current_schema(), @@ -1440,7 +1438,7 @@ async fn retrieve_table_names( }); while let Some(table) = tables_stream.next().await { - let table = table.context(CatalogSnafu)?; + let table = table?; if !table .table_info() .meta @@ -1497,7 +1495,7 @@ async fn retrieve_field_names( .next() .await { - let table = table.context(CatalogSnafu)?; + let table = table?; for column in table.field_columns() { field_columns.insert(column.name); } @@ -1508,8 +1506,7 @@ async fn retrieve_field_names( for table_name in matches { let table = manager .table(catalog, &schema, &table_name, Some(query_ctx)) - .await - .context(CatalogSnafu)? + .await? .with_context(|| TableNotFoundSnafu { catalog: catalog.to_string(), schema: schema.clone(), @@ -1533,8 +1530,7 @@ async fn retrieve_schema_names( let candidate_schemas = catalog_manager .schema_names(catalog, Some(query_ctx)) - .await - .context(CatalogSnafu)?; + .await?; for schema in candidate_schemas { let mut found = true; @@ -1542,8 +1538,7 @@ async fn retrieve_schema_names( if let Some(table_name) = retrieve_metric_name_from_promql(match_item) { let exists = catalog_manager .table_exists(catalog, &schema, &table_name, Some(query_ctx)) - .await - .context(CatalogSnafu)?; + .await?; if !exists { found = false; break; diff --git a/src/servers/src/lib.rs b/src/servers/src/lib.rs index 41d73b109f..44587783be 100644 --- a/src/servers/src/lib.rs +++ b/src/servers/src/lib.rs @@ -42,6 +42,7 @@ pub mod pending_rows_batcher; mod pipeline; pub mod postgres; pub mod prom_remote_write; +pub(crate) mod prom_row_builder; pub mod prom_store; pub mod prometheus; pub mod prometheus_handler; diff --git a/src/servers/src/metrics.rs b/src/servers/src/metrics.rs index e3bff7fdbc..6f151db539 100644 --- a/src/servers/src/metrics.rs +++ b/src/servers/src/metrics.rs @@ -177,6 +177,13 @@ lazy_static! { vec![0.0005, 0.001, 0.005, 0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 60.0] ) .unwrap(); + pub static ref PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED: HistogramVec = register_histogram_vec!( + "greptime_prom_store_pending_rows_batch_flush_stage_elapsed", + "Elapsed time of pending rows batch flush stages in seconds", + &["stage"], + vec![0.0005, 0.001, 0.005, 0.01, 0.05, 0.1, 0.5, 1.0, 5.0, 10.0, 60.0] + ) + .unwrap(); /// Http prometheus read duration per database. pub static ref METRIC_HTTP_PROM_STORE_READ_ELAPSED: HistogramVec = register_histogram_vec!( "greptime_servers_http_prometheus_read_elapsed", diff --git a/src/servers/src/pending_rows_batcher.rs b/src/servers/src/pending_rows_batcher.rs index f8486e3636..b6e07d2a81 100644 --- a/src/servers/src/pending_rows_batcher.rs +++ b/src/servers/src/pending_rows_batcher.rs @@ -12,53 +12,79 @@ // 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::{Duration, Instant}; -use api::helper::ColumnDataTypeWrapper; +use api::v1::meta::Peer; use api::v1::region::{ BulkInsertRequest, RegionRequest, RegionRequestHeader, bulk_insert_request, region_request, }; -use api::v1::value::ValueData; -use api::v1::{ArrowIpc, RowInsertRequests, Rows}; -use arrow::array::{ - ArrayRef, Float64Builder, StringBuilder, TimestampMicrosecondBuilder, - TimestampMillisecondBuilder, TimestampNanosecondBuilder, TimestampSecondBuilder, - new_null_array, -}; -use arrow::compute::{cast, concat_batches, filter_record_batch}; -use arrow::datatypes::{Field, Schema as ArrowSchema}; +use api::v1::{ArrowIpc, ColumnSchema, RowInsertRequests, Rows}; +use arrow::compute::{concat_batches, filter_record_batch}; +use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema, TimeUnit}; use arrow::record_batch::RecordBatch; -use arrow_schema::TimeUnit; +use async_trait::async_trait; use bytes::Bytes; use catalog::CatalogManagerRef; use common_grpc::flight::{FlightEncoder, FlightMessage}; use common_meta::node_manager::NodeManagerRef; -use common_query::prelude::GREPTIME_PHYSICAL_TABLE; +use common_query::prelude::{GREPTIME_PHYSICAL_TABLE, greptime_timestamp, greptime_value}; use common_telemetry::tracing_context::TracingContext; -use common_telemetry::{debug, error, info, warn}; +use common_telemetry::{debug, error, warn}; use dashmap::DashMap; use dashmap::mapref::entry::Entry; -use datatypes::data_type::DataType; -use datatypes::prelude::ConcreteDataType; +use metric_engine::batch_modifier::{TagColumnInfo, modify_batch_sparse}; use partition::manager::PartitionRuleManagerRef; use session::context::QueryContextRef; -use snafu::{ResultExt, ensure}; -use store_api::storage::RegionId; +use smallvec::SmallVec; +use snafu::{OptionExt, ensure}; +use store_api::storage::{RegionId, TableId}; use tokio::sync::{OwnedSemaphorePermit, Semaphore, broadcast, mpsc, oneshot}; use crate::error; use crate::error::{Error, Result}; use crate::metrics::{ FLUSH_DROPPED_ROWS, FLUSH_ELAPSED, FLUSH_FAILURES, FLUSH_ROWS, FLUSH_TOTAL, PENDING_BATCHES, - PENDING_ROWS, PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED, PENDING_WORKERS, + PENDING_ROWS, PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED, PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED, + PENDING_WORKERS, +}; +use crate::prom_row_builder::{ + build_prom_create_table_schema_from_proto, identify_missing_columns_from_proto, + rows_to_aligned_record_batch, }; const PHYSICAL_TABLE_KEY: &str = "physical_table"; /// Whether wait for ingestion result before reply to client. const PENDING_ROWS_BATCH_SYNC_ENV: &str = "PENDING_ROWS_BATCH_SYNC"; const WORKER_IDLE_TIMEOUT_MULTIPLIER: u32 = 3; +const PHYSICAL_REGION_ESSENTIAL_COLUMN_COUNT: usize = 3; + +#[async_trait] +pub trait PendingRowsSchemaAlterer: Send + Sync { + /// Batch-create multiple logical tables that are missing. + /// Each entry is `(table_name, request_schema)`. + async fn create_tables_if_missing_batch( + &self, + catalog: &str, + schema: &str, + tables: &[(&str, &[ColumnSchema])], + with_metric_engine: bool, + ctx: QueryContextRef, + ) -> Result<()>; + + /// Batch-alter multiple logical tables to add missing tag columns. + /// Each entry is `(table_name, missing_column_names)`. + async fn add_missing_prom_tag_columns_batch( + &self, + catalog: &str, + schema: &str, + tables: &[(&str, &[String])], + ctx: QueryContextRef, + ) -> Result<()>; +} + +pub type PendingRowsSchemaAltererRef = Arc; #[derive(Debug, Clone, Hash, Eq, PartialEq)] struct BatchKey { @@ -70,10 +96,22 @@ struct BatchKey { #[derive(Debug)] struct TableBatch { table_name: String, + table_id: TableId, batches: Vec, row_count: usize, } +/// Intermediate planning state for resolving and preparing logical tables +/// before row-to-batch alignment. +struct TableResolutionPlan { + /// Resolved table schema and table id by logical table name. + region_schemas: HashMap, u32)>, + /// Missing tables that need to be created before alignment. + tables_to_create: Vec<(String, Vec)>, + /// Existing tables that need tag-column schema evolution. + tables_to_alter: Vec<(String, Vec)>, +} + struct PendingBatch { tables: HashMap, created_at: Option, @@ -101,7 +139,7 @@ struct PendingWorker { enum WorkerCommand { Submit { - table_batches: Vec<(String, RecordBatch)>, + table_batches: Vec<(String, u32, RecordBatch)>, total_rows: usize, ctx: QueryContextRef, response_tx: oneshot::Sender>, @@ -134,6 +172,8 @@ pub struct PendingRowsBatcher { flush_semaphore: Arc, inflight_semaphore: Arc, worker_channel_capacity: usize, + prom_store_with_metric_engine: bool, + schema_alterer: PendingRowsSchemaAltererRef, pending_rows_batch_sync: bool, shutdown: broadcast::Sender<()>, } @@ -144,6 +184,8 @@ impl PendingRowsBatcher { partition_manager: PartitionRuleManagerRef, node_manager: NodeManagerRef, catalog_manager: CatalogManagerRef, + prom_store_with_metric_engine: bool, + schema_alterer: PendingRowsSchemaAltererRef, flush_interval: Duration, max_batch_rows: usize, max_concurrent_flushes: usize, @@ -178,6 +220,8 @@ impl PendingRowsBatcher { partition_manager, node_manager, catalog_manager, + prom_store_with_metric_engine, + schema_alterer, flush_semaphore: Arc::new(Semaphore::new(max_concurrent_flushes)), inflight_semaphore: Arc::new(Semaphore::new(max_inflight_requests)), worker_channel_capacity, @@ -189,20 +233,13 @@ impl PendingRowsBatcher { pub async fn submit(&self, requests: RowInsertRequests, ctx: QueryContextRef) -> Result { let (table_batches, total_rows) = { let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["submit_build_table_batches"]) + .with_label_values(&["submit_build_and_align"]) .start_timer(); - build_table_batches(requests)? + self.build_and_align_table_batches(requests, &ctx).await? }; if total_rows == 0 { return Ok(0); } - let table_batches = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["submit_align_region_schema"]) - .start_timer(); - self.align_table_batches_to_region_schema(table_batches, &ctx) - .await? - }; let permit = { let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED @@ -212,7 +249,7 @@ impl PendingRowsBatcher { .clone() .acquire_owned() .await - .map_err(|_| Error::BatcherChannelClosed)? + .map_err(|_| error::BatcherChannelClosedSnafu.build())? }; let (response_tx, response_rx) = oneshot::channel(); @@ -260,7 +297,9 @@ impl PendingRowsBatcher { let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED .with_label_values(&["submit_wait_flush_result"]) .start_timer(); - response_rx.await.map_err(|_| Error::BatcherChannelClosed)? + response_rx + .await + .map_err(|_| error::BatcherChannelClosedSnafu.build())? }; result.map(|()| total_rows as u64) } else { @@ -268,43 +307,337 @@ impl PendingRowsBatcher { } } - async fn align_table_batches_to_region_schema( + /// Converts proto `RowInsertRequests` directly into aligned `RecordBatch`es + /// in a single pass, handling table creation, schema alteration, column + /// renaming, reordering, and null-filling without building intermediate + /// RecordBatches. + async fn build_and_align_table_batches( &self, - table_batches: Vec<(String, RecordBatch)>, + requests: RowInsertRequests, ctx: &QueryContextRef, - ) -> Result> { + ) -> Result<(Vec<(String, u32, RecordBatch)>, usize)> { let catalog = ctx.current_catalog().to_string(); let schema = ctx.current_schema(); - let mut region_schemas: HashMap> = HashMap::new(); - let mut aligned_batches = Vec::with_capacity(table_batches.len()); - for (table_name, record_batch) in table_batches { - let region_schema = if let Some(region_schema) = region_schemas.get(&table_name) { - region_schema.clone() + let (table_rows, total_rows) = Self::collect_non_empty_table_rows(requests); + if total_rows == 0 { + return Ok((Vec::new(), 0)); + } + + let unique_tables = Self::collect_unique_table_schemas(&table_rows)?; + let mut plan = self + .plan_table_resolution(&catalog, &schema, ctx, &unique_tables) + .await?; + + self.create_missing_tables_and_refresh_schemas( + &catalog, + &schema, + ctx, + &table_rows, + &mut plan, + ) + .await?; + + self.alter_tables_and_refresh_schemas(&catalog, &schema, ctx, &mut plan) + .await?; + + let aligned_batches = Self::build_aligned_batches(&table_rows, &plan.region_schemas)?; + + Ok((aligned_batches, total_rows)) + } + + /// Extracts non-empty `(table_name, rows)` pairs and computes total row + /// count across the retained entries. + fn collect_non_empty_table_rows(requests: RowInsertRequests) -> (Vec<(String, Rows)>, usize) { + let mut table_rows: Vec<(String, Rows)> = Vec::with_capacity(requests.inserts.len()); + let mut total_rows = 0; + + for request in requests.inserts { + let Some(rows) = request.rows else { + continue; + }; + if rows.rows.is_empty() { + continue; + } + + total_rows += rows.rows.len(); + table_rows.push((request.table_name, rows)); + } + + (table_rows, total_rows) + } + + /// Returns unique `(table_name, proto_schema)` pairs while keeping the + /// first-seen schema for duplicate table names. + fn collect_unique_table_schemas( + table_rows: &[(String, Rows)], + ) -> Result> { + let mut unique_tables: Vec<(&str, &[ColumnSchema])> = Vec::with_capacity(table_rows.len()); + let mut seen = HashSet::new(); + + for (table_name, rows) in table_rows { + if seen.insert(table_name.as_str()) { + unique_tables.push((table_name.as_str(), &rows.schema)); } else { - let table = self - .catalog_manager - .table(&catalog, &schema, &table_name, Some(ctx.as_ref())) - .await - .map_err(|err| Error::Internal { - err_msg: format!( - "Failed to resolve table {} for pending batch alignment: {}", - table_name, err - ), - })? - .ok_or_else(|| Error::Internal { - err_msg: format!( - "Table not found during pending batch alignment: {}", - table_name - ), - })?; - let region_schema = table.table_info().meta.schema.arrow_schema().clone(); - region_schemas.insert(table_name.clone(), region_schema.clone()); - region_schema + // table_rows should group rows by table name. + return error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Found duplicated table name in RowInsertRequest: {}", + table_name + ), + } + .fail(); + } + } + + Ok(unique_tables) + } + + /// Resolves table metadata and classifies each table into existing, + /// to-create, and to-alter groups used by subsequent DDL steps. + async fn plan_table_resolution( + &self, + catalog: &str, + schema: &str, + ctx: &QueryContextRef, + unique_tables: &[(&str, &[ColumnSchema])], + ) -> Result { + let mut plan = TableResolutionPlan { + region_schemas: HashMap::with_capacity(unique_tables.len()), + tables_to_create: Vec::new(), + tables_to_alter: Vec::new(), + }; + + let resolved_tables = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_resolve_table"]) + .start_timer(); + futures::future::join_all(unique_tables.iter().map(|(table_name, _)| { + self.catalog_manager + .table(catalog, schema, table_name, Some(ctx.as_ref())) + })) + .await + }; + + for ((table_name, rows_schema), table_result) in unique_tables.iter().zip(resolved_tables) { + let table = table_result?; + + if let Some(table) = table { + let table_info = table.table_info(); + let table_id = table_info.ident.table_id; + let region_schema = table_info.meta.schema.arrow_schema().clone(); + + let missing_columns = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_identify_missing_columns"]) + .start_timer(); + identify_missing_columns_from_proto(rows_schema, region_schema.as_ref())? + }; + if !missing_columns.is_empty() { + plan.tables_to_alter + .push(((*table_name).to_string(), missing_columns)); + } + plan.region_schemas + .insert((*table_name).to_string(), (region_schema, table_id)); + } else { + let request_schema = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_build_create_table_schema"]) + .start_timer(); + build_prom_create_table_schema_from_proto(rows_schema)? + }; + plan.tables_to_create + .push(((*table_name).to_string(), request_schema)); + } + } + + Ok(plan) + } + + /// Batch-creates missing tables, refreshes their schema metadata, and + /// enqueues follow-up alters for extra tag columns discovered in later rows. + async fn create_missing_tables_and_refresh_schemas( + &self, + catalog: &str, + schema: &str, + ctx: &QueryContextRef, + table_rows: &[(String, Rows)], + plan: &mut TableResolutionPlan, + ) -> Result<()> { + if plan.tables_to_create.is_empty() { + return Ok(()); + } + + let create_refs: Vec<(&str, &[ColumnSchema])> = plan + .tables_to_create + .iter() + .map(|(name, schema)| (name.as_str(), schema.as_slice())) + .collect(); + + { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_batch_create_tables"]) + .start_timer(); + self.schema_alterer + .create_tables_if_missing_batch( + catalog, + schema, + &create_refs, + self.prom_store_with_metric_engine, + ctx.clone(), + ) + .await?; + } + + let created_table_results = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_resolve_table_after_create"]) + .start_timer(); + futures::future::join_all(plan.tables_to_create.iter().map(|(table_name, _)| { + self.catalog_manager + .table(catalog, schema, table_name, Some(ctx.as_ref())) + })) + .await + }; + + for ((table_name, _), table_result) in + plan.tables_to_create.iter().zip(created_table_results) + { + let table = table_result?.with_context(|| error::UnexpectedResultSnafu { + reason: format!( + "Table not found after pending batch create attempt: {}", + table_name + ), + })?; + let table_info = table.table_info(); + let table_id = table_info.ident.table_id; + let region_schema = table_info.meta.schema.arrow_schema().clone(); + plan.region_schemas + .insert(table_name.clone(), (region_schema, table_id)); + } + + Self::enqueue_alter_for_new_tables(table_rows, plan)?; + + Ok(()) + } + + /// For newly created tables, re-checks all row schemas and appends alter + /// operations when additional tag columns are still missing. + fn enqueue_alter_for_new_tables( + table_rows: &[(String, Rows)], + plan: &mut TableResolutionPlan, + ) -> Result<()> { + let created_tables: HashSet<&str> = plan + .tables_to_create + .iter() + .map(|(table_name, _)| table_name.as_str()) + .collect(); + + for (table_name, rows) in table_rows { + if !created_tables.contains(table_name.as_str()) { + continue; + } + + let Some((region_schema, _)) = plan.region_schemas.get(table_name) else { + continue; }; - let record_batch = align_record_batch_to_schema(record_batch, region_schema.as_ref())?; - aligned_batches.push((table_name, record_batch)); + let missing_columns = identify_missing_columns_from_proto(&rows.schema, region_schema)?; + if missing_columns.is_empty() + || plan + .tables_to_alter + .iter() + .any(|(existing_name, _)| existing_name == table_name) + { + continue; + } + + plan.tables_to_alter + .push((table_name.clone(), missing_columns)); + } + + Ok(()) + } + + /// Batch-alters tables that have missing tag columns and refreshes the + /// in-memory schema map used for row alignment. + async fn alter_tables_and_refresh_schemas( + &self, + catalog: &str, + schema: &str, + ctx: &QueryContextRef, + plan: &mut TableResolutionPlan, + ) -> Result<()> { + if plan.tables_to_alter.is_empty() { + return Ok(()); + } + + let alter_refs: Vec<(&str, &[String])> = plan + .tables_to_alter + .iter() + .map(|(name, cols)| (name.as_str(), cols.as_slice())) + .collect(); + { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_batch_add_missing_columns"]) + .start_timer(); + self.schema_alterer + .add_missing_prom_tag_columns_batch(catalog, schema, &alter_refs, ctx.clone()) + .await?; + } + + let altered_table_results = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_resolve_table_after_schema_alter"]) + .start_timer(); + futures::future::join_all(plan.tables_to_alter.iter().map(|(table_name, _)| { + self.catalog_manager + .table(catalog, schema, table_name, Some(ctx.as_ref())) + })) + .await + }; + + for ((table_name, _), table_result) in + plan.tables_to_alter.iter().zip(altered_table_results) + { + let table = table_result?.with_context(|| error::UnexpectedResultSnafu { + reason: format!( + "Table not found after pending batch schema alter: {}", + table_name + ), + })?; + let table_info = table.table_info(); + let table_id = table_info.ident.table_id; + let refreshed_region_schema = table_info.meta.schema.arrow_schema().clone(); + plan.region_schemas + .insert(table_name.clone(), (refreshed_region_schema, table_id)); + } + + Ok(()) + } + + /// Converts proto rows to `RecordBatch` values aligned to resolved region + /// schemas and returns `(table_name, table_id, batch)` tuples. + fn build_aligned_batches( + table_rows: &[(String, Rows)], + region_schemas: &HashMap, u32)>, + ) -> Result> { + let mut aligned_batches = Vec::with_capacity(table_rows.len()); + for (table_name, rows) in table_rows { + let (region_schema, table_id) = + region_schemas.get(table_name).cloned().with_context(|| { + error::UnexpectedResultSnafu { + reason: format!("Region schema not resolved for table: {}", table_name), + } + })?; + + let record_batch = { + let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED + .with_label_values(&["align_rows_to_record_batch"]) + .start_timer(); + rows_to_aligned_record_batch(rows, region_schema.as_ref())? + }; + aligned_batches.push((table_name.clone(), table_id, record_batch)); } Ok(aligned_batches) @@ -422,9 +755,10 @@ fn start_worker( batch.waiters.push(FlushWaiter { response_tx, _permit }); - for (table_name, record_batch) in table_batches { + for (table_name, table_id, record_batch) in table_batches { let entry = batch.tables.entry(table_name.clone()).or_insert_with(|| TableBatch { table_name, + table_id, batches: Vec::new(), row_count: 0, }); @@ -579,6 +913,199 @@ async fn spawn_flush( } } +struct FlushRegionWrite { + region_id: RegionId, + row_count: usize, + datanode: Peer, + request: RegionRequest, +} + +enum FlushWriteResult { + Success { row_count: usize }, + Failed { row_count: usize, message: String }, +} + +fn should_dispatch_concurrently(region_write_count: usize) -> bool { + region_write_count > 1 +} + +/// Classifies columns in a logical-table batch for sparse primary-key conversion. +/// +/// Returns: +/// - `Vec`: all Utf8 tag columns sorted by tag name, used for +/// TSID and sparse primary-key encoding. +/// - `SmallVec<[usize; 3]>`: indices of columns copied into the physical batch +/// after `__primary_key`, ordered as `[greptime_timestamp, greptime_value, +/// partition_tag_columns...]`. +fn columns_taxonomy( + batch_schema: &Arc, + table_name: &str, + name_to_ids: &HashMap, + partition_columns: &HashSet<&str>, +) -> Result<(Vec, SmallVec<[usize; 3]>)> { + let mut tag_columns = Vec::new(); + let mut essential_column_indices = + SmallVec::<[usize; 3]>::with_capacity(2 + partition_columns.len()); + // Placeholder for greptime_timestamp and greptime_value + essential_column_indices.push(0); + essential_column_indices.push(0); + + let mut timestamp_index = None; + let mut value_index = None; + + for (index, field) in batch_schema.fields().iter().enumerate() { + match field.data_type() { + ArrowDataType::Utf8 => { + let column_id = name_to_ids.get(field.name()).copied().with_context(|| { + error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Column '{}' from logical table '{}' not found in physical table column IDs", + field.name(), + table_name + ), + } + })?; + tag_columns.push(TagColumnInfo { + name: field.name().clone(), + index, + column_id, + }); + + if partition_columns.contains(field.name().as_str()) { + essential_column_indices.push(index); + } + } + ArrowDataType::Timestamp(TimeUnit::Millisecond, _) => { + ensure!( + timestamp_index.replace(index).is_none(), + error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Duplicated timestamp column in logical table '{}' batch schema", + table_name + ), + } + ); + } + ArrowDataType::Float64 => { + ensure!( + value_index.replace(index).is_none(), + error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Duplicated value column in logical table '{}' batch schema", + table_name + ), + } + ); + } + datatype => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Unexpected data type '{datatype:?}' in logical table '{}' batch schema", + table_name + ), + } + .fail(); + } + } + } + + let timestamp_index = + timestamp_index.with_context(|| error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Missing essential column '{}' in logical table '{}' batch schema", + greptime_timestamp(), + table_name + ), + })?; + let value_index = value_index.with_context(|| error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Missing essential column '{}' in logical table '{}' batch schema", + greptime_value(), + table_name + ), + })?; + + tag_columns.sort_by(|a, b| a.name.cmp(&b.name)); + + essential_column_indices[0] = timestamp_index; + essential_column_indices[1] = value_index; + + Ok((tag_columns, essential_column_indices)) +} + +fn strip_partition_columns_from_batch(batch: RecordBatch) -> Result { + ensure!( + batch.num_columns() >= PHYSICAL_REGION_ESSENTIAL_COLUMN_COUNT, + error::InternalSnafu { + err_msg: format!( + "Expected at least {} columns in physical batch, got {}", + PHYSICAL_REGION_ESSENTIAL_COLUMN_COUNT, + batch.num_columns() + ), + } + ); + let essential_indices: Vec = (0..PHYSICAL_REGION_ESSENTIAL_COLUMN_COUNT).collect(); + batch + .project(&essential_indices) + .map_err(|err| Error::Internal { + err_msg: format!("Failed to project essential columns from RecordBatch: {err}"), + }) +} + +async fn flush_region_writes_concurrently( + node_manager: NodeManagerRef, + writes: Vec, +) -> Vec { + if !should_dispatch_concurrently(writes.len()) { + let mut results = Vec::with_capacity(writes.len()); + for write in writes { + let datanode = node_manager.datanode(&write.datanode).await; + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_write_region"]) + .start_timer(); + match datanode.handle(write.request).await { + Ok(_) => results.push(FlushWriteResult::Success { + row_count: write.row_count, + }), + Err(err) => results.push(FlushWriteResult::Failed { + row_count: write.row_count, + message: format!( + "Bulk insert flush failed for region {}: {:?}", + write.region_id, err + ), + }), + } + } + return results; + } + + let write_futures = writes.into_iter().map(|write| { + let node_manager = node_manager.clone(); + async move { + let datanode = node_manager.datanode(&write.datanode).await; + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_write_region"]) + .start_timer(); + + match datanode.handle(write.request).await { + Ok(_) => FlushWriteResult::Success { + row_count: write.row_count, + }, + Err(err) => FlushWriteResult::Failed { + row_count: write.row_count, + message: format!( + "Bulk insert flush failed for region {}: {:?}", + write.region_id, err + ), + }, + } + } + }); + + // todo(hl): should be bounded. + futures::future::join_all(write_futures).await +} + async fn flush_batch( flush: FlushBatch, partition_manager: PartitionRuleManagerRef, @@ -594,231 +1121,27 @@ async fn flush_batch( let start = Instant::now(); let mut first_error: Option = None; - let catalog = ctx.current_catalog().to_string(); - let schema = ctx.current_schema(); - - macro_rules! record_failure { - ($row_count:expr, $msg:expr) => {{ - let msg = $msg; - if first_error.is_none() { - first_error = Some(msg.clone()); - } - mark_flush_failure($row_count, &msg); - }}; - } - - for table_batch in table_batches { - let Some(first_batch) = table_batch.batches.first() else { - continue; - }; - - let schema_ref = first_batch.schema(); - let record_batch = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_concat_table_batches"]) - .start_timer(); - match concat_batches(&schema_ref, &table_batch.batches) { - Ok(batch) => batch, - Err(err) => { - record_failure!( - table_batch.row_count, - format!( - "Failed to concat table batch {}: {:?}", - table_batch.table_name, err - ) - ); - continue; - } - } - }; - - let table = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_resolve_table"]) - .start_timer(); - match catalog_manager - .table( - &catalog, - &schema, - &table_batch.table_name, - Some(ctx.as_ref()), - ) - .await - { - Ok(Some(table)) => table, - Ok(None) => { - record_failure!( - table_batch.row_count, - format!( - "Table not found during pending flush: {}", - table_batch.table_name - ) - ); - continue; - } - Err(err) => { - record_failure!( - table_batch.row_count, - format!( - "Failed to resolve table {} for pending flush: {:?}", - table_batch.table_name, err - ) - ); - continue; - } - } - }; - let table_info = table.table_info(); - - let partition_rule = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_fetch_partition_rule"]) - .start_timer(); - match partition_manager - .find_table_partition_rule(&table_info) - .await - { - Ok(rule) => rule, - Err(err) => { - record_failure!( - table_batch.row_count, - format!( - "Failed to fetch partition rule for table {}: {:?}", - table_batch.table_name, err - ) - ); - continue; - } - } - }; - - let region_masks = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_split_record_batch"]) - .start_timer(); - match partition_rule.0.split_record_batch(&record_batch) { - Ok(masks) => masks, - Err(err) => { - record_failure!( - table_batch.row_count, - format!( - "Failed to split record batch for table {}: {:?}", - table_batch.table_name, err - ) - ); - continue; - } - } - }; - - for (region_number, mask) in region_masks { - if mask.select_none() { - continue; - } - - let region_batch = if mask.select_all() { - record_batch.clone() - } else { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_filter_record_batch"]) - .start_timer(); - match filter_record_batch(&record_batch, mask.array()) { - Ok(batch) => batch, - Err(err) => { - record_failure!( - table_batch.row_count, - format!( - "Failed to filter record batch for table {}: {:?}", - table_batch.table_name, err - ) - ); - continue; - } - } - }; - - let row_count = region_batch.num_rows(); - if row_count == 0 { - continue; - } - - let region_id = RegionId::new(table_info.table_id(), region_number); - let datanode = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_resolve_region_leader"]) - .start_timer(); - match partition_manager.find_region_leader(region_id).await { - Ok(peer) => peer, - Err(err) => { - record_failure!( - row_count, - format!("Failed to resolve region leader {}: {:?}", region_id, err) - ); - continue; - } - } - }; - - let (schema_bytes, data_header, payload) = { - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_encode_ipc"]) - .start_timer(); - match record_batch_to_ipc(region_batch) { - Ok(encoded) => encoded, - Err(err) => { - record_failure!( - row_count, - format!( - "Failed to encode Arrow IPC for region {}: {:?}", - region_id, err - ) - ); - continue; - } - } - }; - - let request = RegionRequest { - header: Some(RegionRequestHeader { - tracing_context: TracingContext::from_current_span().to_w3c(), - ..Default::default() - }), - body: Some(region_request::Body::BulkInsert(BulkInsertRequest { - region_id: region_id.as_u64(), - partition_expr_version: None, - body: Some(bulk_insert_request::Body::ArrowIpc(ArrowIpc { - schema: schema_bytes, - data_header, - payload, - })), - })), - }; - - let datanode = node_manager.datanode(&datanode).await; - let _timer = PENDING_ROWS_BATCH_INGEST_STAGE_ELAPSED - .with_label_values(&["flush_write_region"]) - .start_timer(); - match datanode.handle(request).await { - Ok(_) => { - FLUSH_TOTAL.inc(); - FLUSH_ROWS.observe(row_count as f64); - } - Err(err) => { - record_failure!( - row_count, - format!( - "Bulk insert flush failed for region {}: {:?}", - region_id, err - ) - ); - } - } - } - } + // Physical-table-level flush: transform all logical table batches + // into physical format and write them together. + let physical_table_name = ctx + .extension(PHYSICAL_TABLE_KEY) + .unwrap_or(GREPTIME_PHYSICAL_TABLE) + .to_string(); + flush_batch_physical( + &table_batches, + total_row_count, + &physical_table_name, + &ctx, + &partition_manager, + &node_manager, + &catalog_manager, + &mut first_error, + ) + .await; let elapsed = start.elapsed().as_secs_f64(); FLUSH_ELAPSED.observe(elapsed); - info!( + debug!( "Pending rows batch flushed, total rows: {}, elapsed time: {}s", total_row_count, elapsed ); @@ -826,6 +1149,370 @@ async fn flush_batch( notify_waiters(waiters, &first_error); } +/// Attempts to flush all table batches by transforming them into the physical +/// table format (sparse primary key encoding) and writing directly to the +/// physical data regions. +/// +/// This is the only flush path. Any failure in resolving or transforming the +/// physical flush inputs is recorded as flush failure and reported to waiters. +#[allow(clippy::too_many_arguments)] +async fn flush_batch_physical( + table_batches: &[TableBatch], + total_row_count: usize, + physical_table_name: &str, + ctx: &QueryContextRef, + partition_manager: &PartitionRuleManagerRef, + node_manager: &NodeManagerRef, + catalog_manager: &CatalogManagerRef, + first_error: &mut Option, +) { + macro_rules! record_failure { + ($row_count:expr, $msg:expr) => {{ + let msg = $msg; + if first_error.is_none() { + *first_error = Some(msg.clone()); + } + mark_flush_failure($row_count, &msg); + }}; + } + + // 1. Resolve the physical table and get column ID mapping + let physical_table = { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_resolve_table"]) + .start_timer(); + match catalog_manager + .table( + ctx.current_catalog(), + &ctx.current_schema(), + physical_table_name, + Some(ctx.as_ref()), + ) + .await + { + Ok(Some(table)) => table, + Ok(None) => { + record_failure!( + total_row_count, + format!( + "Physical table '{}' not found during pending flush", + physical_table_name + ) + ); + return; + } + Err(err) => { + record_failure!( + total_row_count, + format!( + "Failed to resolve physical table '{}' for pending flush: {:?}", + physical_table_name, err + ) + ); + return; + } + } + }; + + let physical_table_info = physical_table.table_info(); + let name_to_ids = match physical_table_info.name_to_ids() { + Some(ids) => ids, + None => { + record_failure!( + total_row_count, + format!( + "Physical table '{}' has no column IDs for pending flush", + physical_table_name + ) + ); + return; + } + }; + + // 2. Get the physical table's partition rule (one lookup instead of N) + let partition_rule = { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_fetch_partition_rule"]) + .start_timer(); + match partition_manager + .find_table_partition_rule(&physical_table_info) + .await + { + Ok(rule) => rule, + Err(err) => { + record_failure!( + total_row_count, + format!( + "Failed to fetch partition rule for physical table '{}': {:?}", + physical_table_name, err + ) + ); + return; + } + } + }; + let partition_columns = partition_rule.0.partition_columns(); + let partition_columns_set: HashSet<&str> = + partition_columns.iter().map(String::as_str).collect(); + + // 3. Transform each logical table batch into physical format + let mut modified_batches: Vec = Vec::with_capacity(table_batches.len()); + let mut modified_row_count: usize = 0; + + let mut modify_elapsed = Duration::ZERO; + let mut columns_taxonomy_elapsed = Duration::ZERO; + + 'next_table: for table_batch in table_batches { + let table_id = table_batch.table_id; + + // Transform each chunk to physical format directly, avoiding an + // intermediate concat_batches per logical table. + for batch in &table_batch.batches { + // Identify tag columns and non-tag columns from the logical batch schema. + // Chunks within a table_batch may have different schemas if new tag columns + // are added between submits. + // In prom batches, Float64 = value, Timestamp = timestamp, Utf8 = tags. + let batch_schema = batch.schema(); + let start = Instant::now(); + let (tag_columns, essential_col_indices) = match columns_taxonomy( + &batch_schema, + &table_batch.table_name, + &name_to_ids, + &partition_columns_set, + ) { + Ok(columns) => columns, + Err(err) => { + warn!( + "Failed to resolve columns for logical table '{}': {:?}", + table_batch.table_name, err + ); + record_failure!(table_batch.row_count, err.to_string()); + continue 'next_table; + } + }; + + columns_taxonomy_elapsed += start.elapsed(); + if tag_columns.is_empty() && essential_col_indices.is_empty() { + continue; + } + + let modified = { + let start = Instant::now(); + match modify_batch_sparse( + batch.clone(), + table_id, + &tag_columns, + &essential_col_indices, + ) { + Ok(batch) => { + modify_elapsed += start.elapsed(); + batch + } + Err(err) => { + record_failure!( + table_batch.row_count, + format!( + "Failed to modify batch for logical table '{}': {:?}", + table_batch.table_name, err + ) + ); + continue 'next_table; + } + } + }; + + modified_row_count += modified.num_rows(); + modified_batches.push(modified); + } + } + + PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_modify_batch"]) + .observe(modify_elapsed.as_secs_f64()); + PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_columns_taxonomy"]) + .observe(columns_taxonomy_elapsed.as_secs_f64()); + + if modified_batches.is_empty() { + if first_error.is_none() { + record_failure!( + total_row_count, + format!( + "No batches can be transformed for physical table '{}' during pending flush", + physical_table_name + ) + ); + } + return; + } + + // 4. Concatenate all modified batches (all share the same physical schema) + let combined_batch = { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_concat_all"]) + .start_timer(); + let combined_schema = modified_batches[0].schema(); + // todo(hl): maybe limit max rows to concat. + match concat_batches(&combined_schema, &modified_batches) { + Ok(batch) => batch, + Err(err) => { + record_failure!( + modified_row_count, + format!("Failed to concat modified batches: {:?}", err) + ); + return; + } + } + }; + + // 5. Split by physical partition rule and send to regions + let physical_table_id = physical_table_info.table_id(); + let region_masks = { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_split_record_batch"]) + .start_timer(); + match partition_rule.0.split_record_batch(&combined_batch) { + Ok(masks) => masks, + Err(err) => { + record_failure!( + total_row_count, + format!( + "Failed to split combined batch for physical table '{}': {:?}", + physical_table_name, err + ) + ); + return; + } + } + }; + + let stripped_batch = if partition_columns.is_empty() { + combined_batch + } else { + // Strip partition columns before encoding and sending requests. + match strip_partition_columns_from_batch(combined_batch) { + Ok(batch) => batch, + Err(err) => { + record_failure!( + total_row_count, + format!( + "Failed to strip partition columns for physical table '{}': {:?}", + physical_table_name, err + ) + ); + return; + } + } + }; + + let mut region_writes = Vec::new(); + for (region_number, mask) in region_masks { + if mask.select_none() { + continue; + } + + let region_batch = if mask.select_all() { + stripped_batch.clone() + } else { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_filter_record_batch"]) + .start_timer(); + match filter_record_batch(&stripped_batch, mask.array()) { + Ok(batch) => batch, + Err(err) => { + record_failure!( + total_row_count, + format!( + "Failed to filter combined batch for physical table '{}': {:?}", + physical_table_name, err + ) + ); + continue; + } + } + }; + + let row_count = region_batch.num_rows(); + if row_count == 0 { + continue; + } + + let region_id = RegionId::new(physical_table_id, region_number); + let datanode = { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_resolve_region_leader"]) + .start_timer(); + match partition_manager.find_region_leader(region_id).await { + Ok(peer) => peer, + Err(err) => { + record_failure!( + row_count, + format!( + "Failed to resolve region leader for physical region {}: {:?}", + region_id, err + ) + ); + continue; + } + } + }; + + let (schema_bytes, data_header, payload) = { + let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED + .with_label_values(&["flush_physical_encode_ipc"]) + .start_timer(); + match record_batch_to_ipc(region_batch) { + Ok(encoded) => encoded, + Err(err) => { + record_failure!( + row_count, + format!( + "Failed to encode Arrow IPC for physical region {}: {:?}", + region_id, err + ) + ); + continue; + } + } + }; + + let request = RegionRequest { + header: Some(RegionRequestHeader { + tracing_context: TracingContext::from_current_span().to_w3c(), + ..Default::default() + }), + body: Some(region_request::Body::BulkInsert(BulkInsertRequest { + region_id: region_id.as_u64(), + partition_expr_version: None, + body: Some(bulk_insert_request::Body::ArrowIpc(ArrowIpc { + schema: schema_bytes, + data_header, + payload, + })), + })), + }; + + region_writes.push(FlushRegionWrite { + region_id, + row_count, + datanode, + request, + }); + } + + for result in flush_region_writes_concurrently(node_manager.clone(), region_writes).await { + match result { + FlushWriteResult::Success { row_count } => { + FLUSH_TOTAL.inc(); + FLUSH_ROWS.observe(row_count as f64); + } + FlushWriteResult::Failed { row_count, message } => { + record_failure!(row_count, message); + } + } + } +} + fn notify_waiters(waiters: Vec, first_error: &Option) { for waiter in waiters { let result = match first_error { @@ -865,192 +1552,6 @@ fn flush_with_error(batch: &mut PendingBatch, message: &str) { mark_flush_failure(row_count, message); } -fn build_table_batches(requests: RowInsertRequests) -> Result<(Vec<(String, RecordBatch)>, usize)> { - let mut table_batches = Vec::with_capacity(requests.inserts.len()); - let mut total_rows = 0; - - for request in requests.inserts { - let Some(rows) = request.rows else { - continue; - }; - if rows.rows.is_empty() { - continue; - } - - let record_batch = rows_to_record_batch(&rows)?; - total_rows += record_batch.num_rows(); - table_batches.push((request.table_name, record_batch)); - } - - Ok((table_batches, total_rows)) -} - -fn align_record_batch_to_schema( - record_batch: RecordBatch, - target_schema: &ArrowSchema, -) -> Result { - let source_schema = record_batch.schema(); - if source_schema.as_ref() == target_schema { - return Ok(record_batch); - } - - for source_field in source_schema.fields() { - if target_schema - .column_with_name(source_field.name()) - .is_none() - { - return Err(Error::Internal { - err_msg: format!( - "Failed to align record batch schema, column '{}' not found in target schema", - source_field.name() - ), - }); - } - } - - let row_count = record_batch.num_rows(); - let mut columns = Vec::with_capacity(target_schema.fields().len()); - for target_field in target_schema.fields() { - let column = if let Some((index, source_field)) = - source_schema.column_with_name(target_field.name()) - { - let source_column = record_batch.column(index).clone(); - if source_field.data_type() == target_field.data_type() { - source_column - } else { - cast(source_column.as_ref(), target_field.data_type()).map_err(|err| { - Error::Internal { - err_msg: format!( - "Failed to cast column '{}' to target type {:?}: {}", - target_field.name(), - target_field.data_type(), - err - ), - } - })? - } - } else { - new_null_array(target_field.data_type(), row_count) - }; - columns.push(column); - } - - RecordBatch::try_new(Arc::new(target_schema.clone()), columns).map_err(|err| Error::Internal { - err_msg: format!("Failed to build aligned record batch: {}", err), - }) -} - -fn rows_to_record_batch(rows: &Rows) -> Result { - let row_count = rows.rows.len(); - let column_count = rows.schema.len(); - - for (idx, row) in rows.rows.iter().enumerate() { - ensure!( - row.values.len() == column_count, - error::InternalSnafu { - err_msg: format!( - "Column count mismatch in row {}, expected {}, got {}", - idx, - column_count, - row.values.len() - ) - } - ); - } - - let mut fields = Vec::with_capacity(column_count); - let mut columns = Vec::with_capacity(column_count); - - for (idx, column_schema) in rows.schema.iter().enumerate() { - let datatype_wrapper = ColumnDataTypeWrapper::try_new( - column_schema.datatype, - column_schema.datatype_extension.clone(), - )?; - let data_type = ConcreteDataType::from(datatype_wrapper); - fields.push(Field::new( - column_schema.column_name.clone(), - data_type.as_arrow_type(), - true, - )); - columns.push(build_arrow_array( - rows, - idx, - &column_schema.column_name, - data_type.as_arrow_type(), - row_count, - )?); - } - - RecordBatch::try_new(Arc::new(ArrowSchema::new(fields)), columns).context(error::ArrowSnafu) -} - -fn build_arrow_array( - rows: &Rows, - col_idx: usize, - column_name: &String, - column_data_type: arrow::datatypes::DataType, - row_count: usize, -) -> Result { - macro_rules! build_array { - ($builder:expr, $( $pattern:pat => $value:expr ),+ $(,)?) => {{ - let mut builder = $builder; - for row in &rows.rows { - match row.values[col_idx].value_data.as_ref() { - $(Some($pattern) => builder.append_value($value),)+ - Some(v) => { - return error::InvalidPromRemoteRequestSnafu { - msg: format!("Unexpected value: {:?}", v), - } - .fail(); - } - None => builder.append_null(), - } - } - Arc::new(builder.finish()) as ArrayRef - }}; - } - - let array: ArrayRef = match column_data_type { - arrow::datatypes::DataType::Float64 => { - build_array!(Float64Builder::with_capacity(row_count), ValueData::F64Value(v) => *v) - } - arrow::datatypes::DataType::Utf8 => build_array!( - StringBuilder::with_capacity(row_count, 0), - ValueData::StringValue(v) => v - ), - arrow::datatypes::DataType::Timestamp(u, _) => match u { - TimeUnit::Second => build_array!( - TimestampSecondBuilder::with_capacity(row_count), - ValueData::TimestampSecondValue(v) => *v - ), - TimeUnit::Millisecond => build_array!( - TimestampMillisecondBuilder::with_capacity(row_count), - ValueData::TimestampMillisecondValue(v) => *v - ), - TimeUnit::Microsecond => build_array!( - TimestampMicrosecondBuilder::with_capacity(row_count), - ValueData::DatetimeValue(v) => *v, - ValueData::TimestampMicrosecondValue(v) => *v - ), - TimeUnit::Nanosecond => build_array!( - TimestampNanosecondBuilder::with_capacity(row_count), - ValueData::TimestampNanosecondValue(v) => *v - ), - }, - ty => { - return error::InvalidPromRemoteRequestSnafu { - msg: format!( - "Unexpected column type {:?}, column name: {}", - ty, column_name - ), - } - .fail(); - } - }; - - Ok(array) -} - fn record_batch_to_ipc(record_batch: RecordBatch) -> Result<(Bytes, Bytes, Bytes)> { let mut encoder = FlightEncoder::default(); let schema = encoder.encode_schema(record_batch.schema().as_ref()); @@ -1077,132 +1578,154 @@ fn record_batch_to_ipc(record_batch: RecordBatch) -> Result<(Bytes, Bytes, Bytes #[cfg(test)] mod tests { + use std::collections::{HashMap, HashSet}; use std::sync::Arc; + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::time::Duration; - use api::v1::value::ValueData; - use api::v1::{ColumnDataType, ColumnSchema, Row, Rows, SemanticType, Value}; - use arrow::array::{Array, Float64Array, Int32Array, Int64Array, StringArray}; - use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; + use api::region::RegionResponse; + use api::v1::flow::{DirtyWindowRequests, FlowRequest, FlowResponse}; + use api::v1::meta::Peer; + use api::v1::region::{InsertRequests, RegionRequest}; + use api::v1::{ColumnSchema, Row, RowInsertRequest, RowInsertRequests, Rows}; + use arrow::array::{BinaryArray, StringArray, TimestampMillisecondArray}; + use arrow::datatypes::{DataType as ArrowDataType, Field, Schema as ArrowSchema}; use arrow::record_batch::RecordBatch; + use async_trait::async_trait; + use common_meta::error::Result as MetaResult; + use common_meta::node_manager::{ + Datanode, DatanodeManager, DatanodeRef, Flownode, FlownodeManager, FlownodeRef, + }; + use common_query::request::QueryRequest; + use common_recordbatch::SendableRecordBatchStream; use dashmap::DashMap; + use smallvec::SmallVec; + use store_api::storage::RegionId; use tokio::sync::mpsc; + use tokio::time::sleep; use super::{ - BatchKey, PendingWorker, WorkerCommand, align_record_batch_to_schema, - remove_worker_if_same_channel, rows_to_record_batch, should_close_worker_on_idle_timeout, + BatchKey, Error, FlushRegionWrite, FlushWriteResult, PendingRowsBatcher, PendingWorker, + WorkerCommand, columns_taxonomy, flush_region_writes_concurrently, + remove_worker_if_same_channel, should_close_worker_on_idle_timeout, + should_dispatch_concurrently, strip_partition_columns_from_batch, }; + fn mock_rows(row_count: usize, schema_name: &str) -> Rows { + Rows { + schema: vec![ColumnSchema { + column_name: schema_name.to_string(), + ..Default::default() + }], + rows: (0..row_count).map(|_| Row { values: vec![] }).collect(), + } + } + #[test] - fn test_rows_to_record_batch() { - let rows = Rows { - schema: vec![ - ColumnSchema { - column_name: "ts".to_string(), - datatype: ColumnDataType::TimestampMillisecond as i32, - semantic_type: SemanticType::Timestamp as i32, - ..Default::default() + fn test_collect_non_empty_table_rows_filters_empty_payloads() { + let requests = RowInsertRequests { + inserts: vec![ + RowInsertRequest { + table_name: "cpu".to_string(), + rows: Some(mock_rows(2, "host")), }, - ColumnSchema { - column_name: "value".to_string(), - datatype: ColumnDataType::Float64 as i32, - semantic_type: SemanticType::Field as i32, - ..Default::default() + RowInsertRequest { + table_name: "mem".to_string(), + rows: Some(mock_rows(0, "host")), }, - ColumnSchema { - column_name: "host".to_string(), - datatype: ColumnDataType::String as i32, - semantic_type: SemanticType::Tag as i32, - ..Default::default() - }, - ], - rows: vec![ - Row { - values: vec![ - Value { - value_data: Some(ValueData::TimestampMillisecondValue(1000)), - }, - Value { - value_data: Some(ValueData::F64Value(42.0)), - }, - Value { - value_data: Some(ValueData::StringValue("h1".to_string())), - }, - ], - }, - Row { - values: vec![ - Value { - value_data: Some(ValueData::TimestampMillisecondValue(2000)), - }, - Value { value_data: None }, - Value { - value_data: Some(ValueData::StringValue("h2".to_string())), - }, - ], + RowInsertRequest { + table_name: "disk".to_string(), + rows: None, }, ], }; - let rb = rows_to_record_batch(&rows).unwrap(); - assert_eq!(2, rb.num_rows()); - assert_eq!(3, rb.num_columns()); + let (table_rows, total_rows) = PendingRowsBatcher::collect_non_empty_table_rows(requests); + + assert_eq!(2, total_rows); + assert_eq!(1, table_rows.len()); + assert_eq!("cpu", table_rows[0].0); + assert_eq!(2, table_rows[0].1.rows.len()); } - #[test] - fn test_align_record_batch_to_schema_reorder_and_fill_missing() { - let source_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("host", DataType::Utf8, true), - Field::new("value", DataType::Float64, true), - ])); - let source = RecordBatch::try_new( - source_schema, - vec![ - Arc::new(StringArray::from(vec!["h1"])), - Arc::new(Float64Array::from(vec![42.0])), - ], - ) - .unwrap(); - - let target = ArrowSchema::new(vec![ - Field::new("ts", DataType::Int64, true), - Field::new("host", DataType::Utf8, true), - Field::new("value", DataType::Float64, true), - ]); - - let aligned = align_record_batch_to_schema(source, &target).unwrap(); - assert_eq!(aligned.schema().as_ref(), &target); - assert_eq!(1, aligned.num_rows()); - assert_eq!(3, aligned.num_columns()); - let ts = aligned - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert!(ts.is_null(0)); + #[derive(Clone)] + struct ConcurrentMockDatanode { + delay: Duration, + inflight: Arc, + max_inflight: Arc, } - #[test] - fn test_align_record_batch_to_schema_cast_column_type() { - let source_schema = Arc::new(ArrowSchema::new(vec![Field::new( - "value", - DataType::Int32, - true, - )])); - let source = RecordBatch::try_new( - source_schema, - vec![Arc::new(Int32Array::from(vec![Some(7), None]))], - ) - .unwrap(); + #[async_trait] + impl Datanode for ConcurrentMockDatanode { + async fn handle(&self, _request: RegionRequest) -> MetaResult { + let now = self.inflight.fetch_add(1, Ordering::SeqCst) + 1; + loop { + let max = self.max_inflight.load(Ordering::SeqCst); + if now <= max { + break; + } + if self + .max_inflight + .compare_exchange(max, now, Ordering::SeqCst, Ordering::SeqCst) + .is_ok() + { + break; + } + } - let target = ArrowSchema::new(vec![Field::new("value", DataType::Int64, true)]); - let aligned = align_record_batch_to_schema(source, &target).unwrap(); - let value = aligned - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(Some(7), value.iter().next().flatten()); - assert!(value.is_null(1)); + sleep(self.delay).await; + self.inflight.fetch_sub(1, Ordering::SeqCst); + Ok(RegionResponse::new(0)) + } + + async fn handle_query( + &self, + _request: QueryRequest, + ) -> MetaResult { + unimplemented!() + } + } + + #[derive(Clone)] + struct ConcurrentMockNodeManager { + datanodes: Arc>, + } + + #[async_trait] + impl DatanodeManager for ConcurrentMockNodeManager { + async fn datanode(&self, node: &Peer) -> DatanodeRef { + self.datanodes + .get(&node.id) + .expect("datanode not found") + .clone() + } + } + + struct NoopFlownode; + + #[async_trait] + impl Flownode for NoopFlownode { + async fn handle(&self, _request: FlowRequest) -> MetaResult { + unimplemented!() + } + + async fn handle_inserts(&self, _request: InsertRequests) -> MetaResult { + unimplemented!() + } + + async fn handle_mark_window_dirty( + &self, + _req: DirtyWindowRequests, + ) -> MetaResult { + unimplemented!() + } + } + + #[async_trait] + impl FlownodeManager for ConcurrentMockNodeManager { + async fn flownode(&self, _node: &Peer) -> FlownodeRef { + Arc::new(NoopFlownode) + } } #[test] @@ -1250,4 +1773,339 @@ mod tests { assert!(!should_close_worker_on_idle_timeout(1, 0)); assert!(!should_close_worker_on_idle_timeout(0, 1)); } + + #[tokio::test] + async fn test_flush_region_writes_concurrently_dispatches_multiple_datanodes() { + let inflight = Arc::new(AtomicUsize::new(0)); + let max_inflight = Arc::new(AtomicUsize::new(0)); + let datanode1: DatanodeRef = Arc::new(ConcurrentMockDatanode { + delay: Duration::from_millis(100), + inflight: inflight.clone(), + max_inflight: max_inflight.clone(), + }); + let datanode2: DatanodeRef = Arc::new(ConcurrentMockDatanode { + delay: Duration::from_millis(100), + inflight, + max_inflight: max_inflight.clone(), + }); + + let mut datanodes = HashMap::new(); + datanodes.insert(1, datanode1); + datanodes.insert(2, datanode2); + let node_manager = Arc::new(ConcurrentMockNodeManager { + datanodes: Arc::new(datanodes), + }); + + let writes = vec![ + FlushRegionWrite { + region_id: RegionId::new(1024, 1), + row_count: 10, + datanode: Peer { + id: 1, + addr: "node1".to_string(), + }, + request: RegionRequest::default(), + }, + FlushRegionWrite { + region_id: RegionId::new(1024, 2), + row_count: 12, + datanode: Peer { + id: 2, + addr: "node2".to_string(), + }, + request: RegionRequest::default(), + }, + ]; + + let results = flush_region_writes_concurrently(node_manager, writes).await; + assert_eq!(2, results.len()); + assert!( + results + .iter() + .all(|result| matches!(result, FlushWriteResult::Success { .. })) + ); + assert!(max_inflight.load(Ordering::SeqCst) >= 2); + } + + #[test] + fn test_should_dispatch_concurrently_by_region_count() { + assert!(!should_dispatch_concurrently(0)); + assert!(!should_dispatch_concurrently(1)); + assert!(should_dispatch_concurrently(2)); + } + + #[test] + fn test_strip_partition_columns_from_batch_removes_partition_tags() { + let batch = RecordBatch::try_new( + Arc::new(ArrowSchema::new(vec![ + Field::new("__primary_key", ArrowDataType::Binary, false), + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + ])), + vec![ + Arc::new(BinaryArray::from(vec![b"k1".as_slice()])), + Arc::new(TimestampMillisecondArray::from(vec![1000_i64])), + Arc::new(arrow::array::Float64Array::from(vec![42.0_f64])), + Arc::new(StringArray::from(vec!["node-1"])), + ], + ) + .unwrap(); + + let stripped = strip_partition_columns_from_batch(batch).unwrap(); + + assert_eq!(3, stripped.num_columns()); + assert_eq!("__primary_key", stripped.schema().field(0).name()); + assert_eq!("greptime_timestamp", stripped.schema().field(1).name()); + assert_eq!("greptime_value", stripped.schema().field(2).name()); + } + + #[test] + fn test_strip_partition_columns_from_batch_projects_essential_columns_without_lookup() { + let batch = RecordBatch::try_new( + Arc::new(ArrowSchema::new(vec![ + Field::new("__primary_key", ArrowDataType::Binary, false), + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + ])), + vec![ + Arc::new(BinaryArray::from(vec![b"k1".as_slice()])), + Arc::new(TimestampMillisecondArray::from(vec![1000_i64])), + Arc::new(arrow::array::Float64Array::from(vec![42.0_f64])), + Arc::new(StringArray::from(vec!["node-1"])), + ], + ) + .unwrap(); + + let stripped = strip_partition_columns_from_batch(batch).unwrap(); + + assert_eq!(3, stripped.num_columns()); + assert_eq!("__primary_key", stripped.schema().field(0).name()); + assert_eq!("greptime_timestamp", stripped.schema().field(1).name()); + assert_eq!("greptime_value", stripped.schema().field(2).name()); + } + + #[test] + fn test_collect_tag_columns_and_non_tag_indices_keeps_partition_tag_column() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + Field::new("region", ArrowDataType::Utf8, true), + ])); + let name_to_ids = + HashMap::from([("host".to_string(), 1_u32), ("region".to_string(), 2_u32)]); + let partition_columns = HashSet::from(["host"]); + + let (tag_columns, non_tag_indices) = + columns_taxonomy(&schema, "cpu", &name_to_ids, &partition_columns).unwrap(); + + assert_eq!(2, tag_columns.len()); + assert_eq!(&[0, 1, 2], non_tag_indices.as_slice()); + } + + #[test] + fn test_collect_tag_columns_and_non_tag_indices_prioritizes_essential_columns() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("host", ArrowDataType::Utf8, true), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("region", ArrowDataType::Utf8, true), + ])); + let name_to_ids = + HashMap::from([("host".to_string(), 1_u32), ("region".to_string(), 2_u32)]); + let partition_columns = HashSet::from(["host", "region"]); + + let (_tag_columns, non_tag_indices): (_, SmallVec<[usize; 3]>) = + columns_taxonomy(&schema, "cpu", &name_to_ids, &partition_columns).unwrap(); + + assert_eq!(&[2, 1, 0, 3], non_tag_indices.as_slice()); + } + + #[test] + fn test_collect_tag_columns_and_non_tag_indices_rejects_unexpected_data_type() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + Field::new("invalid", ArrowDataType::Boolean, true), + ])); + let name_to_ids = HashMap::from([("host".to_string(), 1_u32)]); + let partition_columns = HashSet::from(["host"]); + + let result = columns_taxonomy(&schema, "cpu", &name_to_ids, &partition_columns); + + assert!(matches!( + result, + Err(Error::InvalidPromRemoteRequest { .. }) + )); + } + + #[test] + fn test_collect_tag_columns_and_non_tag_indices_rejects_int64_timestamp_column() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("greptime_timestamp", ArrowDataType::Int64, false), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + ])); + let name_to_ids = HashMap::from([("host".to_string(), 1_u32)]); + let partition_columns = HashSet::from(["host"]); + + let result = columns_taxonomy(&schema, "cpu", &name_to_ids, &partition_columns); + + assert!(matches!( + result, + Err(Error::InvalidPromRemoteRequest { .. }) + )); + } + + #[test] + fn test_collect_tag_columns_and_non_tag_indices_rejects_duplicated_timestamp_column() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new( + "ts1", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new( + "ts2", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + ])); + let name_to_ids = HashMap::from([("host".to_string(), 1_u32)]); + let partition_columns = HashSet::from(["host"]); + + let result = columns_taxonomy(&schema, "cpu", &name_to_ids, &partition_columns); + + assert!(matches!( + result, + Err(Error::InvalidPromRemoteRequest { .. }) + )); + } + + #[test] + fn test_collect_tag_columns_and_non_tag_indices_rejects_duplicated_value_column() { + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("value1", ArrowDataType::Float64, true), + Field::new("value2", ArrowDataType::Float64, true), + Field::new("host", ArrowDataType::Utf8, true), + ])); + let name_to_ids = HashMap::from([("host".to_string(), 1_u32)]); + let partition_columns = HashSet::from(["host"]); + + let result = columns_taxonomy(&schema, "cpu", &name_to_ids, &partition_columns); + + assert!(matches!( + result, + Err(Error::InvalidPromRemoteRequest { .. }) + )); + } + + #[test] + fn test_modify_batch_sparse_with_taxonomy_per_batch() { + use arrow::array::BinaryArray; + use metric_engine::batch_modifier::modify_batch_sparse; + + let schema1 = Arc::new(ArrowSchema::new(vec![ + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("tag1", ArrowDataType::Utf8, true), + ])); + + let schema2 = Arc::new(ArrowSchema::new(vec![ + Field::new( + "greptime_timestamp", + ArrowDataType::Timestamp(arrow::datatypes::TimeUnit::Millisecond, None), + false, + ), + Field::new("greptime_value", ArrowDataType::Float64, true), + Field::new("tag1", ArrowDataType::Utf8, true), + Field::new("tag2", ArrowDataType::Utf8, true), + ])); + let batch2 = RecordBatch::try_new( + schema2.clone(), + vec![ + Arc::new(TimestampMillisecondArray::from(vec![2000])), + Arc::new(arrow::array::Float64Array::from(vec![2.0])), + Arc::new(StringArray::from(vec!["v1"])), + Arc::new(StringArray::from(vec!["v2"])), + ], + ) + .unwrap(); + + let name_to_ids = HashMap::from([("tag1".to_string(), 1), ("tag2".to_string(), 2)]); + let partition_columns = HashSet::new(); + + // A batch that only has tag1, same values as batch2 for ts and val. + let batch3 = RecordBatch::try_new( + schema1.clone(), + vec![ + Arc::new(TimestampMillisecondArray::from(vec![2000])), + Arc::new(arrow::array::Float64Array::from(vec![2.0])), + Arc::new(StringArray::from(vec!["v1"])), + ], + ) + .unwrap(); + + // Simulate the new loop logic in flush_batch_physical: + // Resolve taxonomy FOR EACH BATCH. + let (tag_columns2, indices2) = + columns_taxonomy(&batch2.schema(), "table", &name_to_ids, &partition_columns).unwrap(); + let modified2 = modify_batch_sparse(batch2, 123, &tag_columns2, &indices2).unwrap(); + + let (tag_columns3, indices3) = + columns_taxonomy(&batch3.schema(), "table", &name_to_ids, &partition_columns).unwrap(); + let modified3 = modify_batch_sparse(batch3, 123, &tag_columns3, &indices3).unwrap(); + + let pk2 = modified2 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let pk3 = modified3 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + + // Now they SHOULD be different because tag2 is included in pk2 but not in pk3. + assert_ne!( + pk2.value(0), + pk3.value(0), + "PK should be different because batch2 has tag2!" + ); + } } diff --git a/src/servers/src/pipeline.rs b/src/servers/src/pipeline.rs index fe7a2f48f3..9de2d63b97 100644 --- a/src/servers/src/pipeline.rs +++ b/src/servers/src/pipeline.rs @@ -28,7 +28,7 @@ use session::context::{Channel, QueryContextRef}; use snafu::ResultExt; use vrl::value::Value as VrlValue; -use crate::error::{CatalogSnafu, PipelineSnafu, Result}; +use crate::error::{PipelineSnafu, Result}; use crate::http::event::PipelineIngestRequest; use crate::metrics::{ METRIC_FAILURE_VALUE, METRIC_HTTP_LOGS_TRANSFORM_ELAPSED, METRIC_SUCCESS_VALUE, @@ -89,10 +89,7 @@ async fn run_identity_pipeline( let table = if pipeline_ctx.channel == Channel::Prometheus { None } else { - handler - .get_table(&table_name, query_ctx) - .await - .context(CatalogSnafu)? + handler.get_table(&table_name, query_ctx).await? }; identity_pipeline(data_array, table, pipeline_ctx) .map(|opt_map| ContextReq::from_opt_map(opt_map, table_name)) @@ -141,10 +138,7 @@ async fn run_custom_pipeline( } }; - let table = handler - .get_table(&table_name, query_ctx) - .await - .context(CatalogSnafu)?; + let table = handler.get_table(&table_name, query_ctx).await?; schema_info.set_table(table); for pipeline_map in pipeline_maps { diff --git a/src/servers/src/prom_row_builder.rs b/src/servers/src/prom_row_builder.rs new file mode 100644 index 0000000000..0fddc0938a --- /dev/null +++ b/src/servers/src/prom_row_builder.rs @@ -0,0 +1,557 @@ +// 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. + +//! Prometheus row-level helpers for converting proto `Rows` into Arrow +//! `RecordBatch`es and aligning / normalizing their schemas against +//! existing table schemas in the catalog. + +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use api::helper::ColumnDataTypeWrapper; +use api::v1::value::ValueData; +use api::v1::{ColumnSchema, Rows, SemanticType}; +use arrow::array::{ + ArrayRef, Float64Builder, StringBuilder, TimestampMicrosecondBuilder, + TimestampMillisecondBuilder, TimestampNanosecondBuilder, TimestampSecondBuilder, + new_null_array, +}; +use arrow::datatypes::{DataType as ArrowDataType, Schema as ArrowSchema}; +use arrow::record_batch::RecordBatch; +use arrow_schema::TimeUnit; +use common_query::prelude::{greptime_timestamp, greptime_value}; +use datatypes::data_type::DataType; +use datatypes::prelude::ConcreteDataType; +use snafu::{OptionExt, ResultExt, ensure}; + +use crate::error; +use crate::error::Result; + +/// Extract timestamp, field, and tag column names from a logical region schema. +fn unzip_logical_region_schema( + target_schema: &ArrowSchema, +) -> Result<(String, String, HashSet)> { + let mut timestamp_column = None; + let mut field_column = None; + let mut tag_columns = HashSet::with_capacity(target_schema.fields.len().saturating_sub(2)); + for field in target_schema.fields() { + if field.name() == greptime_timestamp() { + timestamp_column = Some(field.name().clone()); + continue; + } + + if field.name() == greptime_value() { + field_column = Some(field.name().clone()); + continue; + } + + if timestamp_column.is_none() && matches!(field.data_type(), ArrowDataType::Timestamp(_, _)) + { + timestamp_column = Some(field.name().clone()); + continue; + } + + if field_column.is_none() && matches!(field.data_type(), ArrowDataType::Float64) { + field_column = Some(field.name().clone()); + continue; + } + tag_columns.insert(field.name().clone()); + } + + let timestamp_column = timestamp_column.with_context(|| error::UnexpectedResultSnafu { + reason: "Failed to locate timestamp column in target schema".to_string(), + })?; + let field_column = field_column.with_context(|| error::UnexpectedResultSnafu { + reason: "Failed to locate field column in target schema".to_string(), + })?; + + Ok((timestamp_column, field_column, tag_columns)) +} + +/// Directly converts proto `Rows` into a `RecordBatch` aligned to the given +/// `target_schema`, handling Prometheus column renaming (timestamp/value), +/// reordering, type casting, and null-filling in a single pass. +pub(crate) fn rows_to_aligned_record_batch( + rows: &Rows, + target_schema: &ArrowSchema, +) -> Result { + let row_count = rows.rows.len(); + let column_count = rows.schema.len(); + + for (idx, row) in rows.rows.iter().enumerate() { + ensure!( + row.values.len() == column_count, + error::InternalSnafu { + err_msg: format!( + "Column count mismatch in row {}, expected {}, got {}", + idx, + column_count, + row.values.len() + ) + } + ); + } + + let (target_ts_name, target_field_name, _target_tags) = + unzip_logical_region_schema(target_schema)?; + + // Map effective target column name → (source column index, source arrow type). + // Handles prom renames: Timestamp → target ts name, Float64 → target field name. + let mut source_map: HashMap<&str, (usize, ArrowDataType)> = + HashMap::with_capacity(rows.schema.len()); + + for (src_idx, col) in rows.schema.iter().enumerate() { + let wrapper = ColumnDataTypeWrapper::try_new(col.datatype, col.datatype_extension.clone())?; + let src_arrow_type = ConcreteDataType::from(wrapper).as_arrow_type(); + + match &src_arrow_type { + ArrowDataType::Float64 => { + source_map.insert(&target_field_name, (src_idx, src_arrow_type)); + } + ArrowDataType::Timestamp(unit, _) => { + ensure!( + unit == &TimeUnit::Millisecond, + error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Unexpected remote write batch timestamp unit, expect millisecond, got: {}", + unit + ) + } + ); + source_map.insert(&target_ts_name, (src_idx, src_arrow_type)); + } + ArrowDataType::Utf8 => { + source_map.insert(&col.column_name, (src_idx, src_arrow_type)); + } + other => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Unexpected remote write batch field type {}, field name: {}", + other, col.column_name + ), + } + .fail(); + } + } + } + + // Build columns in target schema order + let mut columns = Vec::with_capacity(target_schema.fields().len()); + for target_field in target_schema.fields() { + if let Some((src_idx, src_arrow_type)) = source_map.get(target_field.name().as_str()) { + let array = build_arrow_array( + rows, + *src_idx, + &rows.schema[*src_idx].column_name, + src_arrow_type.clone(), + row_count, + )?; + columns.push(array); + } else { + columns.push(new_null_array(target_field.data_type(), row_count)); + } + } + + let batch = RecordBatch::try_new(Arc::new(target_schema.clone()), columns) + .context(error::ArrowSnafu)?; + Ok(batch) +} + +/// Identify tag columns in the proto `rows_schema` that are absent from the +/// target region schema, without building an intermediate `RecordBatch`. +pub(crate) fn identify_missing_columns_from_proto( + rows_schema: &[ColumnSchema], + target_schema: &ArrowSchema, +) -> Result> { + let (_, _, target_tags) = unzip_logical_region_schema(target_schema)?; + let mut missing = Vec::new(); + for col in rows_schema { + let wrapper = ColumnDataTypeWrapper::try_new(col.datatype, col.datatype_extension.clone())?; + let arrow_type = ConcreteDataType::from(wrapper).as_arrow_type(); + if matches!(arrow_type, ArrowDataType::Utf8) + && !target_tags.contains(&col.column_name) + && target_schema.column_with_name(&col.column_name).is_none() + { + missing.push(col.column_name.clone()); + } + } + Ok(missing) +} + +/// Build a `Vec` suitable for creating a new Prometheus logical table +/// directly from the proto `rows.schema`, avoiding the round-trip through Arrow schema. +pub fn build_prom_create_table_schema_from_proto( + rows_schema: &[ColumnSchema], +) -> Result> { + rows_schema + .iter() + .map(|col| { + let semantic_type = if col.datatype == api::v1::ColumnDataType::TimestampMillisecond as i32 { + SemanticType::Timestamp + } else if col.datatype == api::v1::ColumnDataType::Float64 as i32 { + SemanticType::Field + } else { + // tag columns must be String type + ensure!(col.datatype == api::v1::ColumnDataType::String as i32, error::InvalidPromRemoteRequestSnafu{ + msg: format!( + "Failed to build create table schema, tag column '{}' must be String but got datatype {}", + col.column_name, col.datatype + ) + }); + SemanticType::Tag + }; + + Ok(ColumnSchema { + column_name: col.column_name.clone(), + datatype: col.datatype, + semantic_type: semantic_type as i32, + datatype_extension: col.datatype_extension.clone(), + options: None, + }) + }) + .collect() +} + +/// Build a single Arrow array for the given column index from proto `Rows`. +fn build_arrow_array( + rows: &Rows, + col_idx: usize, + column_name: &String, + column_data_type: arrow::datatypes::DataType, + row_count: usize, +) -> Result { + macro_rules! build_array { + ($builder:expr, $( $pattern:pat => $value:expr ),+ $(,)?) => {{ + let mut builder = $builder; + for row in &rows.rows { + match row.values[col_idx].value_data.as_ref() { + $(Some($pattern) => builder.append_value($value),)+ + Some(v) => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!("Unexpected value: {:?}", v), + } + .fail(); + } + None => builder.append_null(), + } + } + Arc::new(builder.finish()) as ArrayRef + }}; + } + + let array: ArrayRef = match column_data_type { + arrow::datatypes::DataType::Float64 => { + build_array!(Float64Builder::with_capacity(row_count), ValueData::F64Value(v) => *v) + } + arrow::datatypes::DataType::Utf8 => build_array!( + StringBuilder::with_capacity(row_count, 0), + ValueData::StringValue(v) => v + ), + arrow::datatypes::DataType::Timestamp(u, _) => match u { + TimeUnit::Second => build_array!( + TimestampSecondBuilder::with_capacity(row_count), + ValueData::TimestampSecondValue(v) => *v + ), + TimeUnit::Millisecond => build_array!( + TimestampMillisecondBuilder::with_capacity(row_count), + ValueData::TimestampMillisecondValue(v) => *v + ), + TimeUnit::Microsecond => build_array!( + TimestampMicrosecondBuilder::with_capacity(row_count), + ValueData::DatetimeValue(v) => *v, + ValueData::TimestampMicrosecondValue(v) => *v + ), + TimeUnit::Nanosecond => build_array!( + TimestampNanosecondBuilder::with_capacity(row_count), + ValueData::TimestampNanosecondValue(v) => *v + ), + }, + ty => { + return error::InvalidPromRemoteRequestSnafu { + msg: format!( + "Unexpected column type {:?}, column name: {}", + ty, column_name + ), + } + .fail(); + } + }; + + Ok(array) +} + +#[cfg(test)] +mod tests { + use api::v1::value::ValueData; + use api::v1::{ColumnDataType, ColumnSchema, Row, Rows, SemanticType, Value}; + use arrow::array::{Array, Float64Array, StringArray, TimestampMillisecondArray}; + use arrow::datatypes::{DataType, Field, Schema as ArrowSchema, TimeUnit}; + + use super::{ + build_prom_create_table_schema_from_proto, identify_missing_columns_from_proto, + rows_to_aligned_record_batch, + }; + + #[test] + fn test_rows_to_aligned_record_batch_renames_and_reorders() { + let rows = Rows { + schema: vec![ + ColumnSchema { + column_name: "greptime_timestamp".to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "host".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "greptime_value".to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as i32, + ..Default::default() + }, + ], + rows: vec![ + Row { + values: vec![ + Value { + value_data: Some(ValueData::TimestampMillisecondValue(1000)), + }, + Value { + value_data: Some(ValueData::StringValue("h1".to_string())), + }, + Value { + value_data: Some(ValueData::F64Value(42.0)), + }, + ], + }, + Row { + values: vec![ + Value { + value_data: Some(ValueData::TimestampMillisecondValue(2000)), + }, + Value { + value_data: Some(ValueData::StringValue("h2".to_string())), + }, + Value { + value_data: Some(ValueData::F64Value(99.0)), + }, + ], + }, + ], + }; + + // Target schema has renamed columns and different ordering. + let target = ArrowSchema::new(vec![ + Field::new( + "my_ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + Field::new("host", DataType::Utf8, true), + Field::new("my_value", DataType::Float64, true), + ]); + + let batch = rows_to_aligned_record_batch(&rows, &target).unwrap(); + assert_eq!(batch.schema().as_ref(), &target); + assert_eq!(2, batch.num_rows()); + assert_eq!(3, batch.num_columns()); + + let ts = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(ts.value(0), 1000); + assert_eq!(ts.value(1), 2000); + + let hosts = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(hosts.value(0), "h1"); + assert_eq!(hosts.value(1), "h2"); + + let values = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(values.value(0), 42.0); + assert_eq!(values.value(1), 99.0); + } + + #[test] + fn test_rows_to_aligned_record_batch_fills_nulls() { + let rows = Rows { + schema: vec![ + ColumnSchema { + column_name: "greptime_timestamp".to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "host".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "instance".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "greptime_value".to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as i32, + ..Default::default() + }, + ], + rows: vec![Row { + values: vec![ + Value { + value_data: Some(ValueData::TimestampMillisecondValue(1000)), + }, + Value { + value_data: Some(ValueData::StringValue("h1".to_string())), + }, + Value { + value_data: Some(ValueData::StringValue("i1".to_string())), + }, + Value { + value_data: Some(ValueData::F64Value(1.0)), + }, + ], + }], + }; + + // Target schema has "host" but not "instance"; also has "region" which is missing from source. + let target = ArrowSchema::new(vec![ + Field::new( + "my_ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + Field::new("host", DataType::Utf8, true), + Field::new("region", DataType::Utf8, true), + Field::new("my_value", DataType::Float64, true), + ]); + + let batch = rows_to_aligned_record_batch(&rows, &target).unwrap(); + assert_eq!(batch.schema().as_ref(), &target); + assert_eq!(1, batch.num_rows()); + assert_eq!(4, batch.num_columns()); + + // "region" column should be null-filled. + let region = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(region.is_null(0)); + } + + #[test] + fn test_identify_missing_columns_from_proto() { + let rows_schema = vec![ + ColumnSchema { + column_name: "greptime_timestamp".to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "host".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "instance".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "greptime_value".to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as i32, + ..Default::default() + }, + ]; + + let target = ArrowSchema::new(vec![ + Field::new( + "my_ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + ), + Field::new("host", DataType::Utf8, true), + Field::new("my_value", DataType::Float64, true), + ]); + + let missing = identify_missing_columns_from_proto(&rows_schema, &target).unwrap(); + assert_eq!(missing, vec!["instance".to_string()]); + } + + #[test] + fn test_build_prom_create_table_schema_from_proto() { + let rows_schema = vec![ + ColumnSchema { + column_name: "greptime_timestamp".to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "job".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as i32, + ..Default::default() + }, + ColumnSchema { + column_name: "greptime_value".to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as i32, + ..Default::default() + }, + ]; + + let schema = build_prom_create_table_schema_from_proto(&rows_schema).unwrap(); + assert_eq!(3, schema.len()); + + assert_eq!("greptime_timestamp", schema[0].column_name); + assert_eq!(SemanticType::Timestamp as i32, schema[0].semantic_type); + assert_eq!( + ColumnDataType::TimestampMillisecond as i32, + schema[0].datatype + ); + + assert_eq!("job", schema[1].column_name); + assert_eq!(SemanticType::Tag as i32, schema[1].semantic_type); + assert_eq!(ColumnDataType::String as i32, schema[1].datatype); + + assert_eq!("greptime_value", schema[2].column_name); + assert_eq!(SemanticType::Field as i32, schema[2].semantic_type); + assert_eq!(ColumnDataType::Float64 as i32, schema[2].datatype); + } +} From b75a1125610a146f5b9e38176e82d0cdc7c36db3 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Wed, 1 Apr 2026 17:02:54 +0800 Subject: [PATCH 19/20] feat: implement prefilter for bulk memtable (#7895) * feat: prefilter in memtable Signed-off-by: evenyag * chore: fmt code Signed-off-by: evenyag * feat: bulk part reader also do prefilter Signed-off-by: evenyag * chore: extract pk filters check Signed-off-by: evenyag * fix: scanbench support explain verbose Signed-off-by: evenyag * feat: add metrics for mem prefilter Signed-off-by: evenyag * chore: address review comment Signed-off-by: evenyag * chore: remove dead code Signed-off-by: evenyag --------- Signed-off-by: evenyag --- src/cmd/src/datanode/scanbench.rs | 4 +- src/mito2/src/memtable.rs | 6 + src/mito2/src/memtable/bulk.rs | 2 - src/mito2/src/memtable/bulk/context.rs | 52 ++++- src/mito2/src/memtable/bulk/part.rs | 80 +++----- src/mito2/src/memtable/bulk/part_reader.rs | 105 ++++++++-- src/mito2/src/memtable/partition_tree/tree.rs | 1 + .../src/memtable/simple_bulk_memtable.rs | 1 + src/mito2/src/memtable/time_series.rs | 1 + src/mito2/src/read/scan_util.rs | 17 ++ src/mito2/src/sst/parquet/prefilter.rs | 185 +++++++++++++++++- 11 files changed, 377 insertions(+), 77 deletions(-) diff --git a/src/cmd/src/datanode/scanbench.rs b/src/cmd/src/datanode/scanbench.rs index 6bfe177fc1..51064126fe 100644 --- a/src/cmd/src/datanode/scanbench.rs +++ b/src/cmd/src/datanode/scanbench.rs @@ -677,7 +677,9 @@ impl ScanbenchCommand { // Scan all partitions let num_partitions = scanner.properties().partitions.len(); - let ctx = QueryScanContext::default(); + let ctx = QueryScanContext { + explain_verbose: self.verbose, + }; let metrics_set = ExecutionPlanMetricsSet::new(); let mut scan_futures = FuturesUnordered::new(); diff --git a/src/mito2/src/memtable.rs b/src/mito2/src/memtable.rs index 3ebfdd3628..154d062e07 100644 --- a/src/mito2/src/memtable.rs +++ b/src/mito2/src/memtable.rs @@ -497,6 +497,8 @@ impl MemScanMetrics { metrics.num_rows += inner.num_rows; metrics.num_batches += inner.num_batches; metrics.scan_cost += inner.scan_cost; + metrics.prefilter_cost += inner.prefilter_cost; + metrics.prefilter_rows_filtered += inner.prefilter_rows_filtered; } /// Gets the metrics data. @@ -515,6 +517,10 @@ pub(crate) struct MemScanMetricsData { pub(crate) num_batches: usize, /// Duration to scan the memtable. pub(crate) scan_cost: Duration, + /// Duration of prefilter in memtable scan. + pub(crate) prefilter_cost: Duration, + /// Number of rows filtered by prefilter in memtable scan. + pub(crate) prefilter_rows_filtered: usize, } /// Encoded range in the memtable. diff --git a/src/mito2/src/memtable/bulk.rs b/src/mito2/src/memtable/bulk.rs index 502b61759d..9d25d0c39f 100644 --- a/src/mito2/src/memtable/bulk.rs +++ b/src/mito2/src/memtable/bulk.rs @@ -15,9 +15,7 @@ //! Memtable implementation for bulk load pub(crate) mod chunk_reader; -#[allow(unused)] pub mod context; -#[allow(unused)] pub mod part; pub mod part_reader; mod row_group_reader; diff --git a/src/mito2/src/memtable/bulk/context.rs b/src/mito2/src/memtable/bulk/context.rs index c3274d30e9..7551eb33af 100644 --- a/src/mito2/src/memtable/bulk/context.rs +++ b/src/mito2/src/memtable/bulk/context.rs @@ -17,7 +17,8 @@ use std::collections::VecDeque; use std::sync::Arc; -use mito_codec::row_converter::{DensePrimaryKeyCodec, build_primary_key_codec}; +use common_recordbatch::filter::SimpleFilterEvaluator; +use mito_codec::row_converter::build_primary_key_codec; use parquet::file::metadata::ParquetMetaData; use store_api::metadata::RegionMetadataRef; use store_api::storage::ColumnId; @@ -25,8 +26,8 @@ use table::predicate::Predicate; use crate::error::Result; use crate::sst::parquet::file_range::{PreFilterMode, RangeBase}; -use crate::sst::parquet::flat_format::FlatReadFormat; use crate::sst::parquet::format::ReadFormat; +use crate::sst::parquet::prefilter::CachedPrimaryKeyFilter; use crate::sst::parquet::reader::SimpleFilterContext; use crate::sst::parquet::stats::RowGroupPruningStats; @@ -35,6 +36,9 @@ pub(crate) type BulkIterContextRef = Arc; pub struct BulkIterContext { pub(crate) base: RangeBase, pub(crate) predicate: Option, + /// Pre-extracted primary key filters for PK prefiltering. + /// `None` if PK prefiltering is not applicable. + pk_filters: Option>>, } impl BulkIterContext { @@ -62,7 +66,7 @@ impl BulkIterContext { ) -> Result { let codec = build_primary_key_codec(®ion_metadata); - let simple_filters = predicate + let simple_filters: Vec = predicate .as_ref() .iter() .flat_map(|predicate| { @@ -87,6 +91,9 @@ impl BulkIterContext { .map(|pred| pred.dyn_filters().as_ref().clone()) .unwrap_or_default(); + // Pre-extract PK filters if applicable. + let pk_filters = Self::extract_pk_filters(&read_format, &simple_filters); + Ok(Self { base: RangeBase { filters: simple_filters, @@ -102,6 +109,7 @@ impl BulkIterContext { partition_filter: None, }, predicate, + pk_filters, }) } @@ -133,6 +141,44 @@ impl BulkIterContext { } } + /// Extracts PK filters if flat format with dictionary-encoded PKs is used. + fn extract_pk_filters( + read_format: &ReadFormat, + filters: &[SimpleFilterContext], + ) -> Option>> { + let flat_format = read_format.as_flat()?; + if flat_format.batch_has_raw_pk_columns() { + return None; + } + let metadata = read_format.metadata(); + if metadata.primary_key.is_empty() { + return None; + } + + let pk_filters: Vec<_> = filters + .iter() + .filter_map(|f| f.primary_key_prefilter()) + .collect(); + if pk_filters.is_empty() { + return None; + } + + Some(Arc::new(pk_filters)) + } + + /// Builds a fresh PK filter for a new iterator. Returns `None` if PK + /// prefiltering is not applicable. + pub(crate) fn build_pk_filter(&self) -> Option { + let pk_filters = self.pk_filters.as_ref()?; + let metadata = self.base.read_format.metadata(); + // Parquet PK prefilter always supports the partition column. + let inner = self + .base + .codec + .primary_key_filter(metadata, Arc::clone(pk_filters), false); + Some(CachedPrimaryKeyFilter::new(inner)) + } + pub(crate) fn read_format(&self) -> &ReadFormat { &self.base.read_format } diff --git a/src/mito2/src/memtable/bulk/part.rs b/src/mito2/src/memtable/bulk/part.rs index bf345c038e..986e9409ee 100644 --- a/src/mito2/src/memtable/bulk/part.rs +++ b/src/mito2/src/memtable/bulk/part.rs @@ -14,66 +14,55 @@ //! Bulk part encoder/decoder. -use std::collections::{HashMap, HashSet, VecDeque}; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use std::time::{Duration, Instant}; use api::helper::{ColumnDataTypeWrapper, to_grpc_value}; use api::v1::bulk_wal_entry::Body; -use api::v1::{ArrowIpc, BulkWalEntry, Mutation, OpType, bulk_wal_entry}; +use api::v1::{ArrowIpc, BulkWalEntry, Mutation, OpType}; use bytes::Bytes; use common_grpc::flight::{FlightDecoder, FlightEncoder, FlightMessage}; use common_recordbatch::DfRecordBatch as RecordBatch; use common_time::Timestamp; -use common_time::timestamp::TimeUnit; use datatypes::arrow; -use datatypes::arrow::array::{ - Array, ArrayRef, BinaryBuilder, BinaryDictionaryBuilder, DictionaryArray, StringBuilder, - StringDictionaryBuilder, TimestampMicrosecondArray, TimestampMillisecondArray, - TimestampNanosecondArray, TimestampSecondArray, UInt8Array, UInt8Builder, UInt32Array, - UInt64Array, UInt64Builder, -}; -use datatypes::arrow::compute::{SortColumn, SortOptions, TakeOptions}; +use datatypes::arrow::array::{Array, ArrayRef, StringDictionaryBuilder, UInt8Array, UInt64Array}; +use datatypes::arrow::compute::{SortColumn, SortOptions}; use datatypes::arrow::datatypes::{ DataType as ArrowDataType, Field, Schema, SchemaRef, UInt32Type, }; -use datatypes::arrow_array::BinaryArray; use datatypes::data_type::DataType; -use datatypes::prelude::{MutableVector, ScalarVectorBuilder, Vector}; -use datatypes::value::{Value, ValueRef}; +use datatypes::prelude::{MutableVector, Vector}; +use datatypes::value::ValueRef; use datatypes::vectors::Helper; -use mito_codec::key_values::{KeyValue, KeyValues, KeyValuesRef}; -use mito_codec::row_converter::{ - DensePrimaryKeyCodec, PrimaryKeyCodec, PrimaryKeyCodecExt, build_primary_key_codec, -}; +use mito_codec::key_values::{KeyValue, KeyValues}; +use mito_codec::row_converter::PrimaryKeyCodec; use parquet::arrow::ArrowWriter; use parquet::basic::{Compression, ZstdLevel}; -use parquet::data_type::AsBytes; use parquet::file::metadata::ParquetMetaData; use parquet::file::properties::WriterProperties; use smallvec::SmallVec; -use snafu::{OptionExt, ResultExt, Snafu}; +use snafu::{OptionExt, ResultExt}; use store_api::codec::PrimaryKeyEncoding; -use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataRef}; +use store_api::metadata::{RegionMetadata, RegionMetadataRef}; use store_api::storage::consts::PRIMARY_KEY_COLUMN_NAME; -use store_api::storage::{FileId, RegionId, SequenceNumber, SequenceRange}; -use table::predicate::Predicate; +use store_api::storage::{FileId, SequenceNumber, SequenceRange}; use crate::error::{ - self, ColumnNotFoundSnafu, ComputeArrowSnafu, ConvertColumnDataTypeSnafu, CreateDefaultSnafu, - DataTypeMismatchSnafu, EncodeMemtableSnafu, EncodeSnafu, InvalidMetadataSnafu, - InvalidRequestSnafu, NewRecordBatchSnafu, Result, UnexpectedSnafu, + self, ColumnNotFoundSnafu, ComputeArrowSnafu, CreateDefaultSnafu, DataTypeMismatchSnafu, + EncodeMemtableSnafu, EncodeSnafu, InvalidMetadataSnafu, InvalidRequestSnafu, + NewRecordBatchSnafu, Result, }; use crate::memtable::bulk::context::BulkIterContextRef; use crate::memtable::bulk::part_reader::EncodedBulkPartIter; use crate::memtable::time_series::{ValueBuilder, Values}; use crate::memtable::{BoxedRecordBatchIterator, MemScanMetrics, MemtableStats}; +use crate::sst::SeriesEstimator; use crate::sst::index::IndexOutput; use crate::sst::parquet::file_range::{PreFilterMode, row_group_contains_delete}; use crate::sst::parquet::flat_format::primary_key_column_index; -use crate::sst::parquet::format::{PrimaryKeyArray, PrimaryKeyArrayBuilder, ReadFormat}; +use crate::sst::parquet::format::{PrimaryKeyArray, PrimaryKeyArrayBuilder}; use crate::sst::parquet::{PARQUET_METADATA_KEY, SstInfo}; -use crate::sst::{SeriesEstimator, to_sst_arrow_schema}; const INIT_DICT_VALUE_CAPACITY: usize = 8; @@ -527,8 +516,6 @@ impl PrimaryKeyColumnBuilder { /// Converter that converts structs into [BulkPart]. pub struct BulkPartConverter { - /// Region metadata. - region_metadata: RegionMetadataRef, /// Schema of the converted batch. schema: SchemaRef, /// Primary key codec for encoding keys @@ -577,7 +564,6 @@ impl BulkPartConverter { }; Self { - region_metadata: region_metadata.clone(), schema, primary_key_codec, key_buf: Vec::new(), @@ -1116,7 +1102,6 @@ pub struct BulkPartEncodeMetrics { pub struct BulkPartEncoder { metadata: RegionMetadataRef, - row_group_size: usize, writer_props: Option, } @@ -1141,7 +1126,6 @@ impl BulkPartEncoder { Ok(Self { metadata, - row_group_size, writer_props, }) } @@ -1182,7 +1166,6 @@ impl BulkPartEncoder { iter_start = Instant::now(); } metrics.iter_cost += iter_start.elapsed(); - iter_start = Instant::now(); if total_rows == 0 { return Ok(None); @@ -1348,11 +1331,6 @@ impl MultiBulkPart { self.batches.len() } - /// Returns an iterator over the record batches. - pub(crate) fn batches(&self) -> impl Iterator { - self.batches.iter() - } - /// Returns the estimated memory size of all batches. pub(crate) fn estimated_size(&self) -> usize { self.batches.iter().map(record_batch_estimated_size).sum() @@ -1400,19 +1378,22 @@ impl MultiBulkPart { mod tests { use api::v1::{Row, SemanticType, WriteHint}; use datafusion_common::ScalarValue; - use datatypes::arrow::array::Float64Array; + use datatypes::arrow::array::{ + BinaryArray, DictionaryArray, Float64Array, TimestampMillisecondArray, + }; + use datatypes::arrow::datatypes::UInt32Type; use datatypes::prelude::{ConcreteDataType, Value}; use datatypes::schema::ColumnSchema; + use mito_codec::row_converter::build_primary_key_codec; use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder}; use store_api::storage::RegionId; use store_api::storage::consts::ReservedColumnId; + use table::predicate::Predicate; use super::*; use crate::memtable::bulk::context::BulkIterContext; use crate::sst::{FlatSchemaOptions, to_flat_sst_arrow_schema}; - use crate::test_util::memtable_util::{ - build_key_values_with_ts_seq_values, metadata_for_test, region_metadata_to_row_schema, - }; + use crate::test_util::memtable_util::{build_key_values_with_ts_seq_values, metadata_for_test}; struct MutationInput<'a> { k0: &'a str, @@ -1422,13 +1403,6 @@ mod tests { sequence: u64, } - #[derive(Debug, PartialOrd, PartialEq)] - struct BatchOutput<'a> { - pk_values: &'a [Value], - timestamps: &'a [i64], - v1: &'a [Option], - } - fn encode(input: &[MutationInput]) -> EncodedBulkPart { let metadata = metadata_for_test(); let kvs = input @@ -1482,7 +1456,7 @@ mod tests { ]); let projection = &[4u32]; - let mut reader = part + let reader = part .read( Arc::new( BulkIterContext::new( @@ -1523,7 +1497,7 @@ mod tests { let kvs = key_values .into_iter() .map(|(k0, k1, (start, end), sequence)| { - let ts = (start..end); + let ts = start..end; let v1 = (start..end).map(|_| None); build_key_values_with_ts_seq_values(&metadata, k0.to_string(), k1, ts, v1, sequence) }) @@ -1553,7 +1527,7 @@ mod tests { ) .unwrap(), ); - let mut reader = part + let reader = part .read(context, None, None) .unwrap() .expect("expect at least one row group"); @@ -1626,7 +1600,7 @@ mod tests { 100, ); - /// Predicates over field column can do precise filtering. + // Predicates over field column can do precise filtering. check_prune_row_group( &part, Some(Predicate::new(vec![ diff --git a/src/mito2/src/memtable/bulk/part_reader.rs b/src/mito2/src/memtable/bulk/part_reader.rs index 1375e79542..a9caeef08c 100644 --- a/src/mito2/src/memtable/bulk/part_reader.rs +++ b/src/mito2/src/memtable/bulk/part_reader.rs @@ -17,6 +17,7 @@ use std::time::Instant; use datatypes::arrow::array::BooleanArray; use datatypes::arrow::record_batch::RecordBatch; +use mito_codec::row_converter::PrimaryKeyFilter; use parquet::arrow::ProjectionMask; use parquet::arrow::arrow_reader::ParquetRecordBatchReader; use snafu::ResultExt; @@ -29,7 +30,8 @@ use crate::memtable::bulk::row_group_reader::MemtableRowGroupReaderBuilder; use crate::memtable::{MemScanMetrics, MemScanMetricsData}; use crate::metrics::{READ_ROWS_TOTAL, READ_STAGE_ELAPSED}; use crate::sst::parquet::file_range::{PreFilterMode, TagDecodeState}; -use crate::sst::parquet::flat_format::sequence_column_index; +use crate::sst::parquet::flat_format::{primary_key_column_index, sequence_column_index}; +use crate::sst::parquet::prefilter::{CachedPrimaryKeyFilter, prefilter_flat_batch_by_primary_key}; /// Iterator for reading data inside a bulk part. pub struct EncodedBulkPartIter { @@ -41,6 +43,8 @@ pub struct EncodedBulkPartIter { sequence: Option, /// Cached skip_fields for current row group. current_skip_fields: bool, + /// Primary key filter for prefiltering before convert_batch. + pk_filter: Option, /// Metrics for this iterator. metrics: MemScanMetricsData, /// Optional memory scan metrics to report to. @@ -69,6 +73,9 @@ impl EncodedBulkPartIter { let builder = MemtableRowGroupReaderBuilder::try_new(&context, projection_mask, parquet_meta, data)?; + // Build PK filter if applicable (flat format with dictionary-encoded PKs). + let pk_filter = context.build_pk_filter(); + let (init_reader, current_skip_fields) = match row_groups_to_read.pop_front() { Some(first_row_group) => { let skip_fields = builder.compute_skip_fields(&context, first_row_group); @@ -85,6 +92,7 @@ impl EncodedBulkPartIter { builder, sequence, current_skip_fields, + pk_filter, metrics: MemScanMetricsData { total_series: series_count, ..Default::default() @@ -116,6 +124,10 @@ impl EncodedBulkPartIter { &self.sequence, batch, self.current_skip_fields, + self.pk_filter + .as_mut() + .map(|f| f as &mut dyn PrimaryKeyFilter), + &mut self.metrics, )? { // Update metrics self.metrics.num_batches += 1; @@ -142,6 +154,10 @@ impl EncodedBulkPartIter { &self.sequence, batch, self.current_skip_fields, + self.pk_filter + .as_mut() + .map(|f| f as &mut dyn PrimaryKeyFilter), + &mut self.metrics, )? { // Update metrics self.metrics.num_batches += 1; @@ -175,12 +191,14 @@ impl Iterator for EncodedBulkPartIter { impl Drop for EncodedBulkPartIter { fn drop(&mut self) { common_telemetry::debug!( - "EncodedBulkPartIter region: {}, metrics: total_series={}, num_rows={}, num_batches={}, scan_cost={:?}", + "EncodedBulkPartIter region: {}, metrics: total_series={}, num_rows={}, num_batches={}, scan_cost={:?}, prefilter_cost={:?}, prefilter_rows_filtered={}", self.context.region_id(), self.metrics.total_series, self.metrics.num_rows, self.metrics.num_batches, - self.metrics.scan_cost + self.metrics.scan_cost, + self.metrics.prefilter_cost, + self.metrics.prefilter_rows_filtered ); // Report MemScanMetrics if not already reported @@ -205,6 +223,8 @@ pub struct BulkPartBatchIter { context: BulkIterContextRef, /// Sequence number filter. sequence: Option, + /// Primary key filter for prefiltering before convert_batch. + pk_filter: Option, /// Metrics for this iterator. metrics: MemScanMetricsData, /// Optional memory scan metrics to report to. @@ -222,10 +242,13 @@ impl BulkPartBatchIter { ) -> Self { assert!(context.read_format().as_flat().is_some()); + let pk_filter = context.build_pk_filter(); + Self { batches: VecDeque::from(batches), context, sequence, + pk_filter, metrics: MemScanMetricsData { total_series: series_count, ..Default::default() @@ -282,8 +305,16 @@ impl BulkPartBatchIter { PreFilterMode::SkipFieldsOnDelete => true, }; - let Some(filtered_batch) = - apply_combined_filters(&self.context, &self.sequence, projected_batch, skip_fields)? + let Some(filtered_batch) = apply_combined_filters( + &self.context, + &self.sequence, + projected_batch, + skip_fields, + self.pk_filter + .as_mut() + .map(|f| f as &mut dyn PrimaryKeyFilter), + &mut self.metrics, + )? else { self.metrics.scan_cost += start.elapsed(); return Ok(None); @@ -323,12 +354,14 @@ impl Iterator for BulkPartBatchIter { impl Drop for BulkPartBatchIter { fn drop(&mut self) { common_telemetry::debug!( - "BulkPartBatchIter region: {}, metrics: total_series={}, num_rows={}, num_batches={}, scan_cost={:?}", + "BulkPartBatchIter region: {}, metrics: total_series={}, num_rows={}, num_batches={}, scan_cost={:?}, prefilter_cost={:?}, prefilter_rows_filtered={}", self.context.region_id(), self.metrics.total_series, self.metrics.num_rows, self.metrics.num_batches, - self.metrics.scan_cost + self.metrics.scan_cost, + self.metrics.prefilter_cost, + self.metrics.prefilter_rows_filtered ); // Report MemScanMetrics if not already reported @@ -353,8 +386,32 @@ fn apply_combined_filters( sequence: &Option, record_batch: RecordBatch, skip_fields: bool, + pk_filter: Option<&mut dyn PrimaryKeyFilter>, + metrics: &mut MemScanMetricsData, ) -> error::Result> { - // Converts the format to the flat format first. + // Apply PK prefilter on raw batch before convert_batch to reduce conversion overhead. + let has_pk_prefilter = pk_filter.is_some(); + let record_batch = if let Some(pk_filter) = pk_filter { + let rows_before = record_batch.num_rows(); + let prefilter_start = Instant::now(); + let pk_col_idx = primary_key_column_index(record_batch.num_columns()); + match prefilter_flat_batch_by_primary_key(record_batch, pk_col_idx, pk_filter)? { + Some(batch) => { + metrics.prefilter_cost += prefilter_start.elapsed(); + metrics.prefilter_rows_filtered += rows_before - batch.num_rows(); + batch + } + None => { + metrics.prefilter_cost += prefilter_start.elapsed(); + metrics.prefilter_rows_filtered += rows_before; + return Ok(None); + } + } + } else { + record_batch + }; + + // Converts the format to the flat format. let format = context.read_format().as_flat().unwrap(); let record_batch = format.convert_batch(record_batch, None)?; @@ -362,12 +419,12 @@ fn apply_combined_filters( let mut combined_filter = None; let mut tag_decode_state = TagDecodeState::new(); - // First, apply predicate filters using the shared method. + // Apply predicate filters using the shared method. if !context.base.filters.is_empty() { let predicate_mask = context.base.compute_filter_mask_flat( &record_batch, skip_fields, - false, + has_pk_prefilter, &mut tag_decode_state, )?; // If predicate filters out the entire batch, return None early @@ -433,6 +490,7 @@ mod tests { use super::*; use crate::memtable::bulk::context::BulkIterContext; + use crate::test_util::sst_util::new_primary_key; #[test] fn test_bulk_part_batch_iter() { @@ -461,9 +519,16 @@ mod tests { vec![1000, 2000, 3000], )); - // Create primary key dictionary array + // Create primary key dictionary array with properly encoded PKs use datatypes::arrow::array::{BinaryArray, DictionaryArray, UInt32Array}; - let values = Arc::new(BinaryArray::from_iter_values([b"key1", b"key2", b"key3"])); + let pk1 = new_primary_key(&["key1"]); + let pk2 = new_primary_key(&["key2"]); + let pk3 = new_primary_key(&["key3"]); + let values = Arc::new(BinaryArray::from_iter_values([ + pk1.as_slice(), + pk2.as_slice(), + pk3.as_slice(), + ])); let keys = UInt32Array::from(vec![0, 1, 2]); let primary_key = Arc::new(DictionaryArray::new(keys, values)); @@ -596,12 +661,17 @@ mod tests { ])); // Create first batch with 2 rows + let pk1 = new_primary_key(&["key1"]); + let pk2 = new_primary_key(&["key2"]); let key1_1 = Arc::new(StringArray::from_iter_values(["key1", "key2"])); let field1_1 = Arc::new(Int64Array::from(vec![11, 12])); let timestamp_1 = Arc::new(datatypes::arrow::array::TimestampMillisecondArray::from( vec![1000, 2000], )); - let values_1 = Arc::new(BinaryArray::from_iter_values([b"key1", b"key2"])); + let values_1 = Arc::new(BinaryArray::from_iter_values([ + pk1.as_slice(), + pk2.as_slice(), + ])); let keys_1 = UInt32Array::from(vec![0, 1]); let primary_key_1 = Arc::new(DictionaryArray::new(keys_1, values_1)); let sequence_1 = Arc::new(UInt64Array::from(vec![1, 2])); @@ -621,12 +691,19 @@ mod tests { .unwrap(); // Create second batch with 3 rows + let pk3 = new_primary_key(&["key3"]); + let pk4 = new_primary_key(&["key4"]); + let pk5 = new_primary_key(&["key5"]); let key1_2 = Arc::new(StringArray::from_iter_values(["key3", "key4", "key5"])); let field1_2 = Arc::new(Int64Array::from(vec![13, 14, 15])); let timestamp_2 = Arc::new(datatypes::arrow::array::TimestampMillisecondArray::from( vec![3000, 4000, 5000], )); - let values_2 = Arc::new(BinaryArray::from_iter_values([b"key3", b"key4", b"key5"])); + let values_2 = Arc::new(BinaryArray::from_iter_values([ + pk3.as_slice(), + pk4.as_slice(), + pk5.as_slice(), + ])); let keys_2 = UInt32Array::from(vec![0, 1, 2]); let primary_key_2 = Arc::new(DictionaryArray::new(keys_2, values_2)); let sequence_2 = Arc::new(UInt64Array::from(vec![3, 4, 5])); diff --git a/src/mito2/src/memtable/partition_tree/tree.rs b/src/mito2/src/memtable/partition_tree/tree.rs index 17977db56a..f5863ae0c8 100644 --- a/src/mito2/src/memtable/partition_tree/tree.rs +++ b/src/mito2/src/memtable/partition_tree/tree.rs @@ -490,6 +490,7 @@ impl TreeIter { num_rows: self.metrics.rows_fetched, num_batches: self.metrics.batches_fetched, scan_cost: self.metrics.iter_elapsed, + ..Default::default() }; mem_scan_metrics.merge_inner(&inner); } diff --git a/src/mito2/src/memtable/simple_bulk_memtable.rs b/src/mito2/src/memtable/simple_bulk_memtable.rs index 6d91f00361..1284741347 100644 --- a/src/mito2/src/memtable/simple_bulk_memtable.rs +++ b/src/mito2/src/memtable/simple_bulk_memtable.rs @@ -372,6 +372,7 @@ impl IterBuilder for BatchRangeBuilder { num_rows: batch.num_rows(), num_batches: 1, scan_cost: self.scan_cost, + ..Default::default() }; metrics.merge_inner(&inner); } diff --git a/src/mito2/src/memtable/time_series.rs b/src/mito2/src/memtable/time_series.rs index d3d00d0703..9666bee51c 100644 --- a/src/mito2/src/memtable/time_series.rs +++ b/src/mito2/src/memtable/time_series.rs @@ -567,6 +567,7 @@ impl Iter { num_rows: self.metrics.num_rows, num_batches: self.metrics.num_batches, scan_cost: self.metrics.scan_cost, + ..Default::default() }; mem_scan_metrics.merge_inner(&inner); } diff --git a/src/mito2/src/read/scan_util.rs b/src/mito2/src/read/scan_util.rs index 9bf1c17276..d065657242 100644 --- a/src/mito2/src/read/scan_util.rs +++ b/src/mito2/src/read/scan_util.rs @@ -138,6 +138,10 @@ pub(crate) struct ScanMetricsSet { mem_batches: usize, /// Number of series read from memtables. mem_series: usize, + /// Duration of prefilter in memtable scan. + mem_prefilter_cost: Duration, + /// Number of rows filtered by prefilter in memtable scan. + mem_prefilter_rows_filtered: usize, // SST related metrics: /// Duration to build file ranges. @@ -341,6 +345,8 @@ impl fmt::Debug for ScanMetricsSet { mem_rows, mem_batches, mem_series, + mem_prefilter_cost, + mem_prefilter_rows_filtered, inverted_index_apply_metrics, bloom_filter_apply_metrics, fulltext_index_apply_metrics, @@ -509,6 +515,15 @@ impl fmt::Debug for ScanMetricsSet { if !mem_scan_cost.is_zero() { write!(f, ", \"mem_scan_cost\":\"{mem_scan_cost:?}\"")?; } + if !mem_prefilter_cost.is_zero() { + write!(f, ", \"mem_prefilter_cost\":\"{mem_prefilter_cost:?}\"")?; + } + if *mem_prefilter_rows_filtered > 0 { + write!( + f, + ", \"mem_prefilter_rows_filtered\":{mem_prefilter_rows_filtered}" + )?; + } // Write optional verbose metrics if they are not empty if let Some(metrics) = inverted_index_apply_metrics @@ -1061,6 +1076,8 @@ impl PartitionMetrics { metrics.mem_rows += data.num_rows; metrics.mem_batches += data.num_batches; metrics.mem_series += data.total_series; + metrics.mem_prefilter_cost += data.prefilter_cost; + metrics.mem_prefilter_rows_filtered += data.prefilter_rows_filtered; } /// Merges [ScannerMetrics], `build_reader_cost`, `scan_cost` and `yield_cost`. diff --git a/src/mito2/src/sst/parquet/prefilter.rs b/src/mito2/src/sst/parquet/prefilter.rs index 88df56e401..967ddd491b 100644 --- a/src/mito2/src/sst/parquet/prefilter.rs +++ b/src/mito2/src/sst/parquet/prefilter.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use api::v1::SemanticType; use common_recordbatch::filter::SimpleFilterEvaluator; -use datatypes::arrow::array::BinaryArray; +use datatypes::arrow::array::{BinaryArray, BooleanArray, BooleanBufferBuilder}; use datatypes::arrow::record_batch::RecordBatch; use futures::StreamExt; use mito_codec::row_converter::{PrimaryKeyCodec, PrimaryKeyFilter}; @@ -33,7 +33,7 @@ use parquet::schema::types::SchemaDescriptor; use snafu::{OptionExt, ResultExt}; use store_api::metadata::{RegionMetadata, RegionMetadataRef}; -use crate::error::{DecodeSnafu, ReadParquetSnafu, Result, UnexpectedSnafu}; +use crate::error::{ComputeArrowSnafu, DecodeSnafu, ReadParquetSnafu, Result, UnexpectedSnafu}; use crate::sst::parquet::flat_format::primary_key_column_index; use crate::sst::parquet::format::{PrimaryKeyArray, ReadFormat}; use crate::sst::parquet::reader::{RowGroupBuildContext, RowGroupReaderBuilder}; @@ -93,6 +93,55 @@ pub(crate) fn matching_row_ranges_by_primary_key( Ok(matched_row_ranges) } +/// Filters a flat-format record batch by primary key, returning only rows whose +/// primary key matches the filter. Returns `None` if all rows are filtered out. +pub(crate) fn prefilter_flat_batch_by_primary_key( + input: RecordBatch, + pk_column_index: usize, + pk_filter: &mut dyn PrimaryKeyFilter, +) -> Result> { + if input.num_rows() == 0 { + return Ok(Some(input)); + } + + let matched_row_ranges = + matching_row_ranges_by_primary_key(&input, pk_column_index, pk_filter)?; + if matched_row_ranges.is_empty() { + return Ok(None); + } + + if matched_row_ranges.len() == 1 + && matched_row_ranges[0].start == 0 + && matched_row_ranges[0].end == input.num_rows() + { + return Ok(Some(input)); + } + + if matched_row_ranges.len() == 1 { + let span = &matched_row_ranges[0]; + return Ok(Some(input.slice(span.start, span.end - span.start))); + } + + let mut builder = BooleanBufferBuilder::new(input.num_rows()); + builder.append_n(input.num_rows(), false); + for span in matched_row_ranges { + for i in span { + builder.set_bit(i, true); + } + } + + let filtered = datatypes::arrow::compute::filter_record_batch( + &input, + &BooleanArray::new(builder.finish(), None), + ) + .context(ComputeArrowSnafu)?; + if filtered.num_rows() == 0 { + Ok(None) + } else { + Ok(Some(filtered)) + } +} + /// Returns whether a filter can be applied by parquet primary-key prefiltering. /// /// Unlike `PartitionTreeMemtable`, parquet prefilter always supports predicates @@ -346,12 +395,19 @@ mod tests { use common_recordbatch::filter::SimpleFilterEvaluator; use datafusion_expr::{col, lit}; - use mito_codec::row_converter::PrimaryKeyFilter; + use datatypes::arrow::array::{ + ArrayRef, DictionaryArray, TimestampMillisecondArray, UInt8Array, UInt32Array, UInt64Array, + }; + use datatypes::arrow::datatypes::{Schema, UInt32Type}; + use mito_codec::row_converter::{PrimaryKeyFilter, build_primary_key_codec}; use store_api::codec::PrimaryKeyEncoding; use super::*; + use crate::sst::internal_fields; use crate::sst::parquet::format::ReadFormat; - use crate::test_util::sst_util::{new_primary_key, sst_region_metadata_with_encoding}; + use crate::test_util::sst_util::{ + new_primary_key, sst_region_metadata, sst_region_metadata_with_encoding, + }; #[test] fn test_is_usable_primary_key_filter_skips_legacy_primary_key_batches() { @@ -416,4 +472,125 @@ mod tests { assert_eq!(hits.load(Ordering::Relaxed), 2); } + + fn new_test_filters(exprs: &[datafusion_expr::Expr]) -> Vec { + exprs + .iter() + .filter_map(SimpleFilterEvaluator::try_new) + .collect() + } + + fn new_raw_batch(primary_keys: &[&[u8]], field_values: &[u64]) -> RecordBatch { + assert_eq!(primary_keys.len(), field_values.len()); + + let metadata = Arc::new(sst_region_metadata()); + let arrow_schema = metadata.schema.arrow_schema(); + let field_column = arrow_schema + .field(arrow_schema.index_of("field_0").unwrap()) + .clone(); + let time_index_column = arrow_schema + .field(arrow_schema.index_of("ts").unwrap()) + .clone(); + let mut fields = vec![field_column, time_index_column]; + fields.extend( + internal_fields() + .into_iter() + .map(|field| field.as_ref().clone()), + ); + let schema = Arc::new(Schema::new(fields)); + + let mut dict_values = Vec::new(); + let mut keys = Vec::with_capacity(primary_keys.len()); + for pk in primary_keys { + let key = dict_values + .iter() + .position(|existing: &&[u8]| existing == pk) + .unwrap_or_else(|| { + dict_values.push(*pk); + dict_values.len() - 1 + }); + keys.push(key as u32); + } + let pk_array: ArrayRef = Arc::new(DictionaryArray::::new( + UInt32Array::from(keys), + Arc::new(BinaryArray::from_iter_values(dict_values.iter().copied())), + )); + + RecordBatch::try_new( + schema, + vec![ + Arc::new(UInt64Array::from(field_values.to_vec())), + Arc::new(TimestampMillisecondArray::from_iter_values( + 0..primary_keys.len() as i64, + )), + pk_array, + Arc::new(UInt64Array::from(vec![1; primary_keys.len()])), + Arc::new(UInt8Array::from(vec![1; primary_keys.len()])), + ], + ) + .unwrap() + } + + fn field_values(batch: &RecordBatch) -> Vec { + batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .values() + .to_vec() + } + + #[test] + fn test_prefilter_primary_key_drops_single_dictionary_batch() { + let metadata = Arc::new(sst_region_metadata()); + let filters = Arc::new(new_test_filters(&[col("tag_0").eq(lit("b"))])); + let mut primary_key_filter = build_primary_key_codec(metadata.as_ref()) + .primary_key_filter(&metadata, filters, false); + let pk_a = new_primary_key(&["a", "x"]); + let batch = new_raw_batch(&[pk_a.as_slice(), pk_a.as_slice()], &[10, 11]); + let pk_col_idx = primary_key_column_index(batch.num_columns()); + + let filtered = + prefilter_flat_batch_by_primary_key(batch, pk_col_idx, primary_key_filter.as_mut()) + .unwrap(); + + assert!(filtered.is_none()); + } + + #[test] + fn test_prefilter_primary_key_builds_mask_for_fragmented_matches() { + let metadata = Arc::new(sst_region_metadata()); + let filters = Arc::new(new_test_filters(&[col("tag_0") + .eq(lit("a")) + .or(col("tag_0").eq(lit("c")))])); + let mut primary_key_filter = build_primary_key_codec(metadata.as_ref()) + .primary_key_filter(&metadata, filters, false); + let pk_a = new_primary_key(&["a", "x"]); + let pk_b = new_primary_key(&["b", "x"]); + let pk_c = new_primary_key(&["c", "x"]); + let pk_d = new_primary_key(&["d", "x"]); + let batch = new_raw_batch( + &[ + pk_a.as_slice(), + pk_a.as_slice(), + pk_b.as_slice(), + pk_b.as_slice(), + pk_c.as_slice(), + pk_c.as_slice(), + pk_d.as_slice(), + pk_d.as_slice(), + ], + &[10, 11, 12, 13, 14, 15, 16, 17], + ); + let pk_col_idx = primary_key_column_index(batch.num_columns()); + + let filtered = + prefilter_flat_batch_by_primary_key(batch, pk_col_idx, primary_key_filter.as_mut()) + .unwrap() + .unwrap(); + + assert_eq!(filtered.num_rows(), 4); + assert_eq!(field_values(&filtered), vec![10, 11, 14, 15]); + } } From 3f3407fa2433858d041ed21aad000ea0fe9c76a5 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Wed, 1 Apr 2026 20:14:53 +0800 Subject: [PATCH 20/20] feat: partial success in trace ingestion (#7892) * feat: impl partial success Signed-off-by: shuiyisong * refactor: grouping by resource and scope Signed-off-by: shuiyisong * chore: remove unused code Signed-off-by: shuiyisong * chore: rebase main & fix clippy Signed-off-by: shuiyisong * chore: add trace ingestion failure counter Signed-off-by: shuiyisong * fix: address comments Signed-off-by: shuiyisong * fix: update status list and remove TODO Signed-off-by: shuiyisong * fix: address comments Signed-off-by: shuiyisong * fix: fmt Signed-off-by: shuiyisong * chore: add more tests Signed-off-by: shuiyisong * fix: fmt Signed-off-by: shuiyisong --------- Signed-off-by: shuiyisong --- Cargo.lock | 1 + src/frontend/Cargo.toml | 1 + src/frontend/src/instance/otlp.rs | 554 +++++++++++++++++++++++++++-- src/frontend/src/metrics.rs | 8 + src/servers/src/http/otlp.rs | 13 +- src/servers/src/otlp/trace.rs | 77 +++- src/servers/src/otlp/trace/span.rs | 115 +++++- src/servers/src/otlp/trace/v0.rs | 136 ++++--- src/servers/src/otlp/trace/v1.rs | 125 ++++--- src/servers/src/query_handler.rs | 10 +- tests-integration/tests/http.rs | 89 ++++- 11 files changed, 980 insertions(+), 149 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 54be9bbdcb..695f19b072 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5255,6 +5255,7 @@ dependencies = [ "humantime", "humantime-serde", "hyper-util", + "itertools 0.14.0", "lazy_static", "log-query", "meta-client", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 03b0d35130..1b0ffe6e29 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -52,6 +52,7 @@ futures.workspace = true hostname.workspace = true humantime.workspace = true humantime-serde.workspace = true +itertools.workspace = true lazy_static.workspace = true log-query.workspace = true meta-client.workspace = true diff --git a/src/frontend/src/instance/otlp.rs b/src/frontend/src/instance/otlp.rs index 59174aa89a..8cda639686 100644 --- a/src/frontend/src/instance/otlp.rs +++ b/src/frontend/src/instance/otlp.rs @@ -19,9 +19,11 @@ use api::v1::{ColumnDataType, RowInsertRequests}; use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; use client::Output; -use common_error::ext::BoxedError; +use common_error::ext::{BoxedError, ErrorExt}; +use common_error::status_code::StatusCode; use common_query::prelude::GREPTIME_PHYSICAL_TABLE; use common_telemetry::tracing; +use itertools::Itertools; use opentelemetry_proto::tonic::collector::logs::v1::ExportLogsServiceRequest; use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; use otel_arrow_rust::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; @@ -30,17 +32,57 @@ use servers::error::{self, AuthSnafu, Result as ServerResult}; use servers::http::prom_store::PHYSICAL_TABLE_PARAM; use servers::interceptor::{OpenTelemetryProtocolInterceptor, OpenTelemetryProtocolInterceptorRef}; use servers::otlp; +use servers::otlp::trace::TraceAuxData; use servers::otlp::trace::coerce::{ coerce_value_data, is_supported_trace_coercion, resolve_new_trace_column_type, trace_value_datatype, }; -use servers::query_handler::{OpenTelemetryProtocolHandler, PipelineHandlerRef}; +use servers::otlp::trace::span::{TraceSpan, TraceSpanGroup}; +use servers::query_handler::{ + OpenTelemetryProtocolHandler, PipelineHandlerRef, TraceIngestOutcome, +}; use session::context::QueryContextRef; use snafu::ResultExt; use table::requests::{OTLP_METRIC_COMPAT_KEY, OTLP_METRIC_COMPAT_PROM}; use crate::instance::Instance; -use crate::metrics::{OTLP_LOGS_ROWS, OTLP_METRICS_ROWS, OTLP_TRACES_ROWS}; +use crate::metrics::{ + OTLP_LOGS_ROWS, OTLP_METRICS_ROWS, OTLP_TRACES_FAILURE_COUNT, OTLP_TRACES_ROWS, +}; + +const TRACE_INGEST_CHUNK_SIZE: usize = 64; +const TRACE_FAILURE_MESSAGE_LIMIT: usize = 4; + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +enum ChunkFailureReaction { + RetryPerSpan, + DiscardChunk, + Propagate, +} + +impl ChunkFailureReaction { + fn as_metric_label(self) -> &'static str { + match self { + Self::RetryPerSpan => "retry_per_span", + Self::DiscardChunk => "discard_chunk", + Self::Propagate => "propagate_failure", + } + } +} + +struct TraceChunkIngestContext<'a> { + pipeline_handler: PipelineHandlerRef, + pipeline: &'a PipelineWay, + pipeline_params: &'a GreptimePipelineParams, + table_name: &'a str, + is_trace_v1_model: bool, +} + +struct TraceIngestState { + aux_data: TraceAuxData, + outcome: TraceIngestOutcome, + failure_messages: Vec, +} #[async_trait] impl OpenTelemetryProtocolHandler for Instance { @@ -116,7 +158,7 @@ impl OpenTelemetryProtocolHandler for Instance { pipeline_params: GreptimePipelineParams, table_name: String, ctx: QueryContextRef, - ) -> ServerResult { + ) -> ServerResult { self.plugins .get::() .as_ref() @@ -128,32 +170,16 @@ impl OpenTelemetryProtocolHandler for Instance { .get::>(); interceptor_ref.pre_execute(ctx.clone())?; - let is_trace_v1_model = matches!(pipeline, PipelineWay::OtlpTraceDirectV1); - - let (mut requests, rows) = otlp::trace::to_grpc_insert_requests( - request, - pipeline, - pipeline_params, - table_name, - &ctx, + let spans = otlp::trace::span::parse(request); + self.ingest_trace_spans( pipeline_handler, - )?; - - OTLP_TRACES_ROWS.inc_by(rows as u64); - - if is_trace_v1_model { - self.reconcile_trace_column_types(&mut requests, &ctx) - .await?; - self.handle_trace_inserts(requests, ctx) - .await - .map_err(BoxedError::new) - .context(error::ExecuteGrpcQuerySnafu) - } else { - self.handle_log_inserts(requests, ctx) - .await - .map_err(BoxedError::new) - .context(error::ExecuteGrpcQuerySnafu) - } + &pipeline, + &pipeline_params, + table_name, + spans, + ctx, + ) + .await } #[tracing::instrument(skip_all)] @@ -210,6 +236,316 @@ impl OpenTelemetryProtocolHandler for Instance { } impl Instance { + /// Ingest OTLP trace spans with chunk-level writes and span-level fallback on + /// deterministic chunk failures. + async fn ingest_trace_spans( + &self, + pipeline_handler: PipelineHandlerRef, + pipeline: &PipelineWay, + pipeline_params: &GreptimePipelineParams, + table_name: String, + groups: Vec, + ctx: QueryContextRef, + ) -> ServerResult { + let is_trace_v1_model = matches!(pipeline, PipelineWay::OtlpTraceDirectV1); + let ingest_ctx = TraceChunkIngestContext { + pipeline_handler, + pipeline, + pipeline_params, + table_name: &table_name, + is_trace_v1_model, + }; + let mut ingest_state = TraceIngestState { + aux_data: TraceAuxData::default(), + outcome: TraceIngestOutcome::default(), + failure_messages: Vec::new(), + }; + + for group in groups { + let chunks = group + .spans + .into_iter() + .chunks(TRACE_INGEST_CHUNK_SIZE) + .into_iter() + .map(|chunk| chunk.collect::>()) + .collect::>(); + for chunk in chunks { + self.ingest_trace_chunk(&ingest_ctx, chunk, ctx.clone(), &mut ingest_state) + .await?; + } + } + + OTLP_TRACES_ROWS.inc_by(ingest_state.outcome.accepted_spans as u64); + + if !ingest_state.aux_data.is_empty() { + // Auxiliary trace tables are derived from spans whose main-table + // writes are already confirmed, so they never create new accepted + // spans and they do not affect rejected span counts. + let (aux_requests, _) = otlp::trace::to_grpc_insert_requests_for_aux_tables( + std::mem::take(&mut ingest_state.aux_data), + ingest_ctx.pipeline, + ingest_ctx.table_name, + )?; + + if !aux_requests.inserts.is_empty() { + match self + .insert_trace_requests(aux_requests, ingest_ctx.is_trace_v1_model, ctx) + .await + { + Ok(output) => { + Self::add_trace_write_cost(&mut ingest_state.outcome, output.meta.cost); + } + Err(err) => { + Self::push_trace_failure_message( + &mut ingest_state.failure_messages, + "aux_table_update_failed", + format!( + "Auxiliary trace tables were not fully updated ({})", + err.status_code().as_ref() + ), + ); + } + } + } + } + + ingest_state.outcome.error_message = Self::finish_trace_failure_message( + ingest_state.outcome.accepted_spans, + ingest_state.outcome.rejected_spans, + ingest_state.failure_messages, + ); + + Ok(ingest_state.outcome) + } + + /// Ingest one owned trace chunk so successful spans can be moved into the + /// accepted set without extra cloning. + async fn ingest_trace_chunk( + &self, + ingest_ctx: &TraceChunkIngestContext<'_>, + chunk: Vec, + ctx: QueryContextRef, + ingest_state: &mut TraceIngestState, + ) -> ServerResult<()> { + // Try the fast path first so healthy batches keep their original + // throughput and write amplification stays low. + let (requests, chunk_rows) = otlp::trace::to_grpc_insert_requests_from_spans( + &chunk, + ingest_ctx.pipeline, + ingest_ctx.pipeline_params, + ingest_ctx.table_name, + &ctx, + ingest_ctx.pipeline_handler.clone(), + )?; + + match self + .insert_trace_requests(requests, ingest_ctx.is_trace_v1_model, ctx.clone()) + .await + { + Ok(output) => { + Self::add_trace_write_cost(&mut ingest_state.outcome, output.meta.cost); + ingest_state.outcome.accepted_spans += chunk_rows; + for span in &chunk { + ingest_state.aux_data.observe_span(span); + } + } + Err(err) => match Self::classify_trace_chunk_failure(err.status_code()) { + ChunkFailureReaction::RetryPerSpan => { + Self::push_trace_failure_message( + &mut ingest_state.failure_messages, + ChunkFailureReaction::RetryPerSpan.as_metric_label(), + format!("Chunk fallback triggered by {}", err.status_code().as_ref()), + ); + // Only deterministic failures are retried span by span. + // This includes schemaless table or column creation paths for + // trace ingestion. Ambiguous failures are handled below + // without retrying because the chunk may already have been + // ingested. + self.ingest_trace_chunk_span_by_span( + ingest_ctx, + chunk, + ctx.clone(), + ingest_state, + ) + .await?; + } + ChunkFailureReaction::DiscardChunk => { + ingest_state.outcome.rejected_spans += chunk.len(); + Self::push_trace_failure_message( + &mut ingest_state.failure_messages, + ChunkFailureReaction::DiscardChunk.as_metric_label(), + format!( + "Discarded {} spans after ambiguous chunk failure ({})", + chunk.len(), + err.status_code().as_ref() + ), + ); + // TODO(shuiyisong): Add an idempotent retry-safe recovery path for + // ambiguous chunk failures such as timeout-like errors. + } + // Retryable or ambiguous failures must fail the request instead of + // becoming partial success. This path is not retry-safe because the + // chunk may already have been committed before the error surfaced. + ChunkFailureReaction::Propagate => { + Self::push_trace_failure_message( + &mut ingest_state.failure_messages, + ChunkFailureReaction::Propagate.as_metric_label(), + format!( + "Propagating retryable chunk failure ({})", + err.status_code().as_ref() + ), + ); + return Err(err); + } + }, + } + + Ok(()) + } + + /// Retry spans one by one only after a deterministic chunk failure. + async fn ingest_trace_chunk_span_by_span( + &self, + ingest_ctx: &TraceChunkIngestContext<'_>, + chunk: Vec, + ctx: QueryContextRef, + ingest_state: &mut TraceIngestState, + ) -> ServerResult<()> { + for span in chunk { + let (requests, rows) = otlp::trace::to_grpc_insert_requests_from_spans( + std::slice::from_ref(&span), + ingest_ctx.pipeline, + ingest_ctx.pipeline_params, + ingest_ctx.table_name, + &ctx, + ingest_ctx.pipeline_handler.clone(), + )?; + + match self + .insert_trace_requests(requests, ingest_ctx.is_trace_v1_model, ctx.clone()) + .await + { + Ok(output) => { + Self::add_trace_write_cost(&mut ingest_state.outcome, output.meta.cost); + ingest_state.outcome.accepted_spans += rows; + ingest_state.aux_data.observe_span(&span); + } + Err(err) => { + if Self::should_propagate_trace_span_failure(err.status_code()) { + Self::push_trace_failure_message( + &mut ingest_state.failure_messages, + ChunkFailureReaction::Propagate.as_metric_label(), + format!( + "Propagating retryable span failure for {}:{} ({})", + span.trace_id, + span.span_id, + err.status_code().as_ref() + ), + ); + return Err(err); + } + + ingest_state.outcome.rejected_spans += 1; + Self::push_trace_failure_message( + &mut ingest_state.failure_messages, + "span_rejected", + format!( + "Rejected span {}:{} ({})", + span.trace_id, + span.span_id, + err.status_code().as_ref() + ), + ); + } + } + } + + Ok(()) + } + + /// Reconcile and insert one trace request batch. + async fn insert_trace_requests( + &self, + mut requests: RowInsertRequests, + is_trace_v1_model: bool, + ctx: QueryContextRef, + ) -> ServerResult { + if is_trace_v1_model { + self.reconcile_trace_column_types(&mut requests, &ctx) + .await?; + self.handle_trace_inserts(requests, ctx) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu) + } else { + self.handle_log_inserts(requests, ctx) + .await + .map_err(BoxedError::new) + .context(error::ExecuteGrpcQuerySnafu) + } + } + + fn classify_trace_chunk_failure(status: StatusCode) -> ChunkFailureReaction { + match status { + StatusCode::InvalidArguments + | StatusCode::InvalidSyntax + | StatusCode::Unsupported + | StatusCode::TableNotFound + | StatusCode::TableColumnNotFound => ChunkFailureReaction::RetryPerSpan, + StatusCode::DatabaseNotFound => ChunkFailureReaction::DiscardChunk, + StatusCode::Cancelled | StatusCode::DeadlineExceeded => ChunkFailureReaction::Propagate, + _ if status.is_retryable() => ChunkFailureReaction::Propagate, + _ => ChunkFailureReaction::DiscardChunk, + } + } + + fn should_propagate_trace_span_failure(status: StatusCode) -> bool { + matches!( + Self::classify_trace_chunk_failure(status), + ChunkFailureReaction::Propagate + ) + } + + fn add_trace_write_cost(outcome: &mut TraceIngestOutcome, cost: usize) { + outcome.write_cost += cost; + } + + fn push_trace_failure_message(messages: &mut Vec, label: &str, message: String) { + OTLP_TRACES_FAILURE_COUNT.with_label_values(&[label]).inc(); + + if messages.len() < TRACE_FAILURE_MESSAGE_LIMIT { + messages.push(message); + } else if messages.len() == TRACE_FAILURE_MESSAGE_LIMIT { + tracing::debug!( + label, + limit = TRACE_FAILURE_MESSAGE_LIMIT, + "Trace ingest failure message limit reached; suppressing additional failure details" + ); + } + } + + fn finish_trace_failure_message( + accepted_spans: usize, + rejected_spans: usize, + messages: Vec, + ) -> Option { + if rejected_spans == 0 && messages.is_empty() { + return None; + } + + let mut summary = format!( + "Accepted {} spans, rejected {} spans", + accepted_spans, rejected_spans + ); + + if !messages.is_empty() { + summary.push_str(": "); + summary.push_str(&messages.join("; ")); + } + + Some(summary) + } + /// Picks the final datatype for one trace column. /// /// Existing table schema is authoritative when present. Otherwise we resolve the @@ -428,3 +764,163 @@ fn push_observed_trace_type(observed_types: &mut Vec, datatype: observed_types.push(datatype); } } + +#[cfg(test)] +mod tests { + use common_error::status_code::StatusCode; + use servers::query_handler::TraceIngestOutcome; + + use super::{ChunkFailureReaction, Instance}; + use crate::metrics::OTLP_TRACES_FAILURE_COUNT; + + #[test] + fn test_classify_trace_chunk_failure() { + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::InvalidArguments), + ChunkFailureReaction::RetryPerSpan + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::InvalidSyntax), + ChunkFailureReaction::RetryPerSpan + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::Unsupported), + ChunkFailureReaction::RetryPerSpan + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::TableColumnNotFound), + ChunkFailureReaction::RetryPerSpan + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::TableNotFound), + ChunkFailureReaction::RetryPerSpan + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::DatabaseNotFound), + ChunkFailureReaction::DiscardChunk + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::DeadlineExceeded), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::Cancelled), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::StorageUnavailable), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::Internal), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::RegionNotReady), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::TableUnavailable), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::RegionBusy), + ChunkFailureReaction::Propagate + ); + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::RuntimeResourcesExhausted), + ChunkFailureReaction::Propagate + ); + } + + #[test] + fn test_classify_trace_span_failure() { + assert!(Instance::should_propagate_trace_span_failure( + StatusCode::DeadlineExceeded + )); + assert!(Instance::should_propagate_trace_span_failure( + StatusCode::StorageUnavailable + )); + assert!(!Instance::should_propagate_trace_span_failure( + StatusCode::InvalidArguments + )); + } + + #[test] + fn test_add_trace_write_cost() { + let mut outcome = TraceIngestOutcome::default(); + Instance::add_trace_write_cost(&mut outcome, 3); + Instance::add_trace_write_cost(&mut outcome, 5); + assert_eq!(outcome.write_cost, 8); + } + + #[test] + fn test_finish_trace_failure_message() { + let message = Instance::finish_trace_failure_message( + 3, + 2, + vec!["Rejected span trace:span (InvalidArguments)".to_string()], + ) + .unwrap(); + assert!(message.contains("Accepted 3 spans, rejected 2 spans")); + assert!(message.contains("Rejected span trace:span")); + + assert_eq!(Instance::finish_trace_failure_message(2, 0, vec![]), None); + } + + #[test] + fn test_finish_trace_failure_message_without_detail_messages() { + assert_eq!( + Instance::finish_trace_failure_message(0, 2, vec![]), + Some("Accepted 0 spans, rejected 2 spans".to_string()) + ); + } + + #[test] + fn test_push_trace_failure_message_increments_labeled_counter() { + let label = "retry_per_span_counter_test"; + let initial = OTLP_TRACES_FAILURE_COUNT.with_label_values(&[label]).get(); + let mut messages = Vec::new(); + + Instance::push_trace_failure_message( + &mut messages, + label, + "Chunk fallback triggered by InvalidArguments".to_string(), + ); + + assert_eq!(messages.len(), 1); + assert_eq!( + OTLP_TRACES_FAILURE_COUNT.with_label_values(&[label]).get(), + initial + 1 + ); + } + + #[test] + fn test_push_trace_failure_message_caps_recorded_messages() { + let label = "retry_per_span_limit_test"; + let mut messages = Vec::new(); + + for idx in 0..=4 { + Instance::push_trace_failure_message(&mut messages, label, format!("failure-{idx}")); + } + + assert_eq!(messages.len(), 4); + assert_eq!( + messages, + vec![ + "failure-0".to_string(), + "failure-1".to_string(), + "failure-2".to_string(), + "failure-3".to_string() + ] + ); + } + + #[test] + fn test_classify_trace_chunk_failure_defaults_to_discard() { + assert_eq!( + Instance::classify_trace_chunk_failure(StatusCode::Unknown), + ChunkFailureReaction::DiscardChunk + ); + } +} diff --git a/src/frontend/src/metrics.rs b/src/frontend/src/metrics.rs index 58ba21476a..aba33637cf 100644 --- a/src/frontend/src/metrics.rs +++ b/src/frontend/src/metrics.rs @@ -52,6 +52,14 @@ lazy_static! { ) .unwrap(); + /// The number of OpenTelemetry trace ingest failures on the frontend node. + pub static ref OTLP_TRACES_FAILURE_COUNT: IntCounterVec = register_int_counter_vec!( + "greptime_frontend_otlp_traces_failure_count", + "frontend otlp trace ingest failure count", + &["label"] + ) + .unwrap(); + /// The number of OpenTelemetry logs send by frontend node. pub static ref OTLP_LOGS_ROWS: IntCounter = register_int_counter!( "greptime_frontend_otlp_logs_rows", diff --git a/src/servers/src/http/otlp.rs b/src/servers/src/http/otlp.rs index 4fd2d42122..3d6057f046 100644 --- a/src/servers/src/http/otlp.rs +++ b/src/servers/src/http/otlp.rs @@ -29,7 +29,7 @@ use opentelemetry_proto::tonic::collector::logs::v1::{ }; use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceResponse; use opentelemetry_proto::tonic::collector::trace::v1::{ - ExportTraceServiceRequest, ExportTraceServiceResponse, + ExportTracePartialSuccess, ExportTraceServiceRequest, ExportTraceServiceResponse, }; use otel_arrow_rust::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest; use pipeline::PipelineWay; @@ -175,11 +175,16 @@ pub async fn traces( query_ctx, ) .await - .map(|o| OtlpResponse { + .map(|outcome| OtlpResponse { resp_body: ExportTraceServiceResponse { - partial_success: None, + partial_success: outcome.error_message.map(|error_message| { + ExportTracePartialSuccess { + rejected_spans: outcome.rejected_spans as i64, + error_message, + } + }), }, - write_cost: o.meta.cost, + write_cost: outcome.write_cost, }) } diff --git a/src/servers/src/otlp/trace.rs b/src/servers/src/otlp/trace.rs index ca56f9b868..98f4441923 100644 --- a/src/servers/src/otlp/trace.rs +++ b/src/servers/src/otlp/trace.rs @@ -18,15 +18,17 @@ pub mod span; pub mod v0; pub mod v1; +use std::collections::HashSet; + use api::v1::RowInsertRequests; pub use common_catalog::consts::{ PARENT_SPAN_ID_COLUMN, SPAN_ID_COLUMN, SPAN_NAME_COLUMN, TRACE_ID_COLUMN, }; -use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; use pipeline::{GreptimePipelineParams, PipelineWay}; use session::context::QueryContextRef; use crate::error::{NotSupportedSnafu, Result}; +use crate::otlp::trace::span::TraceSpan; use crate::query_handler::PipelineHandlerRef; // column names @@ -65,27 +67,58 @@ pub const SPAN_STATUS_PREFIX: &str = "STATUS_CODE_"; pub const SPAN_STATUS_UNSET: &str = "STATUS_CODE_UNSET"; pub const SPAN_STATUS_ERROR: &str = "STATUS_CODE_ERROR"; -/// Convert SpanTraces to GreptimeDB row insert requests. -/// Returns `InsertRequests` and total number of rows to ingest -pub fn to_grpc_insert_requests( - request: ExportTraceServiceRequest, - pipeline: PipelineWay, - pipeline_params: GreptimePipelineParams, - table_name: String, +/// Deduplicated auxiliary trace entities derived from successfully ingested +/// spans. +/// +/// The main trace table is written first. Once a span is confirmed accepted, we +/// record the service and operation tuples here so the auxiliary tables can be +/// updated separately without affecting span acceptance accounting. +#[derive(Debug, Default)] +pub struct TraceAuxData { + pub services: HashSet, + pub operations: HashSet<(String, String, String)>, +} + +impl TraceAuxData { + /// Records the auxiliary service and operation rows implied by one accepted + /// span. + pub fn observe_span(&mut self, span: &TraceSpan) { + if let Some(service_name) = &span.service_name { + self.services.insert(service_name.clone()); + self.operations.insert(( + service_name.clone(), + span.span_name.clone(), + span.span_kind.clone(), + )); + } + } + + /// Returns true when no auxiliary table updates are needed. + pub fn is_empty(&self) -> bool { + self.services.is_empty() && self.operations.is_empty() + } +} + +/// Convert a subset of trace spans to GreptimeDB row insert requests. +pub fn to_grpc_insert_requests_from_spans( + spans: &[TraceSpan], + pipeline: &PipelineWay, + pipeline_params: &GreptimePipelineParams, + table_name: &str, query_ctx: &QueryContextRef, pipeline_handler: PipelineHandlerRef, ) -> Result<(RowInsertRequests, usize)> { match pipeline { - PipelineWay::OtlpTraceDirectV0 => v0::v0_to_grpc_insert_requests( - request, + PipelineWay::OtlpTraceDirectV0 => v0::v0_to_grpc_main_insert_requests( + spans, pipeline, pipeline_params, table_name, query_ctx, pipeline_handler, ), - PipelineWay::OtlpTraceDirectV1 => v1::v1_to_grpc_insert_requests( - request, + PipelineWay::OtlpTraceDirectV1 => v1::v1_to_grpc_main_insert_requests( + spans, pipeline, pipeline_params, table_name, @@ -98,3 +131,23 @@ pub fn to_grpc_insert_requests( .fail(), } } + +/// Build insert requests for the auxiliary trace tables derived from accepted +/// spans. +/// +/// "Aux" here refers to the trace service and trace operation tables, not the +/// main trace span table itself. +pub fn to_grpc_insert_requests_for_aux_tables( + aux_data: TraceAuxData, + pipeline: &PipelineWay, + table_name: &str, +) -> Result<(RowInsertRequests, usize)> { + match pipeline { + PipelineWay::OtlpTraceDirectV0 => v0::build_aux_table_requests(aux_data, table_name), + PipelineWay::OtlpTraceDirectV1 => v1::build_aux_table_requests(aux_data, table_name), + _ => NotSupportedSnafu { + feat: "Unsupported pipeline for trace", + } + .fail(), + } +} diff --git a/src/servers/src/otlp/trace/span.rs b/src/servers/src/otlp/trace/span.rs index d96bc17277..19103240f6 100644 --- a/src/servers/src/otlp/trace/span.rs +++ b/src/servers/src/otlp/trace/span.rs @@ -53,6 +53,18 @@ pub struct TraceSpan { pub type TraceSpans = Vec; +#[derive(Debug, Clone)] +pub struct TraceSpanGroup { + pub service_name: Option, + pub resource_attributes: Attributes, + pub scope_name: String, + pub scope_version: String, + pub scope_attributes: Attributes, + pub spans: TraceSpans, +} + +pub type TraceSpanGroups = Vec; + #[derive(Debug, Clone, Serialize)] pub struct SpanLink { pub trace_id: String, @@ -241,14 +253,13 @@ pub fn status_to_string(status: &Option) -> (String, String) { /// See /// /// for data structure of OTLP traces. -pub fn parse(request: ExportTraceServiceRequest) -> TraceSpans { - let span_size = request +pub fn parse(request: ExportTraceServiceRequest) -> TraceSpanGroups { + let group_size = request .resource_spans .iter() .flat_map(|res| res.scope_spans.iter()) - .flat_map(|scope| scope.spans.iter()) .count(); - let mut spans = Vec::with_capacity(span_size); + let mut groups = Vec::with_capacity(group_size); for resource_spans in request.resource_spans { let resource_attrs = resource_spans .resource @@ -268,6 +279,7 @@ pub fn parse(request: ExportTraceServiceRequest) -> TraceSpans { for scope_spans in resource_spans.scope_spans { let scope = scope_spans.scope.unwrap_or_default(); + let mut spans = Vec::with_capacity(scope_spans.spans.len()); for span in scope_spans.spans { spans.push(parse_span( service_name.clone(), @@ -276,16 +288,47 @@ pub fn parse(request: ExportTraceServiceRequest) -> TraceSpans { span, )); } + groups.push(TraceSpanGroup { + service_name: service_name.clone(), + resource_attributes: Attributes::from(&resource_attrs[..]), + scope_name: scope.name, + scope_version: scope.version, + scope_attributes: Attributes::from(scope.attributes), + spans, + }); } } - spans + groups } #[cfg(test)] mod tests { - use opentelemetry_proto::tonic::trace::v1::Status; + use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; + use opentelemetry_proto::tonic::common::v1::{ + AnyValue, InstrumentationScope, KeyValue, any_value, + }; + use opentelemetry_proto::tonic::resource::v1::Resource; + use opentelemetry_proto::tonic::trace::v1::{ResourceSpans, ScopeSpans, Span, Status}; - use crate::otlp::trace::span::{bytes_to_hex_string, status_to_string}; + use crate::otlp::trace::KEY_SERVICE_NAME; + use crate::otlp::trace::span::{bytes_to_hex_string, parse, status_to_string}; + + fn make_kv(key: &str, value: &str) -> KeyValue { + KeyValue { + key: key.to_string(), + value: Some(AnyValue { + value: Some(any_value::Value::StringValue(value.to_string())), + }), + } + } + + fn make_span(trace_id: u8, span_id: u8) -> Span { + Span { + trace_id: vec![trace_id; 16], + span_id: vec![span_id; 8], + ..Default::default() + } + } #[test] fn test_bytes_to_hex_string() { @@ -315,4 +358,62 @@ mod tests { status_to_string(&Some(status)), ); } + + #[test] + fn test_parse_preserves_resource_scope_groups() { + let request = ExportTraceServiceRequest { + resource_spans: vec![ + ResourceSpans { + resource: Some(Resource { + attributes: vec![make_kv(KEY_SERVICE_NAME, "svc-a")], + ..Default::default() + }), + scope_spans: vec![ + ScopeSpans { + scope: Some(InstrumentationScope { + name: "scope-1".to_string(), + ..Default::default() + }), + spans: vec![make_span(0x11, 0x21), make_span(0x12, 0x22)], + ..Default::default() + }, + ScopeSpans { + scope: Some(InstrumentationScope { + name: "scope-2".to_string(), + ..Default::default() + }), + spans: vec![make_span(0x13, 0x23)], + ..Default::default() + }, + ], + ..Default::default() + }, + ResourceSpans { + resource: Some(Resource { + attributes: vec![make_kv(KEY_SERVICE_NAME, "svc-b")], + ..Default::default() + }), + scope_spans: vec![ScopeSpans { + scope: Some(InstrumentationScope { + name: "scope-3".to_string(), + ..Default::default() + }), + spans: vec![make_span(0x14, 0x24)], + ..Default::default() + }], + ..Default::default() + }, + ], + }; + + let groups = parse(request); + assert_eq!(groups.len(), 3); + assert_eq!(groups[0].service_name.as_deref(), Some("svc-a")); + assert_eq!(groups[0].scope_name, "scope-1"); + assert_eq!(groups[0].spans.len(), 2); + assert_eq!(groups[1].scope_name, "scope-2"); + assert_eq!(groups[1].spans.len(), 1); + assert_eq!(groups[2].service_name.as_deref(), Some("svc-b")); + assert_eq!(groups[2].scope_name, "scope-3"); + } } diff --git a/src/servers/src/otlp/trace/v0.rs b/src/servers/src/otlp/trace/v0.rs index b52b406fb2..fa10dcc00f 100644 --- a/src/servers/src/otlp/trace/v0.rs +++ b/src/servers/src/otlp/trace/v0.rs @@ -18,16 +18,16 @@ use api::v1::value::ValueData; use api::v1::{ColumnDataType, RowInsertRequests}; use common_catalog::consts::{trace_operations_table_name, trace_services_table_name}; use common_grpc::precision::Precision; -use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; use pipeline::{GreptimePipelineParams, PipelineWay}; use session::context::QueryContextRef; use crate::error::Result; -use crate::otlp::trace::span::{TraceSpan, parse}; +use crate::otlp::trace::span::TraceSpan; use crate::otlp::trace::{ DURATION_NANO_COLUMN, PARENT_SPAN_ID_COLUMN, SERVICE_NAME_COLUMN, SPAN_ATTRIBUTES_COLUMN, SPAN_EVENTS_COLUMN, SPAN_ID_COLUMN, SPAN_KIND_COLUMN, SPAN_NAME_COLUMN, SPAN_STATUS_CODE, SPAN_STATUS_MESSAGE_COLUMN, TIMESTAMP_COLUMN, TRACE_ID_COLUMN, TRACE_STATE_COLUMN, + TraceAuxData, }; use crate::otlp::utils::{make_column_data, make_string_column_data}; use crate::query_handler::PipelineHandlerRef; @@ -38,56 +38,52 @@ const APPROXIMATE_COLUMN_COUNT: usize = 24; // Use a timestamp(2100-01-01 00:00:00) as large as possible. const MAX_TIMESTAMP: i64 = 4102444800000000000; -/// Convert SpanTraces to GreptimeDB row insert requests. -/// Returns `InsertRequests` and total number of rows to ingest -pub fn v0_to_grpc_insert_requests( - request: ExportTraceServiceRequest, - _pipeline: PipelineWay, - _pipeline_params: GreptimePipelineParams, - table_name: String, +/// Converts trace spans into row insert requests for the main v0 trace table. +/// +/// Auxiliary service and operation table writes are built separately so the +/// caller can update them only after the main span write succeeds. +pub fn v0_to_grpc_main_insert_requests( + spans: &[TraceSpan], + _pipeline: &PipelineWay, + _pipeline_params: &GreptimePipelineParams, + table_name: &str, _query_ctx: &QueryContextRef, _pipeline_handler: PipelineHandlerRef, ) -> Result<(RowInsertRequests, usize)> { - let spans = parse(request); let mut multi_table_writer = MultiTableData::default(); + let trace_writer = build_trace_table_data(spans)?; + multi_table_writer.add_table_data(table_name, trace_writer); + + Ok(multi_table_writer.into_row_insert_requests()) +} + +/// Builds the row-oriented payload for the main v0 trace table. +pub fn build_trace_table_data(spans: &[TraceSpan]) -> Result { let mut trace_writer = TableData::new(APPROXIMATE_COLUMN_COUNT, spans.len()); + for span in spans.iter().cloned() { + write_span_to_row(&mut trace_writer, span)?; + } + + Ok(trace_writer) +} + +/// Builds row insert requests for the v0 trace auxiliary tables. +pub fn build_aux_table_requests( + aux_data: TraceAuxData, + table_name: &str, +) -> Result<(RowInsertRequests, usize)> { + let mut multi_table_writer = MultiTableData::default(); let mut trace_services_writer = TableData::new(APPROXIMATE_COLUMN_COUNT, 1); let mut trace_operations_writer = TableData::new(APPROXIMATE_COLUMN_COUNT, 1); - let mut services = HashSet::new(); - let mut operations = HashSet::new(); - for span in spans { - if let Some(service_name) = &span.service_name { - // Only insert the service name if it's not already in the set. - if !services.contains(service_name) { - services.insert(service_name.clone()); - } - - // Collect operations (service_name + span_name + span_kind). - let operation = ( - service_name.clone(), - span.span_name.clone(), - span.span_kind.clone(), - ); - if !operations.contains(&operation) { - operations.insert(operation); - } - } - write_span_to_row(&mut trace_writer, span)?; - } - write_trace_services_to_row(&mut trace_services_writer, services)?; - write_trace_operations_to_row(&mut trace_operations_writer, operations)?; + write_trace_services_to_row(&mut trace_services_writer, aux_data.services)?; + write_trace_operations_to_row(&mut trace_operations_writer, aux_data.operations)?; + multi_table_writer.add_table_data(trace_services_table_name(table_name), trace_services_writer); multi_table_writer.add_table_data( - trace_services_table_name(&table_name), - trace_services_writer, - ); - multi_table_writer.add_table_data( - trace_operations_table_name(&table_name), + trace_operations_table_name(table_name), trace_operations_writer, ); - multi_table_writer.add_table_data(table_name, trace_writer); - Ok(multi_table_writer.into_row_insert_requests()) } @@ -232,3 +228,63 @@ fn write_trace_operations_to_row( Ok(()) } + +#[cfg(test)] +mod tests { + use super::{build_aux_table_requests, build_trace_table_data}; + use crate::otlp::trace::TraceAuxData; + use crate::otlp::trace::attributes::Attributes; + use crate::otlp::trace::span::{SpanEvents, SpanLinks, TraceSpan}; + + fn make_span(service_name: &str, trace_id: &str, span_id: &str) -> TraceSpan { + TraceSpan { + service_name: Some(service_name.to_string()), + trace_id: trace_id.to_string(), + span_id: span_id.to_string(), + parent_span_id: None, + resource_attributes: Attributes::from(vec![]), + scope_name: "scope".to_string(), + scope_version: "v1".to_string(), + scope_attributes: Attributes::from(vec![]), + trace_state: String::new(), + span_name: "op".to_string(), + span_kind: "SPAN_KIND_SERVER".to_string(), + span_status_code: "STATUS_CODE_UNSET".to_string(), + span_status_message: String::new(), + span_attributes: Attributes::from(vec![]), + span_events: SpanEvents::from(vec![]), + span_links: SpanLinks::from(vec![]), + start_in_nanosecond: 1, + end_in_nanosecond: 2, + } + } + + #[test] + fn test_build_trace_table_data_from_span_subset() { + let spans = [ + make_span("svc-a", "trace-a", "span-a"), + make_span("svc-b", "trace-b", "span-b"), + ]; + + let writer = build_trace_table_data(&spans[..1]).unwrap(); + let (_, rows) = writer.into_schema_and_rows(); + assert_eq!(rows.len(), 1); + } + + #[test] + fn test_build_aux_table_requests_deduplicates_services_and_operations() { + let spans = vec![ + make_span("svc-a", "trace-a", "span-a"), + make_span("svc-a", "trace-b", "span-b"), + ]; + let mut aux_data = TraceAuxData::default(); + for span in &spans { + aux_data.observe_span(span); + } + + let (requests, total_rows) = + build_aux_table_requests(aux_data, "opentelemetry_traces").unwrap(); + assert_eq!(requests.inserts.len(), 2); + assert_eq!(total_rows, 2); + } +} diff --git a/src/servers/src/otlp/trace/v1.rs b/src/servers/src/otlp/trace/v1.rs index 11e986de04..cce6891b0d 100644 --- a/src/servers/src/otlp/trace/v1.rs +++ b/src/servers/src/otlp/trace/v1.rs @@ -18,19 +18,18 @@ use api::v1::value::ValueData; use api::v1::{ColumnDataType, RowInsertRequests, Value}; use common_catalog::consts::{trace_operations_table_name, trace_services_table_name}; use common_grpc::precision::Precision; -use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; use opentelemetry_proto::tonic::common::v1::any_value::Value as OtlpValue; use pipeline::{GreptimePipelineParams, PipelineWay}; use session::context::QueryContextRef; use crate::error::Result; use crate::otlp::trace::attributes::Attributes; -use crate::otlp::trace::span::{TraceSpan, parse}; +use crate::otlp::trace::span::TraceSpan; use crate::otlp::trace::{ DURATION_NANO_COLUMN, KEY_SERVICE_NAME, PARENT_SPAN_ID_COLUMN, SCOPE_NAME_COLUMN, SCOPE_VERSION_COLUMN, SERVICE_NAME_COLUMN, SPAN_EVENTS_COLUMN, SPAN_ID_COLUMN, SPAN_KIND_COLUMN, SPAN_NAME_COLUMN, SPAN_STATUS_CODE, SPAN_STATUS_MESSAGE_COLUMN, - TIMESTAMP_COLUMN, TRACE_ID_COLUMN, TRACE_STATE_COLUMN, + TIMESTAMP_COLUMN, TRACE_ID_COLUMN, TRACE_STATE_COLUMN, TraceAuxData, }; use crate::otlp::utils::{any_value_to_jsonb, make_column_data, make_string_column_data}; use crate::query_handler::PipelineHandlerRef; @@ -41,64 +40,52 @@ const APPROXIMATE_COLUMN_COUNT: usize = 30; // Use a timestamp(2100-01-01 00:00:00) as large as possible. const MAX_TIMESTAMP: i64 = 4102444800000000000; -/// Convert SpanTraces to GreptimeDB row insert requests. -/// Returns `InsertRequests` and total number of rows to ingest +/// Converts trace spans into row insert requests for the main v1 trace table. /// -/// Compared with v0, this v1 implementation: -/// 1. flattens all attribute data into columns. -/// 2. treat `span_id` and `parent_trace_id` as fields. -/// 3. removed `service_name` column because it's already in -/// `resource_attributes.service_name` -/// -/// For other compound data structures like span_links and span_events here we -/// are still using `json` data structure. -pub fn v1_to_grpc_insert_requests( - request: ExportTraceServiceRequest, - _pipeline: PipelineWay, - _pipeline_params: GreptimePipelineParams, - table_name: String, +/// Auxiliary service and operation table writes are built separately so the +/// caller can update them only after the main span write succeeds. +pub fn v1_to_grpc_main_insert_requests( + spans: &[TraceSpan], + _pipeline: &PipelineWay, + _pipeline_params: &GreptimePipelineParams, + table_name: &str, _query_ctx: &QueryContextRef, _pipeline_handler: PipelineHandlerRef, ) -> Result<(RowInsertRequests, usize)> { - let spans = parse(request); let mut multi_table_writer = MultiTableData::default(); + let trace_writer = build_trace_table_data(spans)?; + multi_table_writer.add_table_data(table_name, trace_writer); + + Ok(multi_table_writer.into_row_insert_requests()) +} + +/// Builds the row-oriented payload for the main v1 trace table. +pub fn build_trace_table_data(spans: &[TraceSpan]) -> Result { let mut trace_writer = TableData::new(APPROXIMATE_COLUMN_COUNT, spans.len()); + for span in spans.iter().cloned() { + write_span_to_row(&mut trace_writer, span)?; + } + + Ok(trace_writer) +} + +/// Builds row insert requests for the v1 trace auxiliary tables. +pub fn build_aux_table_requests( + aux_data: TraceAuxData, + table_name: &str, +) -> Result<(RowInsertRequests, usize)> { + let mut multi_table_writer = MultiTableData::default(); let mut trace_services_writer = TableData::new(APPROXIMATE_COLUMN_COUNT, 1); let mut trace_operations_writer = TableData::new(APPROXIMATE_COLUMN_COUNT, 1); - let mut services = HashSet::new(); - let mut operations = HashSet::new(); - for span in spans { - if let Some(service_name) = &span.service_name { - // Only insert the service name if it's not already in the set. - if !services.contains(service_name) { - services.insert(service_name.clone()); - } - - // Only insert the operation if it's not already in the set. - let operation = ( - service_name.clone(), - span.span_name.clone(), - span.span_kind.clone(), - ); - if !operations.contains(&operation) { - operations.insert(operation); - } - } - write_span_to_row(&mut trace_writer, span)?; - } - write_trace_services_to_row(&mut trace_services_writer, services)?; - write_trace_operations_to_row(&mut trace_operations_writer, operations)?; + write_trace_services_to_row(&mut trace_services_writer, aux_data.services)?; + write_trace_operations_to_row(&mut trace_operations_writer, aux_data.operations)?; + multi_table_writer.add_table_data(trace_services_table_name(table_name), trace_services_writer); multi_table_writer.add_table_data( - trace_services_table_name(&table_name), - trace_services_writer, - ); - multi_table_writer.add_table_data( - trace_operations_table_name(&table_name), + trace_operations_table_name(table_name), trace_operations_writer, ); - multi_table_writer.add_table_data(table_name, trace_writer); Ok(multi_table_writer.into_row_insert_requests()) } @@ -319,7 +306,9 @@ mod tests { use opentelemetry_proto::tonic::common::v1::{AnyValue, KeyValue}; use super::*; + use crate::otlp::trace::TraceAuxData; use crate::otlp::trace::attributes::Attributes; + use crate::otlp::trace::span::{SpanEvents, SpanLinks}; use crate::row_writer::TableData; fn make_kv(key: &str, value: OtlpValue) -> KeyValue { @@ -329,6 +318,29 @@ mod tests { } } + fn make_span(service_name: &str, trace_id: &str, span_id: &str) -> TraceSpan { + TraceSpan { + service_name: Some(service_name.to_string()), + trace_id: trace_id.to_string(), + span_id: span_id.to_string(), + parent_span_id: None, + resource_attributes: Attributes::from(vec![]), + scope_name: "scope".to_string(), + scope_version: "v1".to_string(), + scope_attributes: Attributes::from(vec![]), + trace_state: String::new(), + span_name: "op".to_string(), + span_kind: "SPAN_KIND_SERVER".to_string(), + span_status_code: "STATUS_CODE_UNSET".to_string(), + span_status_message: String::new(), + span_attributes: Attributes::from(vec![]), + span_events: SpanEvents::from(vec![]), + span_links: SpanLinks::from(vec![]), + start_in_nanosecond: 1, + end_in_nanosecond: 2, + } + } + #[test] fn test_keep_mixed_numeric_values_until_frontend_reconciliation() { let mut writer = TableData::new(4, 2); @@ -520,5 +532,22 @@ mod tests { Some(ValueData::StringValue("false".to_string())) ); } + + #[test] + fn test_build_aux_table_requests_deduplicates_services_and_operations() { + let spans = vec![ + make_span("svc-a", "trace-a", "span-a"), + make_span("svc-a", "trace-b", "span-b"), + ]; + let mut aux_data = TraceAuxData::default(); + for span in &spans { + aux_data.observe_span(span); + } + + let (requests, total_rows) = + build_aux_table_requests(aux_data, "opentelemetry_traces").unwrap(); + assert_eq!(requests.inserts.len(), 2); + assert_eq!(total_rows, 2); + } // Conversion matrix coverage lives in the shared coercion helper tests. } diff --git a/src/servers/src/query_handler.rs b/src/servers/src/query_handler.rs index b55502e742..d4b272de12 100644 --- a/src/servers/src/query_handler.rs +++ b/src/servers/src/query_handler.rs @@ -63,6 +63,14 @@ pub type PipelineHandlerRef = Arc; pub type LogQueryHandlerRef = Arc; pub type JaegerQueryHandlerRef = Arc; +#[derive(Debug, Default, Clone)] +pub struct TraceIngestOutcome { + pub write_cost: usize, + pub accepted_spans: usize, + pub rejected_spans: usize, + pub error_message: Option, +} + #[async_trait] pub trait InfluxdbLineProtocolHandler { /// A successful request will not return a response. @@ -123,7 +131,7 @@ pub trait OpenTelemetryProtocolHandler: PipelineHandler { pipeline_params: GreptimePipelineParams, table_name: String, ctx: QueryContextRef, - ) -> Result; + ) -> Result; async fn logs( &self, diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index c0d858a592..36ddb1bb38 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -40,7 +40,9 @@ use loki_proto::logproto::{EntryAdapter, LabelPairAdapter, PushRequest, StreamAd use loki_proto::prost_types::Timestamp; use opentelemetry_proto::tonic::collector::logs::v1::ExportLogsServiceRequest; use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceRequest; -use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest; +use opentelemetry_proto::tonic::collector::trace::v1::{ + ExportTraceServiceRequest, ExportTraceServiceResponse, +}; use pipeline::GREPTIME_INTERNAL_TRACE_PIPELINE_V1_NAME; use prost::Message; use serde_json::{Value, json}; @@ -5572,26 +5574,83 @@ pub async fn test_otlp_traces_v1(store_type: StorageType) { ], ); let res = send_trace_v1_req(&client, abort_table_name, abort_req, false).await; - assert_eq!(StatusCode::BAD_REQUEST, res.status()); - let body: Value = res.json().await; + assert_eq!(StatusCode::OK, res.status()); + let body = ExportTraceServiceResponse::decode(res.bytes().await).unwrap(); + let partial_success = body.partial_success.as_ref().unwrap(); + assert_eq!(partial_success.rejected_spans, 1); assert!( - body["error"].as_str().unwrap().contains( - "failed to coerce trace column 'span_attributes.attr_int' in table 'trace_type_abort'" + partial_success + .error_message + .contains("Accepted 1 spans, rejected 1 spans"), + "unexpected partial success body: {body:?}" + ); + assert!( + partial_success.error_message.contains( + "Rejected span 00000000000000000000000000000013:0000000000000013 (InvalidArguments)" ), - "unexpected error body: {body}" + "unexpected partial success body: {body:?}" ); validate_data( "otlp_traces_v1_type_abort_rows", &client, &format!( - "select trace_id, \"span_attributes.attr_int\" from {} order by trace_id;", + "select trace_id, \"span_attributes.attr_int\" from {} order by trace_id", abort_table_name ), - r#"[["00000000000000000000000000000011",10]]"#, + r#"[["00000000000000000000000000000011",10],["00000000000000000000000000000012",20]]"#, ) .await; + let chunk_failure_req = make_trace_v1_request( + "type-discard", + vec![ + make_trace_v1_span( + "00000000000000000000000000000021", + "0000000000000021", + "discard-one", + 1_736_480_942_445_400_000, + 1_736_480_942_445_500_000, + vec![make_string_attr("attr_text", "alpha")], + ), + make_trace_v1_span( + "00000000000000000000000000000022", + "0000000000000022", + "discard-two", + 1_736_480_942_445_600_000, + 1_736_480_942_445_700_000, + vec![make_string_attr("attr_text", "beta")], + ), + ], + ); + let res = send_trace_v1_req_with_db( + &client, + "nonexistent", + "trace_chunk_discard", + chunk_failure_req, + false, + ) + .await; + assert_eq!(StatusCode::OK, res.status()); + let body = ExportTraceServiceResponse::decode(res.bytes().await).unwrap(); + let partial_success = body.partial_success.as_ref().unwrap(); + assert_eq!(partial_success.rejected_spans, 2); + assert!( + partial_success + .error_message + .contains("Accepted 0 spans, rejected 2 spans"), + "unexpected partial success body: {body:?}" + ); + assert!( + partial_success + .error_message + .contains("Chunk fallback triggered by") + || partial_success + .error_message + .contains("Discarded 2 spans after ambiguous chunk failure"), + "unexpected partial success body: {body:?}" + ); + guard.remove_all().await; } @@ -7829,6 +7888,16 @@ async fn send_trace_v1_req( table_name: &str, req: ExportTraceServiceRequest, with_gzip: bool, +) -> TestResponse { + send_trace_v1_req_with_db(client, "public", table_name, req, with_gzip).await +} + +async fn send_trace_v1_req_with_db( + client: &TestClient, + db_name: &str, + table_name: &str, + req: ExportTraceServiceRequest, + with_gzip: bool, ) -> TestResponse { send_req( client, @@ -7845,6 +7914,10 @@ async fn send_trace_v1_req( HeaderName::from_static("x-greptime-trace-table-name"), HeaderValue::from_str(table_name).unwrap(), ), + ( + GREPTIME_DB_HEADER_NAME.clone(), + HeaderValue::from_str(db_name).unwrap(), + ), ], "/v1/otlp/v1/traces", req.encode_to_vec(),