From 0ef54511f710f0ef2c05941c8c600bb4c1fd46c8 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Thu, 14 May 2026 20:18:55 +0800 Subject: [PATCH] chore: pick fixes and bump version to v1.0.2 (#8116) * fix: window sort off by one precision TimeRange&better alias track (#8019) * fix: window sort track alias&off by one precision TimeRange Signed-off-by: discord9 * chore: more test Signed-off-by: discord9 * refactor: clear helper Signed-off-by: discord9 * dedup a bit Signed-off-by: discord9 * feat: even more guard Signed-off-by: discord9 * fix: case insensitive Signed-off-by: discord9 --------- Signed-off-by: discord9 (cherry picked from commit 9fafd879ed1f11a25b7c74dee02df83e2439d93a) Signed-off-by: evenyag * fix(server): describe EXPLAIN statements so bind parameters work (#8035) * fix(server): describe EXPLAIN statements so bind parameters work `do_describe_inner` only planned `Insert`/`Query`/`Delete`, so `EXPLAIN` and `EXPLAIN ANALYZE` fell through to the non-plan branch and had no parameter-type inference. At Bind time the Postgres handler then reported `unsupported_parameter_type` even though the inner query would have worked on its own. Recurse one level into `Statement::Explain` so that an EXPLAIN wrapping a plannable statement goes through the same describe path. Adds a tokio-postgres integration test that exercises `EXPLAIN`/`EXPLAIN ANALYZE` over the extended query protocol. Fixes #8029 Signed-off-by: BootstrapperSBL * refactor(server): extract plannable-inner check into closure Reduce duplication between the direct match and the EXPLAIN inner match by factoring out is_inner_plannable. Behaviour unchanged. Signed-off-by: BootstrapperSBL --------- Signed-off-by: BootstrapperSBL (cherry picked from commit 793545d8e6a71745ca747be0ef6e1d47b430c39d) Signed-off-by: evenyag * fix: windows windowed sort ci (#8039) * fix: windows windowed sort ci Signed-off-by: discord9 * chore Signed-off-by: discord9 * c Signed-off-by: discord9 --------- Signed-off-by: discord9 (cherry picked from commit 760581b2a0344010e7de3eecdb2d26a5d48e229e) Signed-off-by: evenyag * fix: batched prometheus ingest row metric (#8054) * fix: count batched prometheus ingest rows Signed-off-by: Lei, HUANG * fix: align batched ingest metrics Use actual affected rows when updating `DIST_INGEST_ROW_COUNT` and cache the flush database label to avoid repeated `get_db_string` allocation. Files: `src/servers/src/pending_rows_batcher.rs` Signed-off-by: Lei, HUANG --------- Signed-off-by: Lei, HUANG (cherry picked from commit f0b3ee4830b3db5c444f22b43130a5bd6dd3a15f) Signed-off-by: evenyag * fix: preserve case in database name from connection string (#8062) `parse_optional_catalog_and_schema_from_db_string` unconditionally lowercased database/schema names, causing quoted database names (e.g. `CREATE DATABASE "TestQuery"`) to be stored with preserved case but looked up as lowercase on connection, resulting in "Database not found". Fixes #8059 Signed-off-by: Lei, HUANG (cherry picked from commit f5c1d5d9bccbf4a70de93f39d34f6815ad6c39db) Signed-off-by: evenyag * fix(metric-engine): validate column types and require time index in verify_rows (#8018) * fix(metric-engine): validate column types and require time index in verify_rows The remote-write path into the metric engine previously bypassed schema validation. When a row's time index column carried a non-timestamp datatype (e.g. a string), the request reached mito's ValueBuilder::push for the timestamp builder and panicked instead of surfacing a typed error. Cache the (column_id, data_type, semantic_type) tuple for each physical column on PhysicalRegionState and use it in verify_rows to: - reject columns whose datatype or semantic type disagrees with the physical region's schema (mirrors mito's WriteRequest::check_schema) - reject requests that omit the time index column entirely Field columns stay optional; tag completeness needs per-logical-region metadata that verify_rows doesn't have and is left to a follow-up. Fixes #7990. Signed-off-by: BootstrapperSBL * refactor(metric-engine): simplify PhysicalColumnInfo construction - Add From and From<&ColumnMetadata> for PhysicalColumnInfo so call sites can use metadata.into() instead of repeating the field list. - Replace the four struct-literal constructions in create.rs, open.rs and alter.rs with the conversion. - In verify_rows, pass &col.column_name to ColumnNotFoundSnafu instead of cloning it explicitly (snafu's context handles the conversion). Signed-off-by: BootstrapperSBL * perf(metric-engine): cache time index column name in PhysicalRegionState verify_rows previously scanned every physical column on each row batch to find the timestamp column. Since the time index is fixed at region creation and never changes, stash its name on PhysicalRegionState when the region is first registered and read it directly from there. add_physical_columns carries a debug_assert to document the invariant that alter never introduces a new time index. Signed-off-by: BootstrapperSBL * perf(metric-engine): borrow physical column names when building name_to_id On the row-write path we built a HashMap by cloning every column name out of the physical region's cached state. The map is scoped to the block that holds the state's read guard, so there's no need to own the keys. Switch the map to HashMap<&str, ColumnId> and widen RowsIter::new / IterIndex::new to accept any key type that borrows as str. Existing test helpers that pass HashMap keep working through the Borrow bound. Signed-off-by: BootstrapperSBL * fix: validate metric rows against physical schema Cache physical column metadata in the metric engine state so row validation and row modification can use the same source of truth for column IDs, data types, and semantic types. Validate incoming metric rows against the physical schema before writes. Put requests now require the time index and the expected field column, while delete requests keep accepting primary-key-plus-timestamp payloads by skipping the field completeness check. Pass physical column metadata directly into RowsIter instead of rebuilding a name-to-column-id map at each call site, and cover the new validation paths with tests for missing time indexes, missing fields, and duplicate field columns. Signed-off-by: evenyag * fix: do not allow adding a new field Signed-off-by: evenyag * fix: fill default value for fields Signed-off-by: evenyag * fix: fill default for nullable fields Signed-off-by: evenyag --------- Signed-off-by: BootstrapperSBL Signed-off-by: evenyag Co-authored-by: BootstrapperSBL Co-authored-by: evenyag (cherry picked from commit d1873ca31dc7ee4d4220d44ca6d756106a848cd2) Signed-off-by: evenyag * fix: type inference for sql rewrite (#8052) fix: type inference for rewrited sql (cherry picked from commit 5b47ec24ecc31ea9a3e548bd1450c9725dbf8b10) Signed-off-by: evenyag * fix: infer time index from column meta on derived table (#8013) * rough fix Signed-off-by: Ruihang Xia * reorganize Signed-off-by: Ruihang Xia * simplification Signed-off-by: Ruihang Xia * fix format Signed-off-by: Ruihang Xia * add comment Signed-off-by: Ruihang Xia * enhance default by infer Signed-off-by: Ruihang Xia * supply comments Signed-off-by: Ruihang Xia * update sqlness result Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia (cherry picked from commit 0d90f7407c5aaf526e692947764c4d9c734b4f2a) Signed-off-by: evenyag * feat: pre-cast constants (#7926) * init impl Signed-off-by: Ruihang Xia * handle no cast Signed-off-by: Ruihang Xia * refactor using common-expr Signed-off-by: Ruihang Xia * extend matching pattern Signed-off-by: Ruihang Xia * more tests Signed-off-by: Ruihang Xia * simplification Signed-off-by: Ruihang Xia * fix zero timestamp Signed-off-by: Ruihang Xia * fix: normalize sqlness partition count output Signed-off-by: Ruihang Xia * fix: normalize remaining sqlness plan output Signed-off-by: Ruihang Xia * fix: normalize sqlness repartition details in tql explain Signed-off-by: Ruihang Xia * fix: tighten const normalization casts * test: normalize standalone tql explain repartition output * resolve cr comments Signed-off-by: Ruihang Xia * simplify Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia (cherry picked from commit 9133d0464f8a6bff0f72e2966279c5006a084d29) Signed-off-by: evenyag * fix(mito): ignore compaction override in enum option validation (#8094) * fix(mito): ignore compaction override in enum option validation Signed-off-by: QuakeWang * test: cover compaction override without compaction type Signed-off-by: QuakeWang * fix(mito): short-circuit enum option validation Signed-off-by: QuakeWang --------- Signed-off-by: QuakeWang (cherry picked from commit 73c267e64150c351ee5b4bb556e690337cc0db0e) Signed-off-by: evenyag * fix(mito2): drop unsound time-filter cache-key stripping (#8105) * fix(mito2): drop unsound time-filter cache-key stripping Signed-off-by: evenyag * chore: update comments and test Signed-off-by: evenyag --------- Signed-off-by: evenyag (cherry picked from commit 5e468190a55464d90f56fa5edd0dce4505593c7d) Signed-off-by: evenyag * fix: remap batch table route addresses (#8109) (cherry picked from commit a04fa524863dd8db97a780a85e21145fa9ec801c) Signed-off-by: evenyag * chore: bump version to v1.0.2 Signed-off-by: evenyag * fix: avoid stale route update during repartition allocation (#8115) Signed-off-by: WenyXu Signed-off-by: evenyag * test: update sqlness result Signed-off-by: evenyag --------- Signed-off-by: discord9 Signed-off-by: evenyag Signed-off-by: BootstrapperSBL Signed-off-by: Lei, HUANG Signed-off-by: BootstrapperSBL Signed-off-by: Ruihang Xia Signed-off-by: QuakeWang Signed-off-by: WenyXu Co-authored-by: discord9 <55937128+discord9@users.noreply.github.com> Co-authored-by: Yvan Wang <131545713+BootstrapperSBL@users.noreply.github.com> Co-authored-by: Lei, HUANG <6406592+v0y4g3r@users.noreply.github.com> Co-authored-by: BootstrapperSBL Co-authored-by: Ning Sun Co-authored-by: Ruihang Xia Co-authored-by: QuakeWang <45645138+QuakeWang@users.noreply.github.com> Co-authored-by: Weny Xu --- Cargo.lock | 153 +-- Cargo.toml | 4 +- src/common/catalog/src/lib.rs | 6 +- src/common/meta/src/key/table_route.rs | 59 +- src/frontend/src/instance.rs | 17 +- .../procedure/repartition/allocate_region.rs | 160 ++- src/metric-engine/src/batch_modifier.rs | 23 +- src/metric-engine/src/data_region.rs | 14 +- src/metric-engine/src/engine/alter.rs | 2 +- .../src/engine/alter/extract_new_columns.rs | 66 +- src/metric-engine/src/engine/bulk_insert.rs | 14 +- src/metric-engine/src/engine/create.rs | 4 +- .../src/engine/create/extract_new_columns.rs | 17 +- src/metric-engine/src/engine/open.rs | 2 +- src/metric-engine/src/engine/put.rs | 466 ++++++- src/metric-engine/src/engine/state.rs | 65 +- src/metric-engine/src/error.rs | 16 +- src/metric-engine/src/row_modifier.rs | 44 +- src/mito2/src/engine/scan_test.rs | 193 +++ src/mito2/src/read/range_cache.rs | 104 +- src/mito2/src/read/scan_region.rs | 15 +- src/mito2/src/region/options.rs | 69 +- src/query/Cargo.toml | 1 + src/query/src/optimizer.rs | 1 + .../src/optimizer/const_normalization.rs | 1133 +++++++++++++++++ src/query/src/optimizer/windowed_sort.rs | 505 ++++++-- src/query/src/part_sort.rs | 8 +- src/query/src/planner.rs | 36 +- src/query/src/query_engine/state.rs | 2 + src/query/src/range_select/plan_rewrite.rs | 228 +++- src/query/src/window_sort.rs | 123 +- src/servers/src/pending_rows_batcher.rs | 83 +- src/servers/src/postgres/types.rs | 4 +- tests-integration/tests/sql.rs | 60 + .../optimizer/windowed_sort_advance.result | 118 ++ .../optimizer/windowed_sort_advance.sql | 37 + .../optimizer/windowed_sort_nanos.result | 79 ++ .../optimizer/windowed_sort_nanos.sql | 27 + .../common/alter/alter_table.result | 22 +- .../standalone/common/alter/alter_table.sql | 12 +- .../common/create/create_with_options.result | 16 + .../common/create/create_with_options.sql | 12 + .../common/insert/logical_metric_table.result | 34 + .../common/insert/logical_metric_table.sql | 15 + .../standalone/common/range/error.result | 6 + tests/cases/standalone/common/range/error.sql | 4 + .../cases/standalone/common/range/nest.result | 76 ++ tests/cases/standalone/common/range/nest.sql | 48 + .../common/tql-explain-analyze/explain.result | 389 ++++++ .../common/tql-explain-analyze/explain.sql | 23 + .../time_index_filter_pushdown.result | 115 +- .../optimizer/time_index_filter_pushdown.sql | 72 +- .../optimizer/windowed_sort_advance.result | 117 ++ .../optimizer/windowed_sort_advance.sql | 37 + .../optimizer/windowed_sort_nanos.result | 78 ++ .../optimizer/windowed_sort_nanos.sql | 27 + 56 files changed, 4638 insertions(+), 423 deletions(-) create mode 100644 src/query/src/optimizer/const_normalization.rs create mode 100644 tests/cases/distributed/optimizer/windowed_sort_advance.result create mode 100644 tests/cases/distributed/optimizer/windowed_sort_advance.sql create mode 100644 tests/cases/distributed/optimizer/windowed_sort_nanos.result create mode 100644 tests/cases/distributed/optimizer/windowed_sort_nanos.sql create mode 100644 tests/cases/standalone/optimizer/windowed_sort_advance.result create mode 100644 tests/cases/standalone/optimizer/windowed_sort_advance.sql create mode 100644 tests/cases/standalone/optimizer/windowed_sort_nanos.result create mode 100644 tests/cases/standalone/optimizer/windowed_sort_nanos.sql diff --git a/Cargo.lock b/Cargo.lock index d66fa2d686..28a6a378ea 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -212,7 +212,7 @@ checksum = "d301b3b94cb4b2f23d7917810addbbaff90738e0ca2be692bd027e70d7e0330c" [[package]] name = "api" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arrow-schema 57.3.0", "common-base", @@ -933,7 +933,7 @@ checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" [[package]] name = "auth" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -1523,7 +1523,7 @@ dependencies = [ [[package]] name = "cache" -version = "1.0.1" +version = "1.0.2" dependencies = [ "catalog", "common-error", @@ -1559,7 +1559,7 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "catalog" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arrow 57.3.0", @@ -1894,7 +1894,7 @@ checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" [[package]] name = "cli" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-stream", "async-trait", @@ -1951,7 +1951,7 @@ dependencies = [ [[package]] name = "client" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arc-swap", @@ -1983,7 +1983,7 @@ dependencies = [ "serde_json", "snafu 0.8.6", "store-api", - "substrait 1.0.1", + "substrait 1.0.2", "tokio", "tokio-stream", "tonic 0.14.2", @@ -2023,7 +2023,7 @@ dependencies = [ [[package]] name = "cmd" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -2155,7 +2155,7 @@ dependencies = [ [[package]] name = "common-base" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "anymap2", @@ -2175,14 +2175,14 @@ dependencies = [ [[package]] name = "common-catalog" -version = "1.0.1" +version = "1.0.2" dependencies = [ "const_format", ] [[package]] name = "common-config" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-base", "common-error", @@ -2206,7 +2206,7 @@ dependencies = [ [[package]] name = "common-datasource" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arrow 57.3.0", "arrow-schema 57.3.0", @@ -2242,7 +2242,7 @@ dependencies = [ [[package]] name = "common-decimal" -version = "1.0.1" +version = "1.0.2" dependencies = [ "bigdecimal 0.4.8", "common-error", @@ -2255,7 +2255,7 @@ dependencies = [ [[package]] name = "common-error" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-macro", "http 1.3.1", @@ -2266,7 +2266,7 @@ dependencies = [ [[package]] name = "common-event-recorder" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -2289,7 +2289,7 @@ dependencies = [ [[package]] name = "common-frontend" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -2310,7 +2310,7 @@ dependencies = [ [[package]] name = "common-function" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "api", @@ -2374,7 +2374,7 @@ dependencies = [ [[package]] name = "common-greptimedb-telemetry" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "common-runtime", @@ -2391,7 +2391,7 @@ dependencies = [ [[package]] name = "common-grpc" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arrow-flight", @@ -2426,7 +2426,7 @@ dependencies = [ [[package]] name = "common-grpc-expr" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "common-base", @@ -2446,7 +2446,7 @@ dependencies = [ [[package]] name = "common-macro" -version = "1.0.1" +version = "1.0.2" dependencies = [ "greptime-proto", "once_cell", @@ -2457,7 +2457,7 @@ dependencies = [ [[package]] name = "common-mem-prof" -version = "1.0.1" +version = "1.0.2" dependencies = [ "anyhow", "common-error", @@ -2473,7 +2473,7 @@ dependencies = [ [[package]] name = "common-memory-manager" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-error", "common-macro", @@ -2485,7 +2485,7 @@ dependencies = [ [[package]] name = "common-meta" -version = "1.0.1" +version = "1.0.2" dependencies = [ "anymap2", "api", @@ -2556,7 +2556,7 @@ dependencies = [ [[package]] name = "common-options" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-grpc", "humantime-serde", @@ -2566,11 +2566,11 @@ dependencies = [ [[package]] name = "common-plugins" -version = "1.0.1" +version = "1.0.2" [[package]] name = "common-pprof" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-error", "common-macro", @@ -2581,7 +2581,7 @@ dependencies = [ [[package]] name = "common-procedure" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-stream", @@ -2610,7 +2610,7 @@ dependencies = [ [[package]] name = "common-procedure-test" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "common-procedure", @@ -2620,7 +2620,7 @@ dependencies = [ [[package]] name = "common-query" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -2646,7 +2646,7 @@ dependencies = [ [[package]] name = "common-recordbatch" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arc-swap", "common-base", @@ -2671,7 +2671,7 @@ dependencies = [ [[package]] name = "common-runtime" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "clap", @@ -2700,7 +2700,7 @@ dependencies = [ [[package]] name = "common-session" -version = "1.0.1" +version = "1.0.2" dependencies = [ "serde", "strum 0.27.1", @@ -2708,7 +2708,7 @@ dependencies = [ [[package]] name = "common-sql" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arrow-schema 57.3.0", "common-base", @@ -2728,7 +2728,7 @@ dependencies = [ [[package]] name = "common-stat" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-base", "common-runtime", @@ -2743,7 +2743,7 @@ dependencies = [ [[package]] name = "common-telemetry" -version = "1.0.1" +version = "1.0.2" dependencies = [ "backtrace", "common-base", @@ -2772,7 +2772,7 @@ dependencies = [ [[package]] name = "common-test-util" -version = "1.0.1" +version = "1.0.2" dependencies = [ "client", "common-grpc", @@ -2785,7 +2785,7 @@ dependencies = [ [[package]] name = "common-time" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arrow 57.3.0", "chrono", @@ -2803,7 +2803,7 @@ dependencies = [ [[package]] name = "common-version" -version = "1.0.1" +version = "1.0.2" dependencies = [ "cargo-manifest", "const_format", @@ -2813,7 +2813,7 @@ dependencies = [ [[package]] name = "common-wal" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-base", "common-error", @@ -2836,7 +2836,7 @@ dependencies = [ [[package]] name = "common-workload" -version = "1.0.1" +version = "1.0.2" dependencies = [ "common-telemetry", "serde", @@ -4198,7 +4198,7 @@ dependencies = [ [[package]] name = "datanode" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arrow-flight", @@ -4266,7 +4266,7 @@ dependencies = [ [[package]] name = "datatypes" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arrow 57.3.0", "arrow-array 57.3.0", @@ -4944,7 +4944,7 @@ checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "file-engine" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -5076,7 +5076,7 @@ checksum = "8bf7cc16383c4b8d58b9905a8509f02926ce3058053c056376248d958c9df1e8" [[package]] name = "flow" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arrow 57.3.0", @@ -5145,7 +5145,7 @@ dependencies = [ "sql", "store-api", "strum 0.27.1", - "substrait 1.0.1", + "substrait 1.0.2", "table", "tokio", "tonic 0.14.2", @@ -5206,7 +5206,7 @@ checksum = "28dd6caf6059519a65843af8fe2a3ae298b14b80179855aeb4adc2c1934ee619" [[package]] name = "frontend" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arc-swap", @@ -6453,7 +6453,7 @@ dependencies = [ [[package]] name = "index" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "asynchronous-codec", @@ -7421,7 +7421,7 @@ checksum = "5e5032e24019045c762d3c0f28f5b6b8bbf38563a65908389bf7978758920897" [[package]] name = "log-query" -version = "1.0.1" +version = "1.0.2" dependencies = [ "chrono", "common-error", @@ -7433,7 +7433,7 @@ dependencies = [ [[package]] name = "log-store" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-stream", "async-trait", @@ -7724,7 +7724,7 @@ dependencies = [ [[package]] name = "meta-client" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -7755,7 +7755,7 @@ dependencies = [ [[package]] name = "meta-srv" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -7855,7 +7855,7 @@ dependencies = [ [[package]] name = "metric-engine" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "aquamarine", @@ -7956,7 +7956,7 @@ dependencies = [ [[package]] name = "mito-codec" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "bytes", @@ -7981,7 +7981,7 @@ dependencies = [ [[package]] name = "mito2" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "aquamarine", @@ -8705,7 +8705,7 @@ dependencies = [ [[package]] name = "object-store" -version = "1.0.1" +version = "1.0.2" dependencies = [ "anyhow", "bytes", @@ -9032,7 +9032,7 @@ dependencies = [ [[package]] name = "operator" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "api", @@ -9092,7 +9092,7 @@ dependencies = [ "sql", "sqlparser", "store-api", - "substrait 1.0.1", + "substrait 1.0.2", "table", "tokio", "tokio-util", @@ -9368,7 +9368,7 @@ checksum = "e3c406c9e2aa74554e662d2c2ee11cd3e73756988800be7e6f5eddb16fed4699" [[package]] name = "partition" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "async-trait", @@ -9724,7 +9724,7 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pipeline" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "api", @@ -9881,7 +9881,7 @@ dependencies = [ [[package]] name = "plugins" -version = "1.0.1" +version = "1.0.2" dependencies = [ "auth", "catalog", @@ -10199,7 +10199,7 @@ dependencies = [ [[package]] name = "promql" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "async-trait", @@ -10551,7 +10551,7 @@ dependencies = [ [[package]] name = "puffin" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-compression", "async-trait", @@ -10613,7 +10613,7 @@ dependencies = [ [[package]] name = "query" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "api", @@ -10643,6 +10643,7 @@ dependencies = [ "datafusion", "datafusion-common", "datafusion-expr", + "datafusion-expr-common", "datafusion-functions", "datafusion-optimizer", "datafusion-physical-expr", @@ -10680,7 +10681,7 @@ dependencies = [ "sql", "sqlparser", "store-api", - "substrait 1.0.1", + "substrait 1.0.2", "table", "tokio", "tokio-stream", @@ -11984,7 +11985,7 @@ dependencies = [ [[package]] name = "servers" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "api", @@ -12120,7 +12121,7 @@ dependencies = [ [[package]] name = "session" -version = "1.0.1" +version = "1.0.2" dependencies = [ "ahash 0.8.12", "api", @@ -12452,7 +12453,7 @@ dependencies = [ [[package]] name = "sql" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arrow-buffer 57.3.0", @@ -12513,7 +12514,7 @@ dependencies = [ [[package]] name = "sqlness-runner" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "clap", @@ -12793,7 +12794,7 @@ dependencies = [ [[package]] name = "standalone" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "catalog", @@ -12837,7 +12838,7 @@ checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" [[package]] name = "store-api" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "aquamarine", @@ -13029,7 +13030,7 @@ dependencies = [ [[package]] name = "substrait" -version = "1.0.1" +version = "1.0.2" dependencies = [ "async-trait", "bytes", @@ -13151,7 +13152,7 @@ dependencies = [ [[package]] name = "table" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arc-swap", @@ -13421,7 +13422,7 @@ checksum = "8f50febec83f5ee1df3015341d8bd429f2d1cc62bcba7ea2076759d315084683" [[package]] name = "tests-fuzz" -version = "1.0.1" +version = "1.0.2" dependencies = [ "arbitrary", "async-trait", @@ -13465,7 +13466,7 @@ dependencies = [ [[package]] name = "tests-integration" -version = "1.0.1" +version = "1.0.2" dependencies = [ "api", "arrow-flight", @@ -13542,7 +13543,7 @@ dependencies = [ "sqlx", "standalone", "store-api", - "substrait 1.0.1", + "substrait 1.0.2", "table", "tempfile", "time", diff --git a/Cargo.toml b/Cargo.toml index aa3d3d0c6c..408819a277 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -75,7 +75,7 @@ members = [ resolver = "2" [workspace.package] -version = "1.0.1" +version = "1.0.2" edition = "2024" license = "Apache-2.0" @@ -131,6 +131,7 @@ datafusion = "=52.1" datafusion-common = "=52.1" datafusion-datasource = "=52.1" datafusion-expr = "=52.1" +datafusion-expr-common = "=52.1" datafusion-functions = "=52.1" datafusion-functions-aggregate-common = "=52.1" datafusion-functions-window-common = "=52.1" @@ -335,6 +336,7 @@ rev = "5618e779cf2bb4755b499c630fba4c35e91898cb" datafusion = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } datafusion-common = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } datafusion-expr = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } +datafusion-expr-common = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } datafusion-functions = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } datafusion-functions-aggregate-common = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } datafusion-functions-window-common = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "02b82535e0160c4545667f36a03e1ff9d1d2e51f" } diff --git a/src/common/catalog/src/lib.rs b/src/common/catalog/src/lib.rs index 1e9534532e..592e0df8fe 100644 --- a/src/common/catalog/src/lib.rs +++ b/src/common/catalog/src/lib.rs @@ -74,9 +74,9 @@ pub fn parse_catalog_and_schema_from_db_string(db: &str) -> (String, String) { pub fn parse_optional_catalog_and_schema_from_db_string(db: &str) -> (Option, String) { let parts = db.splitn(2, '-').collect::>(); if parts.len() == 2 { - (Some(parts[0].to_lowercase()), parts[1].to_lowercase()) + (Some(parts[0].to_string()), parts[1].to_string()) } else { - (None, db.to_lowercase()) + (None, db.to_string()) } } @@ -118,7 +118,7 @@ mod tests { ); assert_eq!( - (Some("catalog".to_string()), "schema".to_string()), + (Some("CATALOG".to_string()), "SCHEMA".to_string()), parse_optional_catalog_and_schema_from_db_string("CATALOG-SCHEMA") ); diff --git a/src/common/meta/src/key/table_route.rs b/src/common/meta/src/key/table_route.rs index 4c618addda..c2906eedf4 100644 --- a/src/common/meta/src/key/table_route.rs +++ b/src/common/meta/src/key/table_route.rs @@ -717,7 +717,7 @@ impl TableRouteStorage { /// Returns batch of [`TableRouteValue`] that respects the order of `table_ids`. pub async fn batch_get(&self, table_ids: &[TableId]) -> Result>> { - let raw_table_routes = self.batch_get_inner(table_ids).await?; + let raw_table_routes = self.batch_get_with_raw_bytes(table_ids).await?; Ok(raw_table_routes .into_iter() @@ -1085,6 +1085,63 @@ mod tests { assert_eq!(results[3].as_ref().unwrap(), &routes[0].1); } + #[tokio::test] + async fn test_table_route_batch_get_remaps_addresses() { + let kv = Arc::new(MemoryKvBackend::default()); + let table_route_storage = TableRouteStorage::new(kv.clone()); + let table_route_manager = TableRouteManager::new(kv.clone()); + let table_route_value = TableRouteValue::Physical(PhysicalTableRouteValue { + region_routes: vec![RegionRoute { + leader_peer: Some(Peer { + id: 1, + addr: "old-leader".to_string(), + }), + follower_peers: vec![Peer { + id: 2, + addr: "old-follower".to_string(), + }], + ..Default::default() + }], + max_region_number: 0, + version: 0, + }); + let (txn, _) = table_route_manager + .table_route_storage() + .build_create_txn(1024, &table_route_value) + .unwrap(); + let r = kv.txn(txn).await.unwrap(); + assert!(r.succeeded); + + for (node_id, addr) in [(1, "new-leader"), (2, "new-follower")] { + kv.put(PutRequest { + key: NodeAddressKey::with_datanode(node_id).to_bytes(), + value: NodeAddressValue { + peer: Peer { + id: node_id, + addr: addr.to_string(), + }, + } + .try_as_raw_value() + .unwrap(), + ..Default::default() + }) + .await + .unwrap(); + } + + let results = table_route_storage.batch_get(&[1024]).await.unwrap(); + let TableRouteValue::Physical(physical_table_route) = results[0].as_ref().unwrap() else { + panic!("Expected PhysicalTableRouteValue"); + }; + + let region_route = &physical_table_route.region_routes[0]; + assert_eq!( + region_route.leader_peer.as_ref().unwrap().addr, + "new-leader" + ); + assert_eq!(region_route.follower_peers[0].addr, "new-follower"); + } + #[tokio::test] async fn remap_route_address_updates_addresses() { let kv = Arc::new(MemoryKvBackend::default()); diff --git a/src/frontend/src/instance.rs b/src/frontend/src/instance.rs index ce589bb677..b477eeb8b9 100644 --- a/src/frontend/src/instance.rs +++ b/src/frontend/src/instance.rs @@ -677,10 +677,19 @@ impl Instance { ) -> Result> { ensure!(!self.is_suspended(), error::SuspendedSnafu); - if matches!( - stmt, - Statement::Insert(_) | Statement::Query(_) | Statement::Delete(_) - ) { + // EXPLAIN / EXPLAIN ANALYZE wrap an inner statement; describe them when the + // wrapped statement is something we already plan (so that bind parameters + // in the inner query get their types inferred). See #8029. + let is_inner_plannable = |s: &Statement| { + matches!( + s, + Statement::Insert(_) | Statement::Query(_) | Statement::Delete(_) + ) + }; + let plannable = is_inner_plannable(&stmt) + || matches!(&stmt, Statement::Explain(explain) if is_inner_plannable(explain.statement.as_ref())); + + if plannable { self.plugins .get::() .as_ref() diff --git a/src/meta-srv/src/procedure/repartition/allocate_region.rs b/src/meta-srv/src/procedure/repartition/allocate_region.rs index c1f3ca2503..e38d6d3a95 100644 --- a/src/meta-srv/src/procedure/repartition/allocate_region.rs +++ b/src/meta-srv/src/procedure/repartition/allocate_region.rs @@ -148,9 +148,6 @@ impl ExecutePlan { let region_number_and_partition_exprs = AllocateRegion::prepare_region_allocation_data(&allocate_regions)?; let table_info_value = ctx.get_table_info_value().await?; - let table_route_value = ctx.get_table_route_value().await?; - // Safety: it is physical table route value. - let region_routes = table_route_value.region_routes().unwrap(); let new_allocated_region_routes = ctx .region_routes_allocator .allocate( @@ -205,6 +202,11 @@ impl ExecutePlan { // Updates the table routes. let table_lock = TableLock::Write(table_id).into(); let _guard = procedure_ctx.provider.acquire_lock(&table_lock).await; + // MUST refresh the table route value after acquiring the lock to avoid lost update. + // Otherwise, the new allocated regions might be overridden by concurrent repartition procedures. + let table_route_value = ctx.get_table_route_value().await?; + // Safety: it is physical table route value. + let region_routes = table_route_value.region_routes().unwrap(); let new_region_routes = AllocateRegion::generate_region_routes(region_routes, &new_allocated_region_routes); ctx.update_table_route(&table_route_value, new_region_routes, wal_options) @@ -397,14 +399,26 @@ impl AllocateRegion { #[cfg(test)] mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use common_meta::key::TableMetadataManagerRef; + use common_meta::key::datanode_table::DatanodeTableKey; use common_meta::peer::Peer; use common_meta::rpc::router::{Region, RegionRoute}; + use common_meta::test_util::MockDatanodeManager; + use common_procedure::{ContextProvider, ProcedureId, ProcedureState}; + use common_procedure_test::MockContextProvider; use store_api::storage::RegionId; + use tokio::sync::watch; use uuid::Uuid; use super::*; use crate::procedure::repartition::State; - use crate::procedure::repartition::test_util::range_expr; + use crate::procedure::repartition::test_util::{ + TestingEnv, current_parent_region_routes, new_parent_context, range_expr, + test_region_wal_options, + }; fn create_region_descriptor( table_id: TableId, @@ -461,6 +475,88 @@ mod tests { .collect() } + struct ConcurrentTableRouteUpdateProvider { + inner: MockContextProvider, + table_metadata_manager: TableMetadataManagerRef, + table_id: TableId, + concurrent_region_route: RegionRoute, + region_wal_options: HashMap, + } + + #[async_trait::async_trait] + impl ContextProvider for ConcurrentTableRouteUpdateProvider { + async fn procedure_state( + &self, + procedure_id: ProcedureId, + ) -> common_procedure::Result> { + self.inner.procedure_state(procedure_id).await + } + + async fn procedure_state_receiver( + &self, + procedure_id: ProcedureId, + ) -> common_procedure::Result>> { + self.inner.procedure_state_receiver(procedure_id).await + } + + async fn try_put_poison( + &self, + key: &common_procedure::PoisonKey, + procedure_id: ProcedureId, + ) -> common_procedure::Result<()> { + self.inner.try_put_poison(key, procedure_id).await + } + + async fn acquire_lock( + &self, + key: &common_procedure::StringKey, + ) -> common_procedure::local::DynamicKeyLockGuard { + let current_table_route_value = self + .table_metadata_manager + .table_route_manager() + .table_route_storage() + .get_with_raw_bytes(self.table_id) + .await + .unwrap() + .unwrap(); + let mut region_routes = current_table_route_value.region_routes().unwrap().clone(); + + if !region_routes + .iter() + .any(|route| route.region.id == self.concurrent_region_route.region.id) + { + region_routes.push(self.concurrent_region_route.clone()); + let datanode_id = current_table_route_value.region_routes().unwrap()[0] + .leader_peer + .as_ref() + .unwrap() + .id; + let datanode_table_value = self + .table_metadata_manager + .datanode_table_manager() + .get(&DatanodeTableKey::new(datanode_id, self.table_id)) + .await + .unwrap() + .unwrap(); + let region_options = &datanode_table_value.region_info.region_options; + + self.table_metadata_manager + .update_table_route( + self.table_id, + datanode_table_value.region_info.clone(), + ¤t_table_route_value, + region_routes, + region_options, + &self.region_wal_options, + ) + .await + .unwrap(); + } + + self.inner.acquire_lock(key).await + } + } + #[test] fn test_convert_to_repartition_plans_no_allocation() { let table_id = 1024; @@ -617,6 +713,62 @@ mod tests { assert!(!result[1].1.is_empty()); } + #[tokio::test] + async fn test_execute_plan_uses_latest_table_route_after_lock() { + let env = TestingEnv::new(); + let table_id = 1024; + let original_region_routes = create_current_region_routes(table_id, &[1]); + env.create_physical_table_metadata_for_repartition( + table_id, + original_region_routes, + test_region_wal_options(&[1]), + ) + .await; + + let node_manager = Arc::new(MockDatanodeManager::new(())); + let mut ctx = new_parent_context(&env, node_manager, table_id); + ctx.persistent_ctx.plans = vec![RepartitionPlanEntry { + group_id: Uuid::new_v4(), + source_regions: vec![], + target_regions: vec![create_region_descriptor(table_id, 3, "x", 0, 100)], + allocated_region_ids: vec![RegionId::new(table_id, 3)], + pending_deallocate_region_ids: vec![], + transition_map: vec![], + original_target_routes: vec![], + }]; + let concurrent_region_route = create_current_region_routes(table_id, &[2]) + .into_iter() + .next() + .unwrap(); + let procedure_ctx = ProcedureContext { + procedure_id: ProcedureId::random(), + provider: Arc::new(ConcurrentTableRouteUpdateProvider { + inner: MockContextProvider::default(), + table_metadata_manager: env.table_metadata_manager.clone(), + table_id, + concurrent_region_route, + region_wal_options: test_region_wal_options(&[1, 2]), + }), + }; + let mut state = ExecutePlan; + + state.next(&mut ctx, &procedure_ctx).await.unwrap(); + + let region_ids = current_parent_region_routes(&ctx) + .await + .into_iter() + .map(|route| route.region.id) + .collect::>(); + assert_eq!( + region_ids, + vec![ + RegionId::new(table_id, 1), + RegionId::new(table_id, 2), + RegionId::new(table_id, 3), + ] + ); + } + #[test] fn test_allocate_region_state_backward_compatibility() { // Arrange diff --git a/src/metric-engine/src/batch_modifier.rs b/src/metric-engine/src/batch_modifier.rs index 76d9bb418a..2162215d1b 100644 --- a/src/metric-engine/src/batch_modifier.rs +++ b/src/metric-engine/src/batch_modifier.rs @@ -201,6 +201,7 @@ mod tests { use datatypes::arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; use datatypes::arrow::record_batch::RecordBatch; use store_api::codec::PrimaryKeyEncoding; + use store_api::metadata::ColumnMetadata; use store_api::storage::consts::PRIMARY_KEY_COLUMN_NAME; use super::*; @@ -364,11 +365,23 @@ mod tests { let modified = modify_batch_sparse(batch, table_id, &tag_columns, &non_tag_indices).unwrap(); - let name_to_column_id: HashMap = [ - ("greptime_timestamp".to_string(), 0), - ("greptime_value".to_string(), 1), - ("namespace".to_string(), 2), - ("host".to_string(), 3), + let make_info = |name: &str, column_id: ColumnId| ColumnMetadata { + column_schema: datatypes::schema::ColumnSchema::new( + name.to_string(), + datatypes::prelude::ConcreteDataType::string_datatype(), + false, + ), + semantic_type: SemanticType::Tag, + column_id, + }; + let name_to_column_id: HashMap = [ + ( + "greptime_timestamp".to_string(), + make_info("greptime_timestamp", 0), + ), + ("greptime_value".to_string(), make_info("greptime_value", 1)), + ("namespace".to_string(), make_info("namespace", 2)), + ("host".to_string(), make_info("host", 3)), ] .into_iter() .collect(); diff --git a/src/metric-engine/src/data_region.rs b/src/metric-engine/src/data_region.rs index 9bc22e1102..b64d61a287 100644 --- a/src/metric-engine/src/data_region.rs +++ b/src/metric-engine/src/data_region.rs @@ -13,7 +13,7 @@ // limitations under the License. use api::v1::SemanticType; -use common_telemetry::{debug, info, warn}; +use common_telemetry::{debug, info}; use datatypes::schema::{SkippingIndexOptions, SkippingIndexType}; use mito2::engine::MitoEngine; use snafu::ResultExt; @@ -27,8 +27,8 @@ use store_api::storage::{ConcreteDataType, RegionId}; use crate::engine::IndexOptions; use crate::error::{ - ColumnTypeMismatchSnafu, ForbiddenPhysicalAlterSnafu, MitoReadOperationSnafu, - MitoWriteOperationSnafu, Result, SetSkippingIndexOptionSnafu, + AddingFieldColumnSnafu, ColumnTypeMismatchSnafu, ForbiddenPhysicalAlterSnafu, + MitoReadOperationSnafu, MitoWriteOperationSnafu, Result, SetSkippingIndexOptionSnafu, }; use crate::metrics::{FORBIDDEN_OPERATION_COUNT, MITO_DDL_DURATION, PHYSICAL_COLUMN_COUNT}; use crate::utils; @@ -132,10 +132,10 @@ impl DataRegion { .fail(); } } else { - warn!( - "Column {} in region {region_id} is not a tag", - c.column_schema.name - ); + return AddingFieldColumnSnafu { + name: &c.column_schema.name, + } + .fail(); }; c.column_id = new_column_id_start + delta as u32; diff --git a/src/metric-engine/src/engine/alter.rs b/src/metric-engine/src/engine/alter.rs index 4b6b67f31b..892e0c91c2 100644 --- a/src/metric-engine/src/engine/alter.rs +++ b/src/metric-engine/src/engine/alter.rs @@ -189,7 +189,7 @@ impl MetricEngineInner { let new_add_columns = new_column_names.iter().map(|name| { // Safety: previous steps ensure the physical region exist let column_metadata = *physical_schema_map.get(name).unwrap(); - (name.to_string(), column_metadata.column_id) + (name.to_string(), column_metadata.clone()) }); // Writes logical regions metadata to metadata region diff --git a/src/metric-engine/src/engine/alter/extract_new_columns.rs b/src/metric-engine/src/engine/alter/extract_new_columns.rs index fdb1ef6126..c3bef1ba4f 100644 --- a/src/metric-engine/src/engine/alter/extract_new_columns.rs +++ b/src/metric-engine/src/engine/alter/extract_new_columns.rs @@ -14,11 +14,13 @@ use std::collections::{HashMap, HashSet}; +use api::v1::SemanticType; +use snafu::ensure; use store_api::metadata::ColumnMetadata; use store_api::region_request::{AlterKind, RegionAlterRequest}; -use store_api::storage::{ColumnId, RegionId}; +use store_api::storage::RegionId; -use crate::error::Result; +use crate::error::{AddingFieldColumnSnafu, Result}; /// Extract new columns from the create requests. /// @@ -27,7 +29,7 @@ use crate::error::Result; /// This function will panic if the alter kind is not `AddColumns`. pub fn extract_new_columns<'a>( requests: &'a [(RegionId, RegionAlterRequest)], - physical_columns: &HashMap, + physical_columns: &HashMap, new_column_names: &mut HashSet<&'a str>, new_columns: &mut Vec, ) -> Result<()> { @@ -40,6 +42,12 @@ pub fn extract_new_columns<'a>( if !physical_columns.contains_key(column_name) && !new_column_names.contains(column_name) { + ensure!( + col.column_metadata.semantic_type != SemanticType::Field, + AddingFieldColumnSnafu { + name: column_name.to_string(), + } + ); new_column_names.insert(column_name); // TODO(weny): avoid clone new_columns.push(col.column_metadata.clone()); @@ -49,3 +57,55 @@ pub fn extract_new_columns<'a>( Ok(()) } + +#[cfg(test)] +mod tests { + use std::collections::{HashMap, HashSet}; + + use api::v1::SemanticType; + use datatypes::prelude::ConcreteDataType; + use datatypes::schema::ColumnSchema; + use store_api::metadata::ColumnMetadata; + use store_api::region_request::{AddColumn, AlterKind, RegionAlterRequest}; + use store_api::storage::RegionId; + + use super::*; + use crate::error::Error; + + #[test] + fn test_extract_new_columns_with_field_type() { + let requests = vec![( + RegionId::new(1, 1), + RegionAlterRequest { + kind: AlterKind::AddColumns { + columns: vec![AddColumn { + column_metadata: ColumnMetadata { + column_schema: ColumnSchema::new( + "new_column".to_string(), + ConcreteDataType::string_datatype(), + false, + ), + semantic_type: SemanticType::Field, + column_id: 0, + }, + location: None, + }], + }, + }, + )]; + + let physical_columns = HashMap::new(); + let mut new_column_names = HashSet::new(); + let mut new_columns = Vec::new(); + + let err = extract_new_columns( + &requests, + &physical_columns, + &mut new_column_names, + &mut new_columns, + ) + .unwrap_err(); + + assert!(matches!(err, Error::AddingFieldColumn { .. })); + } +} diff --git a/src/metric-engine/src/engine/bulk_insert.rs b/src/metric-engine/src/engine/bulk_insert.rs index 942dae1136..24c9e7934c 100644 --- a/src/metric-engine/src/engine/bulk_insert.rs +++ b/src/metric-engine/src/engine/bulk_insert.rs @@ -190,13 +190,13 @@ impl MetricEngineInner { for (index, field) in batch.schema().fields().iter().enumerate() { let name = field.name(); - let column_id = - *physical_columns - .get(name) - .with_context(|| error::ColumnNotFoundSnafu { - name: name.clone(), - region_id: logical_region_id, - })?; + let column_id = physical_columns + .get(name) + .map(|info| info.column_id) + .with_context(|| error::ColumnNotFoundSnafu { + name: name.clone(), + region_id: logical_region_id, + })?; if tag_names.contains(name.as_str()) { tag_columns.push(TagColumnInfo { name: name.clone(), diff --git a/src/metric-engine/src/engine/create.rs b/src/metric-engine/src/engine/create.rs index 131ec1dbd4..34aeb5577c 100644 --- a/src/metric-engine/src/engine/create.rs +++ b/src/metric-engine/src/engine/create.rs @@ -145,7 +145,7 @@ impl MetricEngineInner { let physical_columns = create_data_region_request .column_metadatas .iter() - .map(|metadata| (metadata.column_schema.name.clone(), metadata.column_id)) + .map(|metadata| (metadata.column_schema.name.clone(), metadata.clone())) .collect::>(); let time_index_unit = create_data_region_request .column_metadatas @@ -321,7 +321,7 @@ impl MetricEngineInner { let new_add_columns = new_column_names.iter().map(|name| { // Safety: previous steps ensure the physical region exist let column_metadata = *physical_schema_map.get(name).unwrap(); - (name.to_string(), column_metadata.column_id) + (name.to_string(), column_metadata.clone()) }); extension_return_value.insert( 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 9d1de9ebb2..82cc53ce3d 100644 --- a/src/metric-engine/src/engine/create/extract_new_columns.rs +++ b/src/metric-engine/src/engine/create/extract_new_columns.rs @@ -18,14 +18,14 @@ use api::v1::SemanticType; use snafu::ensure; use store_api::metadata::ColumnMetadata; use store_api::region_request::RegionCreateRequest; -use store_api::storage::{ColumnId, RegionId}; +use store_api::storage::RegionId; use crate::error::{AddingFieldColumnSnafu, Result}; /// Extract new columns from the create requests. pub fn extract_new_columns<'a>( requests: &'a [(RegionId, RegionCreateRequest)], - physical_columns: &HashMap, + physical_columns: &HashMap, new_column_names: &mut HashSet<&'a str>, new_columns: &mut Vec, ) -> Result<()> { @@ -123,7 +123,18 @@ mod tests { ]; let mut physical_columns = HashMap::new(); - physical_columns.insert("existing_column".to_string(), 0); + physical_columns.insert( + "existing_column".to_string(), + ColumnMetadata { + column_schema: ColumnSchema::new( + "existing_column".to_string(), + ConcreteDataType::string_datatype(), + false, + ), + semantic_type: SemanticType::Tag, + column_id: 0, + }, + ); let mut new_column_names = HashSet::new(); let mut new_columns = Vec::new(); diff --git a/src/metric-engine/src/engine/open.rs b/src/metric-engine/src/engine/open.rs index 70e3280183..59b1cfd928 100644 --- a/src/metric-engine/src/engine/open.rs +++ b/src/metric-engine/src/engine/open.rs @@ -326,7 +326,7 @@ impl MetricEngineInner { .unwrap(); let physical_columns = physical_columns .into_iter() - .map(|col| (col.column_schema.name, col.column_id)) + .map(|col| (col.column_schema.name.clone(), col)) .collect(); state.add_physical_region( physical_region_id, diff --git a/src/metric-engine/src/engine/put.rs b/src/metric-engine/src/engine/put.rs index edae0d2bb4..07adfae120 100644 --- a/src/metric-engine/src/engine/put.rs +++ b/src/metric-engine/src/engine/put.rs @@ -14,13 +14,16 @@ use std::collections::HashMap; +use api::helper::ColumnDataTypeWrapper; use api::v1::{ - ColumnSchema, PrimaryKeyEncoding as PrimaryKeyEncodingProto, Row, Rows, Value, WriteHint, + ColumnSchema, PrimaryKeyEncoding as PrimaryKeyEncodingProto, Row, Rows, SemanticType, Value, + WriteHint, }; use common_telemetry::{error, info}; use fxhash::FxHashMap; -use snafu::{OptionExt, ensure}; +use snafu::{OptionExt, ResultExt, ensure}; use store_api::codec::PrimaryKeyEncoding; +use store_api::metadata::ColumnMetadata; use store_api::region_request::{ AffectedRows, RegionDeleteRequest, RegionPutRequest, RegionRequest, }; @@ -28,8 +31,9 @@ use store_api::storage::{RegionId, TableId}; use crate::engine::MetricEngineInner; use crate::error::{ - ColumnNotFoundSnafu, ForbiddenPhysicalAlterSnafu, InvalidRequestSnafu, - LogicalRegionNotFoundSnafu, PhysicalRegionNotFoundSnafu, Result, UnsupportedRegionRequestSnafu, + ColumnNotFoundSnafu, CreateDefaultSnafu, ForbiddenPhysicalAlterSnafu, InvalidRequestSnafu, + LogicalRegionNotFoundSnafu, PhysicalRegionNotFoundSnafu, Result, UnexpectedRequestSnafu, + UnsupportedRegionRequestSnafu, }; use crate::metrics::{FORBIDDEN_OPERATION_COUNT, MITO_OPERATION_ELAPSED}; use crate::row_modifier::{RowsIter, TableIdInput}; @@ -116,7 +120,7 @@ impl MetricEngineInner { async fn put_regions_batch_single_physical( &self, physical_region_id: RegionId, - requests: Vec<(RegionId, RegionPutRequest)>, + mut requests: Vec<(RegionId, RegionPutRequest)>, ) -> Result { if requests.is_empty() { return Ok(0); @@ -126,7 +130,7 @@ impl MetricEngineInner { let primary_key_encoding = self.get_primary_key_encoding(data_region_id)?; // Validate all requests - self.validate_batch_requests(physical_region_id, &requests) + self.validate_batch_requests(physical_region_id, &mut requests) .await?; // Merge requests according to encoding strategy @@ -157,11 +161,16 @@ impl MetricEngineInner { async fn validate_batch_requests( &self, physical_region_id: RegionId, - requests: &[(RegionId, RegionPutRequest)], + requests: &mut [(RegionId, RegionPutRequest)], ) -> Result<()> { for (logical_region_id, request) in requests { - self.verify_rows(*logical_region_id, physical_region_id, &request.rows) - .await?; + self.verify_rows( + *logical_region_id, + physical_region_id, + &mut request.rows, + true, + ) + .await?; } Ok(()) } @@ -248,7 +257,7 @@ impl MetricEngineInner { // Batch-modify all rows (add __table_id and __tsid columns) let final_rows = { let state = self.state.read().unwrap(); - let name_to_id = state + let physical_columns = state .physical_region_states() .get(&data_region_id) .with_context(|| PhysicalRegionNotFoundSnafu { @@ -261,7 +270,7 @@ impl MetricEngineInner { schema: merged_schema, rows: merged_rows, }, - name_to_id, + physical_columns, ); self.row_modifier.modify_rows( @@ -406,8 +415,13 @@ impl MetricEngineInner { let (physical_region_id, data_region_id, primary_key_encoding) = self.find_data_region_meta(logical_region_id)?; - self.verify_rows(logical_region_id, physical_region_id, &request.rows) - .await?; + self.verify_rows( + logical_region_id, + physical_region_id, + &mut request.rows, + true, + ) + .await?; // write to data region // TODO: retrieve table name @@ -439,8 +453,13 @@ impl MetricEngineInner { let (physical_region_id, data_region_id, primary_key_encoding) = self.find_data_region_meta(logical_region_id)?; - self.verify_rows(logical_region_id, physical_region_id, &request.rows) - .await?; + self.verify_rows( + logical_region_id, + physical_region_id, + &mut request.rows, + false, + ) + .await?; // write to data region // TODO: retrieve table name @@ -484,12 +503,18 @@ impl MetricEngineInner { /// /// Includes: /// - Check if the logical region exists - /// - Check if the columns exist + /// - Check if every column in the request exists in the physical region + /// - Check each column's datatype and semantic type match the physical region's schema + /// - Check the time index column is present + /// - When `check_fields` is true, check every physical field column is present. + /// Set this to `false` for delete requests, which legitimately carry only + /// the primary key + timestamp. async fn verify_rows( &self, logical_region_id: RegionId, physical_region_id: RegionId, - rows: &Rows, + rows: &mut Rows, + check_fields: bool, ) -> Result<()> { // Check if the region exists let data_region_id = to_data_region_id(physical_region_id); @@ -502,22 +527,137 @@ impl MetricEngineInner { .fail(); } - // Check if a physical column exists - let physical_columns = state + // Type + semantic check on every column in the request schema. + let physical_state = state .physical_region_states() .get(&data_region_id) .context(PhysicalRegionNotFoundSnafu { region_id: data_region_id, - })? - .physical_columns(); + })?; + let physical_columns = physical_state.physical_columns(); for col in &rows.schema { - ensure!( - physical_columns.contains_key(&col.column_name), - ColumnNotFoundSnafu { - name: col.column_name.clone(), + let info = physical_columns + .get(&col.column_name) + .context(ColumnNotFoundSnafu { + name: &col.column_name, region_id: logical_region_id, + })?; + + ensure!( + api::helper::is_column_type_value_eq( + col.datatype, + col.datatype_extension.clone(), + &info.column_schema.data_type + ), + InvalidRequestSnafu { + region_id: logical_region_id, + reason: format!( + "column {} expect type {:?}, given: {}({})", + col.column_name, + info.column_schema.data_type, + api::v1::ColumnDataType::try_from(col.datatype) + .map(|v| v.as_str_name()) + .unwrap_or("Unknown"), + col.datatype, + ), } ); + + ensure!( + api::helper::is_semantic_type_eq(col.semantic_type, info.semantic_type), + InvalidRequestSnafu { + region_id: logical_region_id, + reason: format!( + "column {} expect semantic type {:?}, given: {}({})", + col.column_name, + info.semantic_type, + api::v1::SemanticType::try_from(col.semantic_type) + .map(|v| v.as_str_name()) + .unwrap_or("Unknown"), + col.semantic_type, + ), + } + ); + } + + let ts_name = physical_state.time_index_column_name(); + ensure!( + rows.schema.iter().any(|col| col.column_name == ts_name), + InvalidRequestSnafu { + region_id: logical_region_id, + reason: format!("missing required time index column {ts_name}"), + } + ); + + if check_fields { + let field_name = physical_state.field_column_name(); + if !rows.schema.iter().any(|col| col.column_name == field_name) { + let field_meta = + physical_columns + .get(field_name) + .with_context(|| ColumnNotFoundSnafu { + name: field_name, + region_id: logical_region_id, + })?; + Self::fill_missing_field_column(logical_region_id, field_name, field_meta, rows)?; + } + } + + Ok(()) + } + + fn fill_missing_field_column( + logical_region_id: RegionId, + field_name: &str, + field_meta: &ColumnMetadata, + rows: &mut Rows, + ) -> Result<()> { + ensure!( + !field_meta.column_schema.is_default_impure(), + UnexpectedRequestSnafu { + reason: format!( + "unexpected impure default value with region_id: {logical_region_id}, column: {field_name}, default_value: {:?}", + field_meta.column_schema.default_constraint(), + ), + } + ); + + let default_value = field_meta + .column_schema + .create_default() + .context(CreateDefaultSnafu { + region_id: logical_region_id, + column: field_name, + })? + .with_context(|| InvalidRequestSnafu { + region_id: logical_region_id, + reason: format!("missing required field column {field_name}"), + })?; + let default_value = api::helper::to_grpc_value(default_value); + let (datatype, datatype_extension) = + ColumnDataTypeWrapper::try_from(field_meta.column_schema.data_type.clone()) + .map_err(|e| { + InvalidRequestSnafu { + region_id: logical_region_id, + reason: format!( + "no protobuf type for field column {field_name} ({:?}): {e}", + field_meta.column_schema.data_type + ), + } + .build() + })? + .to_parts(); + + rows.schema.push(ColumnSchema { + column_name: field_name.to_string(), + datatype: datatype as i32, + semantic_type: SemanticType::Field as i32, + datatype_extension, + options: None, + }); + + for row in &mut rows.rows { + row.values.push(default_value.clone()); } Ok(()) @@ -536,14 +676,14 @@ impl MetricEngineInner { let input = std::mem::take(rows); let iter = { let state = self.state.read().unwrap(); - let name_to_id = state + let physical_columns = state .physical_region_states() .get(&physical_region_id) .with_context(|| PhysicalRegionNotFoundSnafu { region_id: physical_region_id, })? .physical_columns(); - RowsIter::new(input, name_to_id) + RowsIter::new(input, physical_columns) }; let output = self.row_modifier @@ -557,12 +697,17 @@ impl MetricEngineInner { mod tests { use std::collections::HashSet; + use api::v1::value::ValueData; + use api::v1::{ColumnDataType, ColumnSchema as PbColumnSchema}; use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; use common_function::utils::partition_expr_version; use common_recordbatch::RecordBatches; + use datatypes::prelude::ConcreteDataType; + use datatypes::schema::{ColumnDefaultConstraint, ColumnSchema}; use datatypes::value::Value as PartitionValue; use partition::expr::col; + use store_api::metadata::ColumnMetadata; use store_api::metric_engine_consts::{ DATA_SCHEMA_TABLE_ID_COLUMN_NAME, DATA_SCHEMA_TSID_COLUMN_NAME, MEMTABLE_PARTITION_TREE_PRIMARY_KEY_ENCODING, @@ -1258,4 +1403,271 @@ mod tests { .unwrap(); assert_eq!(response.affected_rows, 3); } + + /// Regression test for issue #7990: the metric engine must reject a row + /// whose timestamp column carries a non-timestamp datatype, rather than + /// letting it panic inside mito's `ValueBuilder::push`. + #[tokio::test] + async fn test_verify_rows_rejects_wrong_type() { + use api::v1::value::ValueData; + use api::v1::{ColumnDataType, ColumnSchema as PbColumnSchema, SemanticType}; + use common_query::prelude::{greptime_timestamp, greptime_value}; + + let env = TestEnv::new().await; + env.init_metric_region().await; + + let logical_region_id = env.default_logical_region_id(); + + // Timestamp column is declared as String — the very payload that + // caused #7990. It should surface a typed error rather than panic. + let schema = vec![ + PbColumnSchema { + column_name: greptime_timestamp().to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Timestamp as _, + datatype_extension: None, + options: None, + }, + PbColumnSchema { + column_name: greptime_value().to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as _, + datatype_extension: None, + options: None, + }, + PbColumnSchema { + column_name: "job".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as _, + datatype_extension: None, + options: None, + }, + ]; + let rows = vec![Row { + values: vec![ + Value { + value_data: Some(ValueData::StringValue("not-a-timestamp".to_string())), + }, + Value { + value_data: Some(ValueData::F64Value(1.0)), + }, + Value { + value_data: Some(ValueData::StringValue("tag_0".to_string())), + }, + ], + }]; + + let err = env + .metric() + .handle_request( + logical_region_id, + RegionRequest::Put(RegionPutRequest { + rows: Rows { schema, rows }, + hint: None, + partition_expr_version: None, + }), + ) + .await + .unwrap_err(); + assert_eq!(err.status_code(), StatusCode::InvalidArguments); + } + + /// The completeness check must reject requests that omit the time index + /// column, since mito cannot default-fill a `TimeIndex` column and would + /// previously panic on the empty builder. + #[tokio::test] + async fn test_verify_rows_rejects_missing_time_index() { + use api::v1::{ColumnDataType, ColumnSchema as PbColumnSchema, SemanticType}; + use common_query::prelude::greptime_value; + + let env = TestEnv::new().await; + env.init_metric_region().await; + + let logical_region_id = env.default_logical_region_id(); + + // Payload only carries the field and a tag — no timestamp column. + let schema = vec![ + PbColumnSchema { + column_name: greptime_value().to_string(), + datatype: ColumnDataType::Float64 as i32, + semantic_type: SemanticType::Field as _, + datatype_extension: None, + options: None, + }, + PbColumnSchema { + column_name: "job".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as _, + datatype_extension: None, + options: None, + }, + ]; + let rows = vec![Row { + values: vec![ + Value { + value_data: Some(api::v1::value::ValueData::F64Value(1.0)), + }, + Value { + value_data: Some(api::v1::value::ValueData::StringValue("tag_0".to_string())), + }, + ], + }]; + + let err = env + .metric() + .handle_request( + logical_region_id, + RegionRequest::Put(RegionPutRequest { + rows: Rows { schema, rows }, + hint: None, + partition_expr_version: None, + }), + ) + .await + .unwrap_err(); + assert_eq!(err.status_code(), StatusCode::InvalidArguments); + } + + #[tokio::test] + async fn test_verify_rows_rejects_missing_field() { + use api::v1::value::ValueData; + use api::v1::{ColumnDataType, ColumnSchema as PbColumnSchema, SemanticType}; + use common_query::prelude::greptime_timestamp; + + let env = TestEnv::new().await; + env.init_metric_region().await; + + let logical_region_id = env.default_logical_region_id(); + + // Schema has timestamp + tag but no field column. + let schema = vec![ + PbColumnSchema { + column_name: greptime_timestamp().to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as _, + datatype_extension: None, + options: None, + }, + PbColumnSchema { + column_name: "job".to_string(), + datatype: ColumnDataType::String as i32, + semantic_type: SemanticType::Tag as _, + datatype_extension: None, + options: None, + }, + ]; + let rows = vec![Row { + values: vec![ + Value { + value_data: Some(ValueData::TimestampMillisecondValue(0)), + }, + Value { + value_data: Some(ValueData::StringValue("tag_0".to_string())), + }, + ], + }]; + + let err = env + .metric() + .handle_request( + logical_region_id, + RegionRequest::Put(RegionPutRequest { + rows: Rows { schema, rows }, + hint: None, + partition_expr_version: None, + }), + ) + .await + .unwrap_err(); + let message = err.to_string(); + assert!( + message.contains("missing required field column"), + "expected field-completeness rejection, got: {message}" + ); + assert_eq!(err.status_code(), StatusCode::InvalidArguments); + } + + #[test] + fn test_fill_missing_field_column_nullable_no_default() { + let field_meta = ColumnMetadata { + column_id: 1, + semantic_type: SemanticType::Field, + column_schema: ColumnSchema::new( + "greptime_value".to_string(), + ConcreteDataType::float64_datatype(), + true, // nullable, no default + ), + }; + let mut rows = Rows { + schema: vec![PbColumnSchema { + column_name: "ts".to_string(), + datatype: ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as _, + datatype_extension: None, + options: None, + }], + rows: vec![Row { + values: vec![Value { + value_data: Some(ValueData::TimestampMillisecondValue(0)), + }], + }], + }; + + MetricEngineInner::fill_missing_field_column( + RegionId::new(1, 1), + "greptime_value", + &field_meta, + &mut rows, + ) + .unwrap(); + + assert_eq!(rows.schema.len(), 2); + assert_eq!(rows.schema[1].column_name, "greptime_value"); + assert_eq!(rows.rows[0].values.len(), 2); + assert!( + rows.rows[0].values[1].value_data.is_none(), + "missing nullable field should be filled with null" + ); + } + + #[test] + fn test_fill_missing_field_column_rejects_impure_default() { + let field_meta = ColumnMetadata { + column_id: 1, + semantic_type: SemanticType::Field, + column_schema: ColumnSchema::new( + "greptime_value".to_string(), + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ) + .with_default_constraint(Some(ColumnDefaultConstraint::Function("now()".to_string()))) + .unwrap(), + }; + let mut rows = Rows { + schema: vec![PbColumnSchema { + column_name: "ts".to_string(), + datatype: api::v1::ColumnDataType::TimestampMillisecond as i32, + semantic_type: SemanticType::Timestamp as _, + datatype_extension: None, + options: None, + }], + rows: vec![Row { + values: vec![Value { + value_data: Some(ValueData::TimestampMillisecondValue(0)), + }], + }], + }; + + let err = MetricEngineInner::fill_missing_field_column( + RegionId::new(1, 1), + "greptime_value", + &field_meta, + &mut rows, + ) + .unwrap_err(); + assert!( + err.to_string().contains("impure default value"), + "expected impure-default rejection, got: {err}" + ); + } } diff --git a/src/metric-engine/src/engine/state.rs b/src/metric-engine/src/engine/state.rs index 7c64758f48..25416380f1 100644 --- a/src/metric-engine/src/engine/state.rs +++ b/src/metric-engine/src/engine/state.rs @@ -16,11 +16,13 @@ use std::collections::{HashMap, HashSet}; +use api::v1::SemanticType; +use common_telemetry::warn; use common_time::timestamp::TimeUnit; use snafu::OptionExt; use store_api::codec::PrimaryKeyEncoding; use store_api::metadata::ColumnMetadata; -use store_api::storage::{ColumnId, RegionId}; +use store_api::storage::RegionId; use crate::engine::options::PhysicalRegionOptions; use crate::error::{PhysicalRegionNotFoundSnafu, Result}; @@ -29,7 +31,16 @@ use crate::utils::to_data_region_id; pub struct PhysicalRegionState { logical_regions: HashSet, - physical_columns: HashMap, + physical_columns: HashMap, + /// Name of the time index column, cached at region load so that the write + /// path doesn't have to scan `physical_columns` for the timestamp on every + /// row batch. The time index is fixed at region creation and never + /// changes, so this stays in sync with `physical_columns`. + time_index_column_name: String, + /// Name of the field column. Metric regions have exactly one field column + /// verified at creation time, so the write path can validate completeness + /// without consulting per-logical-region metadata. + field_column_name: String, primary_key_encoding: PrimaryKeyEncoding, options: PhysicalRegionOptions, time_index_unit: TimeUnit, @@ -37,14 +48,29 @@ pub struct PhysicalRegionState { impl PhysicalRegionState { pub fn new( - physical_columns: HashMap, + physical_columns: HashMap, primary_key_encoding: PrimaryKeyEncoding, options: PhysicalRegionOptions, time_index_unit: TimeUnit, ) -> Self { + // Safety: a valid physical region always has exactly one time index + // column; callers validate this before reaching here (see + // `create_data_region_request` and the open path). + let time_index_column_name = physical_columns + .iter() + .find(|(_, meta)| meta.semantic_type == SemanticType::Timestamp) + .map(|(name, _)| name.clone()) + .unwrap_or_default(); + let field_column_name = physical_columns + .iter() + .find(|(_, meta)| meta.semantic_type == SemanticType::Field) + .map(|(name, _)| name.clone()) + .unwrap_or_default(); Self { logical_regions: HashSet::new(), physical_columns, + time_index_column_name, + field_column_name, primary_key_encoding, options, time_index_unit, @@ -57,10 +83,20 @@ impl PhysicalRegionState { } /// Returns a reference to the physical columns. - pub fn physical_columns(&self) -> &HashMap { + pub fn physical_columns(&self) -> &HashMap { &self.physical_columns } + /// Returns the cached name of the time index column. + pub fn time_index_column_name(&self) -> &str { + &self.time_index_column_name + } + + /// Returns the cached name of the field column. + pub fn field_column_name(&self) -> &str { + &self.field_column_name + } + /// Returns a reference to the physical region options. pub fn options(&self) -> &PhysicalRegionOptions { &self.options @@ -90,7 +126,7 @@ impl MetricEngineState { pub fn add_physical_region( &mut self, physical_region_id: RegionId, - physical_columns: HashMap, + physical_columns: HashMap, primary_key_encoding: PrimaryKeyEncoding, options: PhysicalRegionOptions, time_index_unit: TimeUnit, @@ -112,12 +148,25 @@ impl MetricEngineState { pub fn add_physical_columns( &mut self, physical_region_id: RegionId, - physical_columns: impl IntoIterator, + physical_columns: impl IntoIterator, ) { let physical_region_id = to_data_region_id(physical_region_id); let state = self.physical_regions.get_mut(&physical_region_id).unwrap(); - for (col, id) in physical_columns { - state.physical_columns.insert(col, id); + for (col, meta) in physical_columns { + // The time index is fixed at region creation and alter cannot add + // a new one; keep the cached name in sync defensively. + debug_assert_ne!( + meta.semantic_type, + SemanticType::Timestamp, + "unexpected time index column {col} added to an existing physical region" + ); + if meta.semantic_type == SemanticType::Field { + warn!( + "Unexpected field column {col} added to physical region {physical_region_id}; cached field column remains {}", + state.field_column_name + ); + } + state.physical_columns.insert(col, meta); } } diff --git a/src/metric-engine/src/error.rs b/src/metric-engine/src/error.rs index 6fb3406826..284b1b0298 100644 --- a/src/metric-engine/src/error.rs +++ b/src/metric-engine/src/error.rs @@ -343,6 +343,19 @@ pub enum Error { location: Location, }, + #[snafu(display( + "Failed to create default value for column {} of region {}", + column, + region_id + ))] + CreateDefault { + region_id: RegionId, + column: String, + source: datatypes::error::Error, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Unexpected request: {}", reason))] UnexpectedRequest { reason: String, @@ -394,7 +407,8 @@ impl ErrorExt for Error { | UnsupportedAlterKind { .. } | UnsupportedRemapManifestsRequest { .. } | UnsupportedSyncRegionFromRequest { .. } - | InvalidRequest { .. } => StatusCode::InvalidArguments, + | InvalidRequest { .. } + | CreateDefault { .. } => StatusCode::InvalidArguments, ForbiddenPhysicalAlter { .. } | UnsupportedRegionRequest { .. } diff --git a/src/metric-engine/src/row_modifier.rs b/src/metric-engine/src/row_modifier.rs index d5ed1cc9b0..7d68f94aa0 100644 --- a/src/metric-engine/src/row_modifier.rs +++ b/src/metric-engine/src/row_modifier.rs @@ -23,6 +23,7 @@ use mito_codec::row_converter::SparsePrimaryKeyCodec; use smallvec::SmallVec; use snafu::ResultExt; use store_api::codec::PrimaryKeyEncoding; +use store_api::metadata::ColumnMetadata; use store_api::metric_engine_consts::{ DATA_SCHEMA_TABLE_ID_COLUMN_NAME, DATA_SCHEMA_TSID_COLUMN_NAME, }; @@ -264,7 +265,10 @@ struct IterIndex { } impl IterIndex { - fn new(row_schema: &[ColumnSchema], name_to_column_id: &HashMap) -> Self { + fn new( + row_schema: &[ColumnSchema], + physical_columns: &HashMap, + ) -> Self { let mut reserved_indices = SmallVec::<[ValueIndex; 2]>::new(); // Uses BTreeMap to keep the primary key column name order (lexicographical) let mut primary_key_indices = BTreeMap::new(); @@ -290,7 +294,10 @@ impl IterIndex { primary_key_indices.insert( col.column_name.as_str(), ValueIndex { - column_id: *name_to_column_id.get(&col.column_name).unwrap(), + column_id: physical_columns + .get(&col.column_name) + .unwrap() + .column_id, index: idx, }, ); @@ -298,13 +305,13 @@ impl IterIndex { }, SemanticType::Field => { field_indices.push(ValueIndex { - column_id: *name_to_column_id.get(&col.column_name).unwrap(), + column_id: physical_columns.get(&col.column_name).unwrap().column_id, index: idx, }); } SemanticType::Timestamp => { ts_index = Some(ValueIndex { - column_id: *name_to_column_id.get(&col.column_name).unwrap(), + column_id: physical_columns.get(&col.column_name).unwrap().column_id, index: idx, }); } @@ -338,8 +345,8 @@ pub struct RowsIter { } impl RowsIter { - pub fn new(rows: Rows, name_to_column_id: &HashMap) -> Self { - let index: IterIndex = IterIndex::new(&rows.schema, name_to_column_id); + pub fn new(rows: Rows, physical_columns: &HashMap) -> Self { + let index: IterIndex = IterIndex::new(&rows.schema, physical_columns); Self { rows, index } } @@ -455,8 +462,23 @@ mod tests { } } - fn test_name_to_column_id() -> HashMap { - HashMap::from([("namespace".to_string(), 1), ("host".to_string(), 2)]) + fn make_info(name: &str, column_id: ColumnId) -> ColumnMetadata { + ColumnMetadata { + column_schema: datatypes::schema::ColumnSchema::new( + name.to_string(), + datatypes::prelude::ConcreteDataType::string_datatype(), + false, + ), + semantic_type: SemanticType::Tag, + column_id, + } + } + + fn test_name_to_column_id() -> HashMap { + HashMap::from([ + ("namespace".to_string(), make_info("namespace", 1)), + ("host".to_string(), make_info("host", 2)), + ]) } #[test] @@ -657,11 +679,11 @@ mod tests { } /// Helper function to create a name_to_column_id map - fn create_name_to_column_id(labels: &[&str]) -> HashMap { + fn create_name_to_column_id(labels: &[&str]) -> HashMap { labels .iter() .enumerate() - .map(|(idx, name)| (name.to_string(), idx as ColumnId + 1)) + .map(|(idx, name)| (name.to_string(), make_info(name, idx as ColumnId + 1))) .collect() } @@ -692,7 +714,7 @@ mod tests { fn extract_tsid( schema: Vec, row: Row, - name_to_column_id: &HashMap, + name_to_column_id: &HashMap, table_id: TableId, ) -> u64 { let rows = Rows { diff --git a/src/mito2/src/engine/scan_test.rs b/src/mito2/src/engine/scan_test.rs index 119b4493fd..c1a5e0f321 100644 --- a/src/mito2/src/engine/scan_test.rs +++ b/src/mito2/src/engine/scan_test.rs @@ -15,10 +15,13 @@ use std::collections::BTreeMap; use api::v1::Rows; +use common_base::readable_size::ReadableSize; use common_error::ext::ErrorExt; use common_error::status_code::StatusCode; use common_recordbatch::RecordBatches; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_common::ScalarValue; +use datafusion_expr::{col, lit}; use datatypes::arrow::array::AsArray; use datatypes::arrow::datatypes::{Float64Type, TimestampMillisecondType}; use futures::TryStreamExt; @@ -403,3 +406,193 @@ fn collect_and_assert_partition_rows( actual_rows.sort_by(|a, b| a.0.cmp(&b.0).then(a.2.cmp(&b.2))); actual_rows } + +/// Tests series scan with multiple partition ranges (each with multiple overlapping sources) +/// and small semaphore permits (controlled by num_partitions). +#[tokio::test] +async fn test_series_scan_flat_small_permits() { + let mut env = TestEnv::with_prefix("test_series_scan_small_permits").await; + let engine = env + .create_engine(MitoConfig { + default_flat_format: true, + ..Default::default() + }) + .await; + + let region_id = RegionId::new(1, 1); + let request = CreateRequestBuilder::new() + .insert_option("compaction.type", "twcs") + .insert_option("compaction.twcs.time_window", "1h") + .build(); + let column_schemas = test_util::rows_schema(&request); + + engine + .handle_request(region_id, RegionRequest::Create(request)) + .await + .unwrap(); + + // Create overlapping SSTs in each time window so partition ranges have multiple sources. + let put_flush_rows = async |start, end| { + let rows = Rows { + schema: column_schemas.clone(), + rows: test_util::build_rows(start, end), + }; + test_util::put_rows(&engine, region_id, rows).await; + test_util::flush_region(&engine, region_id, None).await; + }; + // Window 0 (0s-999s): 3 overlapping SSTs + put_flush_rows(0, 3).await; + put_flush_rows(1, 5).await; + put_flush_rows(3, 7).await; + // Window 1 (3600s-4599s): 2 overlapping SSTs + put_flush_rows(3600, 3603).await; + put_flush_rows(3601, 3605).await; + // Window 2 (7200s-8199s): 2 overlapping SSTs + put_flush_rows(7200, 7203).await; + put_flush_rows(7201, 7204).await; + + let mut expected_rows = Vec::new(); + for value in [ + 0_i64, 1, 2, 3, 4, 5, 6, 3600, 3601, 3602, 3603, 3604, 7200, 7201, 7202, 7203, + ] { + expected_rows.push((value.to_string(), value as f64, value * 1000)); + } + expected_rows.sort_by(|a, b| a.0.cmp(&b.0).then(a.2.cmp(&b.2))); + + // Test with different semaphore sizes (num_partitions controls Semaphore::new(num_partitions)). + for num_partitions in [1, 2] { + let request = ScanRequest { + distribution: Some(TimeSeriesDistribution::PerSeries), + ..Default::default() + }; + let scanner = engine.scanner(region_id, request).await.unwrap(); + let Scanner::Series(mut scanner) = scanner else { + panic!("Scanner should be series scan"); + }; + + // Collect all partition ranges and redistribute into `num_partitions` partitions. + let raw_ranges: Vec<_> = scanner + .properties() + .partitions + .iter() + .flatten() + .cloned() + .collect(); + assert!( + raw_ranges.len() >= 3, + "expected at least 3 partition ranges, got {}", + raw_ranges.len() + ); + + let mut new_ranges = vec![vec![]; num_partitions]; + for (i, range) in raw_ranges.into_iter().enumerate() { + new_ranges[i % num_partitions].push(range); + } + scanner + .prepare(PrepareRequest { + ranges: Some(new_ranges), + ..Default::default() + }) + .unwrap(); + + let actual_rows = collect_partition_rows_round_robin(&scanner, num_partitions).await; + assert_eq!( + expected_rows, actual_rows, + "mismatch with num_partitions={num_partitions}" + ); + } +} + +// Regression test: `ts = a OR ts = b` extracts to a `TimestampRange` that +// `GenericRange::or` widens into `[min(a, b), max(a, b) + 1)`. Two such +// predicates with different `a` values can both extract to ranges that cover +// the same partition while selecting different (or no) rows. The previous +// cover check would strip both predicates from the cache key, letting the +// second scan return the first scan's cached row. +#[tokio::test] +async fn test_range_cache_separates_or_equality_time_filters() { + let mut env = TestEnv::new().await; + let engine = env + .create_engine(MitoConfig { + default_flat_format: true, + // Explicitly enable the range result cache: the bug only reproduces + // when the second scan can replay the first scan's cached batches. + range_result_cache_size: ReadableSize::mb(64), + ..Default::default() + }) + .await; + + let region_id = RegionId::new(1, 1); + let request = CreateRequestBuilder::new().build(); + let column_schemas = test_util::rows_schema(&request); + + engine + .handle_request(region_id, RegionRequest::Create(request)) + .await + .unwrap(); + + // Partition rows ts=5..10 (5000ms..9000ms), flushed to SST. + test_util::put_rows( + &engine, + region_id, + Rows { + schema: column_schemas.clone(), + rows: test_util::build_rows(5, 10), + }, + ) + .await; + test_util::flush_region(&engine, region_id, None).await; + + let ts_lit = |ms: i64| lit(ScalarValue::TimestampMillisecond(Some(ms), None)); + let tag_filter = || col("tag_0").gt_eq(lit(ScalarValue::Utf8(Some("0".to_string())))); + + // First scan: (ts = 5000) OR (ts = 100000) -- extracts to `[5000, 100001)`, + // which covers the partition `[5000, 9000]`. Selects ts=5. + let stream = engine + .scan_to_stream( + region_id, + ScanRequest { + filters: vec![ + tag_filter(), + col("ts").eq(ts_lit(5000)).or(col("ts").eq(ts_lit(100000))), + ], + ..Default::default() + }, + ) + .await + .unwrap(); + let batches = RecordBatches::try_collect(stream).await.unwrap(); + let expected_first = "\ ++-------+---------+---------------------+ +| tag_0 | field_0 | ts | ++-------+---------+---------------------+ +| 5 | 5.0 | 1970-01-01T00:00:05 | ++-------+---------+---------------------+"; + assert_eq!(expected_first, batches.pretty_print().unwrap()); + + // Second scan: (ts = 3000) OR (ts = 100000) -- extracts to `[3000, 100001)`, + // which also covers the partition. Selects nothing. With the buggy cover + // check both scans built the same cache key (tag filter only), so this scan + // would replay the first scan's cached row and incorrectly return ts=5. + let stream = engine + .scan_to_stream( + region_id, + ScanRequest { + filters: vec![ + tag_filter(), + col("ts").eq(ts_lit(3000)).or(col("ts").eq(ts_lit(100000))), + ], + ..Default::default() + }, + ) + .await + .unwrap(); + let batches = RecordBatches::try_collect(stream).await.unwrap(); + let row_count: usize = batches.iter().map(|b| b.num_rows()).sum(); + assert_eq!( + 0, + row_count, + "expected empty result, got: {}", + batches.pretty_print().unwrap() + ); +} diff --git a/src/mito2/src/read/range_cache.rs b/src/mito2/src/read/range_cache.rs index 2431a21f6a..59ce279c6f 100644 --- a/src/mito2/src/read/range_cache.rs +++ b/src/mito2/src/read/range_cache.rs @@ -18,7 +18,6 @@ use std::mem; use std::sync::Arc; use async_stream::try_stream; -use common_time::range::TimestampRange; use datatypes::arrow::array::{Array, AsArray, DictionaryArray}; use datatypes::arrow::datatypes::UInt32Type; use datatypes::arrow::record_batch::RecordBatch; @@ -32,7 +31,6 @@ use crate::read::BoxedRecordBatchStream; use crate::read::scan_region::StreamContext; use crate::read::scan_util::PartitionMetrics; use crate::region::options::MergeMode; -use crate::sst::file::FileTimeRange; use crate::sst::parquet::flat_format::primary_key_column_index; /// Fingerprint of the scan request fields that affect partition range cache reuse. @@ -135,6 +133,7 @@ impl ScanRequestFingerprint { .unwrap_or(&[]) } + #[allow(dead_code)] pub(crate) fn without_time_filters(&self) -> Self { Self { inner: Arc::clone(&self.inner), @@ -266,15 +265,17 @@ pub(crate) fn build_range_cache_key( return None; } - let range_meta = &stream_ctx.ranges[part_range.identifier]; - let scan = if query_time_range_covers_partition_range( - stream_ctx.input.time_range.as_ref(), - range_meta.time_range, - ) { - fingerprint.without_time_filters() - } else { - fingerprint.clone() - }; + // TODO(yingwen): We used to call `fingerprint.without_time_filters()` when the query's + // `TimestampRange` fully covered the partition's `FileTimeRange`, so different queries that + // all enclosed the same partition could share a cache entry. The cover check turned out to + // be too coarse: it returned true in cases where the dropped time predicates would still + // have excluded rows, so the cache served results that should have been filtered. Reviving + // the optimization needs a per-predicate implication check that walks each time-only `Expr` + // (recursing through AND/OR/NOT) and proves the predicate is satisfied for every timestamp + // inside the partition's `FileTimeRange` — not the looser "does `extract_time_range_from_expr` + // return a range that covers the partition" used previously. Until then, always carry the + // full fingerprint so cache reuse stays correct. + let scan = fingerprint.clone(); Some(RangeScanCacheKey { region_id: stream_ctx.input.region_metadata().region_id, @@ -283,19 +284,6 @@ pub(crate) fn build_range_cache_key( }) } -#[allow(dead_code)] -fn query_time_range_covers_partition_range( - query_time_range: Option<&TimestampRange>, - partition_time_range: FileTimeRange, -) -> bool { - let Some(query_time_range) = query_time_range else { - return true; - }; - - let (part_start, part_end) = partition_time_range; - query_time_range.contains(&part_start) && query_time_range.contains(&part_end) -} - /// Returns a stream that replays cached record batches. #[allow(dead_code)] pub(crate) fn cached_flat_range_stream(value: Arc) -> BoxedRecordBatchStream { @@ -496,6 +484,7 @@ mod tests { use crate::read::projection::ProjectionMapper; use crate::read::range::{RangeMeta, RowGroupIndex, SourceIndex}; use crate::read::scan_region::{PredicateGroup, ScanInput}; + use crate::sst::file::FileTimeRange; use crate::test_util::memtable_util::metadata_with_primary_key; use crate::test_util::scheduler_util::SchedulerEnv; use crate::test_util::sst_util::sst_file_handle_with_file_id; @@ -557,9 +546,40 @@ mod tests { lit(ScalarValue::TimestampMillisecond(Some(val), None)) } + fn normalized_exprs(exprs: impl IntoIterator) -> Vec { + let mut exprs = exprs + .into_iter() + .map(|expr| expr.to_string()) + .collect::>(); + exprs.sort_unstable(); + exprs + } + + async fn assert_range_cache_filters( + filters: Vec, + query_time_range: Option, + partition_time_range: FileTimeRange, + expected_filters: Vec, + expected_time_filters: Vec, + ) { + let (stream_ctx, part_range) = + new_stream_context(filters, query_time_range, partition_time_range).await; + + let key = build_range_cache_key(&stream_ctx, &part_range).unwrap(); + + assert_eq!( + key.scan.filters(), + normalized_exprs(expected_filters).as_slice() + ); + assert_eq!( + key.scan.time_filters(), + normalized_exprs(expected_time_filters).as_slice() + ); + } + #[tokio::test] - async fn strips_time_only_filters_when_query_covers_partition_range() { - let (stream_ctx, part_range) = new_stream_context( + async fn preserves_time_filters_when_query_covers_partition_range() { + assert_range_cache_filters( vec![ col("ts").gt_eq(ts_lit(1000)), col("ts").lt(ts_lit(2001)), @@ -571,20 +591,10 @@ mod tests { Timestamp::new_millisecond(1000), Timestamp::new_millisecond(2000), ), + vec![col("k0").eq(lit("foo")), col("ts").is_not_null()], + vec![col("ts").gt_eq(ts_lit(1000)), col("ts").lt(ts_lit(2001))], ) .await; - - let key = build_range_cache_key(&stream_ctx, &part_range).unwrap(); - - // Range-reducible time filters should be cleared when query covers partition range. - assert!(key.scan.time_filters().is_empty()); - // Non-range time predicates stay in filters. - let mut expected_filters = [ - col("k0").eq(lit("foo")).to_string(), - col("ts").is_not_null().to_string(), - ]; - expected_filters.sort_unstable(); - assert_eq!(key.scan.filters(), expected_filters.as_slice()); } #[tokio::test] @@ -613,8 +623,8 @@ mod tests { } #[tokio::test] - async fn strips_time_only_filters_when_query_has_no_time_range_limit() { - let (stream_ctx, part_range) = new_stream_context( + async fn preserves_time_filters_when_query_has_no_time_range_limit() { + assert_range_cache_filters( vec![ col("ts").gt_eq(ts_lit(1000)), col("ts").is_not_null(), @@ -625,20 +635,10 @@ mod tests { Timestamp::new_millisecond(1000), Timestamp::new_millisecond(2000), ), + vec![col("k0").eq(lit("foo")), col("ts").is_not_null()], + vec![col("ts").gt_eq(ts_lit(1000))], ) .await; - - let key = build_range_cache_key(&stream_ctx, &part_range).unwrap(); - - // Range-reducible time filters should be cleared when query has no time range limit. - assert!(key.scan.time_filters().is_empty()); - // Non-range time predicates stay in filters. - let mut expected_filters = [ - col("k0").eq(lit("foo")).to_string(), - col("ts").is_not_null().to_string(), - ]; - expected_filters.sort_unstable(); - assert_eq!(key.scan.filters(), expected_filters.as_slice()); } #[test] diff --git a/src/mito2/src/read/scan_region.rs b/src/mito2/src/read/scan_region.rs index cbda7265b6..682e8a6b45 100644 --- a/src/mito2/src/read/scan_region.rs +++ b/src/mito2/src/read/scan_region.rs @@ -1440,16 +1440,21 @@ pub(crate) fn build_scan_fingerprint(input: &ScanInput) -> Option false, }; + // TODO(yingwen): The split between `time_filters` and `filters` is currently inert + // because `build_range_cache_key()` always keeps both in the cache key. We used to + // strip `time_filters` when the query's `TimestampRange` covered the partition's + // `FileTimeRange`, but `extract_time_range_from_expr` is not precise enough to prove + // a time predicate is implied by that range (it can return a wider range than the + // predicate, and it does not analyze AND/OR shapes), which let the cache reuse rows + // that should have been filtered. Reviving the optimization needs a per-predicate + // implication check that walks each time-only `Expr` (recursing through AND/OR/NOT) + // and proves the predicate holds for every timestamp inside the partition's + // `FileTimeRange`; until then both buckets land in the fingerprint. if is_time_only && extract_time_range_from_expr(&time_index_name, ts_col_unit, expr).is_some() { - // Range-reducible time predicates can be safely dropped from the - // cache key when the query time range covers the partition range. time_filters.push(expr.to_string()); } else { - // Non-time filters and non-range time predicates (those that - // extract_time_range_from_expr cannot convert to a TimestampRange) - // always stay in the cache key. filters.push(expr.to_string()); } } diff --git a/src/mito2/src/region/options.rs b/src/mito2/src/region/options.rs index fcf68a9216..1c183bf24d 100644 --- a/src/mito2/src/region/options.rs +++ b/src/mito2/src/region/options.rs @@ -38,6 +38,8 @@ use crate::memtable::partition_tree::{DEFAULT_FREEZE_THRESHOLD, DEFAULT_MAX_KEYS use crate::sst::FormatType; const DEFAULT_INDEX_SEGMENT_ROW_COUNT: usize = 1024; +const COMPACTION_TWCS_PREFIX: &str = "compaction.twcs."; +const MEMTABLE_PARTITION_TREE_PREFIX: &str = "memtable.partition_tree."; pub(crate) fn parse_wal_options( options_map: &HashMap, @@ -137,7 +139,8 @@ impl TryFrom<&HashMap> for RegionOptions { // See https://github.com/serde-rs/serde/issues/1626 let options: RegionOptionsWithoutEnum = serde_json::from_str(&json).context(JsonOptionsSnafu)?; - let has_compaction_type = validate_enum_options(options_map, "compaction.type")?; + let has_compaction_type = + validate_enum_options(options_map, "compaction.type", &[COMPACTION_TWCS_PREFIX])?; let compaction = if has_compaction_type { serde_json::from_str(&json).context(JsonOptionsSnafu)? } else { @@ -147,7 +150,11 @@ impl TryFrom<&HashMap> for RegionOptions { let wal_options = parse_wal_options(options_map).context(JsonOptionsSnafu)?; let index_options: IndexOptions = serde_json::from_str(&json).context(JsonOptionsSnafu)?; - let memtable = if validate_enum_options(options_map, "memtable.type")? { + let memtable = if validate_enum_options( + options_map, + "memtable.type", + &[MEMTABLE_PARTITION_TREE_PREFIX], + )? { Some(serde_json::from_str(&json).context(JsonOptionsSnafu)?) } else { None @@ -436,25 +443,36 @@ fn options_map_to_value(options: &HashMap) -> Value { // `#[serde(default)]` doesn't support enum (https://github.com/serde-rs/serde/issues/1799) so we // check the type key first. /// Validates whether the `options_map` has valid options for specific `enum_tag_key` -/// and returns `true` if the map contains enum options. +/// and returns `true` if the map contains the enum tag. +/// +/// Variant options must start with one of `enum_option_prefixes`. If variant options +/// are provided, the tagged enum type key must also be provided. fn validate_enum_options( options_map: &HashMap, enum_tag_key: &str, + enum_option_prefixes: &[&str], ) -> Result { - let enum_type = enum_tag_key.split('.').next().unwrap(); - let mut has_other_options = false; + let mut has_enum_options = false; let mut has_tag = false; for key in options_map.keys() { if key == enum_tag_key { has_tag = true; - } else if key.starts_with(enum_type) { - has_other_options = true; + } else if !has_enum_options + && enum_option_prefixes + .iter() + .any(|prefix| key.starts_with(prefix)) + { + has_enum_options = true; + } + + if has_tag && has_enum_options { + break; } } // If tag is not provided, then other options for the enum should not exist. ensure!( - has_tag || !has_other_options, + has_tag || !has_enum_options, InvalidRegionOptionsSnafu { reason: format!("missing key {} in options", enum_tag_key), } @@ -537,6 +555,34 @@ mod tests { assert_eq!(expect, options); } + #[test] + fn test_with_compaction_override_true_without_compaction_type() { + let map = make_map(&[(COMPACTION_OVERRIDE, "true")]); + let options = RegionOptions::try_from(&map).unwrap(); + let expect = RegionOptions { + compaction_override: true, + ..Default::default() + }; + assert_eq!(expect, options); + } + + #[test] + fn test_with_compaction_override_false_without_compaction_type() { + let map = make_map(&[(COMPACTION_OVERRIDE, "false")]); + let options = RegionOptions::try_from(&map).unwrap(); + assert_eq!(RegionOptions::default(), options); + } + + #[test] + fn test_compaction_twcs_options_still_require_compaction_type_with_override() { + let map = make_map(&[ + (COMPACTION_OVERRIDE, "true"), + ("compaction.twcs.time_window", "2h"), + ]); + let err = RegionOptions::try_from(&map).unwrap_err(); + assert_eq!(StatusCode::InvalidArguments, err.status_code()); + } + fn test_with_wal_options(wal_options: &WalOptions) -> bool { let encoded_wal_options = serde_json::to_string(&wal_options).unwrap(); let map = make_map(&[(WAL_OPTIONS_KEY, &encoded_wal_options)]); @@ -607,6 +653,13 @@ mod tests { assert_eq!(StatusCode::InvalidArguments, err.status_code()); } + #[test] + fn test_without_memtable_type() { + let map = make_map(&[("memtable.partition_tree.index_max_keys_per_shard", "2048")]); + let err = RegionOptions::try_from(&map).unwrap_err(); + assert_eq!(StatusCode::InvalidArguments, err.status_code()); + } + #[test] fn test_with_merge_mode() { let map = make_map(&[("merge_mode", "last_row")]); diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index af81325aa9..126b47ab63 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -40,6 +40,7 @@ common-time.workspace = true datafusion.workspace = true datafusion-common.workspace = true datafusion-expr.workspace = true +datafusion-expr-common.workspace = true datafusion-functions.workspace = true datafusion-optimizer.workspace = true datafusion-physical-expr.workspace = true diff --git a/src/query/src/optimizer.rs b/src/query/src/optimizer.rs index aaac1e3124..752cac1436 100644 --- a/src/query/src/optimizer.rs +++ b/src/query/src/optimizer.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod const_normalization; pub mod constant_term; pub mod count_nest_aggr; pub mod count_wildcard; diff --git a/src/query/src/optimizer/const_normalization.rs b/src/query/src/optimizer/const_normalization.rs new file mode 100644 index 0000000000..fac0e90538 --- /dev/null +++ b/src/query/src/optimizer/const_normalization.rs @@ -0,0 +1,1133 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use arrow_schema::{DataType, TimeUnit as ArrowTimeUnit}; +use datafusion::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRecursion, TreeNodeRewriter}; +use datafusion_common::{DFSchemaRef, Result, ScalarValue}; +use datafusion_expr::expr::{Cast, InList, Like, TryCast}; +use datafusion_expr::{Between, BinaryExpr, Expr, ExprSchemable, LogicalPlan, Operator, lit}; +use datafusion_expr_common::casts::try_cast_literal_to_type; +use datafusion_optimizer::analyzer::AnalyzerRule; + +use crate::plan::ExtractExpr; + +/// ConstNormalizationRule rewrites castable constants against their +/// non-constant comparison operand ahead of filter pushdown. +#[derive(Debug)] +pub struct ConstNormalizationRule; + +impl AnalyzerRule for ConstNormalizationRule { + fn analyze(&self, plan: LogicalPlan, _config: &ConfigOptions) -> Result { + plan.transform(|plan| match plan { + LogicalPlan::Filter(filter) => { + let schema = filter.input.schema().clone(); + rewrite_plan_exprs(LogicalPlan::Filter(filter), schema) + } + LogicalPlan::TableScan(scan) => { + let schema = scan.projected_schema.clone(); + rewrite_plan_exprs(LogicalPlan::TableScan(scan), schema) + } + _ => Ok(Transformed::no(plan)), + }) + .map(|x| x.data) + } + + fn name(&self) -> &str { + "ConstNormalizationRule" + } +} + +fn rewrite_plan_exprs(plan: LogicalPlan, schema: DFSchemaRef) -> Result> { + let mut rewriter = ConstNormalizationRewriter { + schema, + transformed: false, + }; + let exprs = plan + .expressions_consider_join() + .into_iter() + .map(|expr| expr.rewrite(&mut rewriter).map(|rewritten| rewritten.data)) + .collect::>>()?; + if !rewriter.transformed { + return Ok(Transformed::no(plan)); + } + + let inputs = plan.inputs().into_iter().cloned().collect::>(); + plan.with_new_exprs(exprs, inputs).map(Transformed::yes) +} + +struct ConstNormalizationRewriter { + schema: DFSchemaRef, + transformed: bool, +} + +impl TreeNodeRewriter for ConstNormalizationRewriter { + type Node = Expr; + + fn f_down(&mut self, expr: Expr) -> Result> { + let recursion = if matches!( + expr, + Expr::Exists(_) | Expr::InSubquery(_) | Expr::ScalarSubquery(_) + ) { + TreeNodeRecursion::Jump + } else { + TreeNodeRecursion::Continue + }; + + Ok(Transformed::new(expr, false, recursion)) + } + + fn f_up(&mut self, expr: Expr) -> Result> { + let rewritten = rewrite_expr_node(expr, &self.schema)?; + self.transformed |= rewritten.transformed; + Ok(rewritten) + } +} + +fn rewrite_expr_node(expr: Expr, schema: &DFSchemaRef) -> Result> { + match expr { + Expr::BinaryExpr(binary) => match rewrite_binary_expr(binary.clone(), schema)? { + Some(expr) => Ok(Transformed::yes(expr)), + None => Ok(Transformed::no(Expr::BinaryExpr(binary))), + }, + Expr::Between(between) => match rewrite_between_expr(between.clone(), schema)? { + Some(expr) => Ok(Transformed::yes(expr)), + None => Ok(Transformed::no(Expr::Between(between))), + }, + Expr::InList(in_list) => match rewrite_in_list_expr(in_list.clone(), schema)? { + Some(expr) => Ok(Transformed::yes(expr)), + None => Ok(Transformed::no(Expr::InList(in_list))), + }, + Expr::Like(like) => rewrite_like_expr(like, PatternMatchKind::Like, schema), + Expr::SimilarTo(like) => rewrite_like_expr(like, PatternMatchKind::SimilarTo, schema), + expr => Ok(Transformed::no(expr)), + } +} + +fn rewrite_between_expr(between: Between, schema: &DFSchemaRef) -> Result> { + let Between { + expr, + negated, + low, + high, + } = between; + let expr = *expr; + let low_expr = *low; + let high_expr = *high; + let Some((target, constants)) = + extract_rewrite_operands(&expr, &[low_expr.clone(), high_expr.clone()], schema)? + else { + return Ok(None); + }; + + if let Some(mut constants) = target.normalize_constants(&constants) { + let high = constants + .pop() + .expect("between normalization expects high constant"); + let low = constants + .pop() + .expect("between normalization expects low constant"); + return Ok(Some(Expr::Between(Between { + expr: Box::new(target.expr.clone()), + negated, + low: Box::new(lit(low)), + high: Box::new(lit(high)), + }))); + } + + Ok((!negated) + .then(|| target.normalize_timestamp_between(&constants[0], &constants[1])) + .flatten()) +} + +fn rewrite_in_list_expr(in_list: InList, schema: &DFSchemaRef) -> Result> { + let InList { + expr, + list, + negated, + } = in_list; + let expr = *expr; + let Some((target, constants)) = extract_rewrite_operands(&expr, &list, schema)? else { + return Ok(None); + }; + + Ok(target.normalize_constants(&constants).map(|constants| { + target + .expr + .clone() + .in_list(constants.into_iter().map(lit).collect(), negated) + })) +} + +fn rewrite_like_expr( + like: Like, + kind: PatternMatchKind, + schema: &DFSchemaRef, +) -> Result> { + let original = match kind { + PatternMatchKind::Like => Expr::Like(like.clone()), + PatternMatchKind::SimilarTo => Expr::SimilarTo(like.clone()), + }; + let Like { + negated, + expr, + pattern, + escape_char, + case_insensitive, + } = like; + let expr = *expr; + let pattern = *pattern; + let Some((target, constants)) = + extract_rewrite_operands(&expr, std::slice::from_ref(&pattern), schema)? + else { + return Ok(Transformed::no(original)); + }; + let Some(mut constants) = target.normalize_constants(&constants) else { + return Ok(Transformed::no(original)); + }; + + let pattern = lit(constants + .pop() + .expect("pattern normalization expects one constant")); + let like = Like::new( + negated, + Box::new(target.expr.clone()), + Box::new(pattern), + escape_char, + case_insensitive, + ); + let rewritten = match kind { + PatternMatchKind::Like => Expr::Like(like), + PatternMatchKind::SimilarTo => Expr::SimilarTo(like), + }; + Ok(Transformed::yes(rewritten)) +} + +fn rewrite_binary_expr(binary: BinaryExpr, schema: &DFSchemaRef) -> Result> { + if !binary.op.supports_propagation() { + return Ok(None); + } + + let BinaryExpr { left, op, right } = binary; + let left = *left; + let right = *right; + if let Some(expr) = rewrite_binary_side(left.clone(), op, right.clone(), schema)? { + return Ok(Some(expr)); + } + + let Some(swapped_op) = op.swap() else { + return Ok(None); + }; + + rewrite_binary_side(right, swapped_op, left, schema) +} + +fn rewrite_binary_side( + target_expr: Expr, + op: Operator, + constant_expr: Expr, + schema: &DFSchemaRef, +) -> Result> { + let Some((target, constants)) = + extract_rewrite_operands(&target_expr, std::slice::from_ref(&constant_expr), schema)? + else { + return Ok(None); + }; + + if let Some(mut constants) = target.normalize_constants(&constants) { + let constant = constants + .pop() + .expect("binary normalization expects one constant"); + return Ok(Some(Expr::BinaryExpr(BinaryExpr { + left: Box::new(target.expr.clone()), + op, + right: Box::new(lit(constant)), + }))); + } + + Ok(target.normalize_timestamp_binary(op, &constants[0])) +} + +fn extract_rewrite_operands( + target_expr: &Expr, + constant_exprs: &[Expr], + schema: &DFSchemaRef, +) -> Result)>> { + let Some(target) = extract_normalization_target(target_expr, schema)? else { + return Ok(None); + }; + + extract_constant_scalars(constant_exprs) + .map(|constants| constants.map(|constants| (target, constants))) +} + +#[derive(Clone)] +struct NormalizationTarget { + expr: Expr, + data_type: DataType, + kind: NormalizationKind, +} + +#[derive(Clone)] +enum NormalizationKind { + /// The cast preserves every source value exactly, so literals can be cast directly. + Lossless, + /// The cast drops timestamp precision and must widen predicate bounds to preserve semantics. + TimestampDowncast { + source_unit: ArrowTimeUnit, + target_unit: ArrowTimeUnit, + timezone: Option>, + }, +} + +impl NormalizationTarget { + /// Normalizes constants for rewrites that can preserve the original predicate with a direct + /// literal cast. Timestamp precision-changing casts are handled by timestamp-specific helpers. + fn normalize_constants(&self, constants: &[ScalarValue]) -> Option> { + constants + .iter() + .map(|constant| self.normalize_constant(constant)) + .collect() + } + + fn normalize_constant(&self, constant: &ScalarValue) -> Option { + match self.kind { + NormalizationKind::TimestampDowncast { .. } => None, + NormalizationKind::Lossless => try_cast_literal_to_type(constant, &self.data_type), + } + } + + /// Rewrites predicates over timestamp downcasts into source-side half-open bounds. + fn normalize_timestamp_binary(&self, op: Operator, constant: &ScalarValue) -> Option { + let NormalizationKind::TimestampDowncast { + source_unit, + target_unit, + timezone, + } = &self.kind + else { + return None; + }; + + let constant = constant + .cast_to(&DataType::Timestamp(*target_unit, timezone.clone())) + .ok()?; + let value = timestamp_scalar_value(&constant)?; + let bound = match op { + Operator::GtEq => lower_bound_for_ge(value, *source_unit, *target_unit)?, + Operator::Gt => lower_bound_for_ge(value.checked_add(1)?, *source_unit, *target_unit)?, + Operator::Lt => lower_bound_for_ge(value, *source_unit, *target_unit)?, + Operator::LtEq => { + lower_bound_for_ge(value.checked_add(1)?, *source_unit, *target_unit)? + } + _ => return None, + }; + + let normalized_op = match op { + Operator::GtEq | Operator::Gt => Operator::GtEq, + Operator::Lt | Operator::LtEq => Operator::Lt, + _ => return None, + }; + + Some(match normalized_op { + Operator::GtEq => self.expr.clone().gt_eq(lit(timestamp_scalar( + *source_unit, + timezone.clone(), + bound, + ))), + Operator::Lt => { + self.expr + .clone() + .lt(lit(timestamp_scalar(*source_unit, timezone.clone(), bound))) + } + _ => unreachable!("timestamp normalization only rewrites to >= or <"), + }) + } + + /// Rewrites `BETWEEN` over timestamp downcasts into an inclusive lower bound and exclusive + /// upper bound over the source timestamp unit. + fn normalize_timestamp_between(&self, low: &ScalarValue, high: &ScalarValue) -> Option { + let NormalizationKind::TimestampDowncast { + source_unit, + target_unit, + timezone, + } = &self.kind + else { + return None; + }; + + let target_type = DataType::Timestamp(*target_unit, timezone.clone()); + let low = low.cast_to(&target_type).ok()?; + let high = high.cast_to(&target_type).ok()?; + let low = timestamp_scalar_value(&low)?; + let high = timestamp_scalar_value(&high)?; + + let lower = lower_bound_for_ge(low, *source_unit, *target_unit)?; + let upper = lower_bound_for_ge(high.checked_add(1)?, *source_unit, *target_unit)?; + + Some( + self.expr + .clone() + .gt_eq(lit(timestamp_scalar(*source_unit, timezone.clone(), lower))) + .and(self.expr.clone().lt(lit(timestamp_scalar( + *source_unit, + timezone.clone(), + upper, + )))), + ) + } +} + +/// Returns the non-constant side we should normalize against. +/// +/// Plain expressions normalize literals to their own type. Cast expressions only participate when +/// the cast is lossless or when timestamp downcasts can be rewritten as wider source-side bounds. +fn extract_normalization_target( + expr: &Expr, + schema: &DFSchemaRef, +) -> Result> { + if extract_constant_scalar(expr)?.is_some() { + return Ok(None); + } + + let Some((_, source_expr, target_type)) = extract_cast_input(expr) else { + return Ok(Some(NormalizationTarget { + expr: expr.clone(), + data_type: expr.get_type(schema)?, + kind: NormalizationKind::Lossless, + })); + }; + + let data_type = source_expr.get_type(schema)?; + let Some(kind) = classify_normalization_kind(&data_type, target_type) else { + return Ok(None); + }; + + Ok(Some(NormalizationTarget { + expr: source_expr.clone(), + data_type, + kind, + })) +} + +fn classify_normalization_kind( + source_type: &DataType, + target_type: &DataType, +) -> Option { + // Timestamp casts that change precision need boundary-aware rewrites. A finer target literal + // may not map exactly back to the coarser source unit, so the generic lossless path is only + // safe for timestamp casts that keep the same unit. + if is_lossless_cast(source_type, target_type) { + return Some(NormalizationKind::Lossless); + } + + match (source_type, target_type) { + ( + DataType::Timestamp(source_unit, source_tz), + DataType::Timestamp(target_unit, target_tz), + ) if source_tz == target_tz + && time_unit_rank(*source_unit) > time_unit_rank(*target_unit) => + { + Some(NormalizationKind::TimestampDowncast { + source_unit: *source_unit, + target_unit: *target_unit, + timezone: source_tz.clone(), + }) + } + _ => None, + } +} + +/// Returns whether every value of `source_type` is representable in `target_type`. +fn is_lossless_cast(source_type: &DataType, target_type: &DataType) -> bool { + match (source_type, target_type) { + (DataType::Int8, DataType::Int16 | DataType::Int32 | DataType::Int64) + | (DataType::Int16, DataType::Int32 | DataType::Int64) + | (DataType::Int32, DataType::Int64) + | (DataType::UInt8, DataType::UInt16 | DataType::UInt32 | DataType::UInt64) + | (DataType::UInt8, DataType::Int16 | DataType::Int32 | DataType::Int64) + | (DataType::UInt16, DataType::UInt32 | DataType::UInt64) + | (DataType::UInt16, DataType::Int32 | DataType::Int64) + | (DataType::UInt32, DataType::UInt64 | DataType::Int64) + | (DataType::Utf8, DataType::Utf8View | DataType::LargeUtf8) => true, + ( + DataType::Timestamp(source_unit, source_tz), + DataType::Timestamp(target_unit, target_tz), + ) => source_tz == target_tz && source_unit == target_unit, + _ => false, + } +} + +#[derive(Clone, Copy)] +enum PatternMatchKind { + Like, + SimilarTo, +} + +fn extract_constant_scalars(exprs: &[Expr]) -> Result>> { + let mut values = Vec::with_capacity(exprs.len()); + for expr in exprs { + let Some(value) = extract_constant_scalar(expr)? else { + return Ok(None); + }; + values.push(value); + } + + Ok(Some(values)) +} + +/// Extracts a literal scalar from an expression, folding constant `CAST` and `TRY_CAST` nodes. +fn extract_constant_scalar(expr: &Expr) -> Result> { + if let Some(value) = expr.as_literal() { + return Ok(Some(value.clone())); + } + + let Some((kind, expr, data_type)) = extract_cast_input(expr) else { + return Ok(None); + }; + + match kind { + CastInputKind::Cast => extract_constant_scalar(expr)? + .map(|value| value.cast_to(data_type)) + .transpose(), + CastInputKind::TryCast => { + Ok(extract_constant_scalar(expr)?.and_then(|value| value.cast_to(data_type).ok())) + } + } +} + +#[derive(Clone, Copy)] +enum CastInputKind { + Cast, + TryCast, +} + +/// Returns the input expression and target type for `CAST` and `TRY_CAST` expressions. +fn extract_cast_input(expr: &Expr) -> Option<(CastInputKind, &Expr, &DataType)> { + match expr { + Expr::Cast(Cast { expr, data_type }) => { + Some((CastInputKind::Cast, expr.as_ref(), data_type)) + } + Expr::TryCast(TryCast { expr, data_type }) => { + Some((CastInputKind::TryCast, expr.as_ref(), data_type)) + } + _ => None, + } +} + +fn time_unit_rank(unit: ArrowTimeUnit) -> usize { + match unit { + ArrowTimeUnit::Second => 0, + ArrowTimeUnit::Millisecond => 1, + ArrowTimeUnit::Microsecond => 2, + ArrowTimeUnit::Nanosecond => 3, + } +} + +fn time_unit_scale(unit: ArrowTimeUnit) -> i64 { + match unit { + ArrowTimeUnit::Second => 1, + ArrowTimeUnit::Millisecond => 1_000, + ArrowTimeUnit::Microsecond => 1_000_000, + ArrowTimeUnit::Nanosecond => 1_000_000_000, + } +} + +/// Returns the number of source-unit ticks in one target-unit tick for finer-to-coarser casts. +fn finer_to_coarser_ratio(source_unit: ArrowTimeUnit, target_unit: ArrowTimeUnit) -> Option { + let source_scale = time_unit_scale(source_unit); + let target_scale = time_unit_scale(target_unit); + (source_scale >= target_scale).then_some(source_scale / target_scale) +} + +/// Returns the smallest source-unit timestamp whose downcast is greater than or equal to +/// `target_value`. +/// +/// DataFusion timestamp downcasts truncate toward zero. For non-positive buckets that means the +/// bucket starts before `target_value * ratio`, so `<= x` can be rewritten as `< lower_bound(x+1)` +/// without dropping rows near zero or across negative boundaries. +fn lower_bound_for_ge( + target_value: i64, + source_unit: ArrowTimeUnit, + target_unit: ArrowTimeUnit, +) -> Option { + let ratio = finer_to_coarser_ratio(source_unit, target_unit)?; + let base = target_value.checked_mul(ratio)?; + if target_value <= 0 { + base.checked_sub(ratio - 1) + } else { + Some(base) + } +} + +fn timestamp_scalar_value(value: &ScalarValue) -> Option { + match value { + ScalarValue::TimestampSecond(Some(value), _) + | ScalarValue::TimestampMillisecond(Some(value), _) + | ScalarValue::TimestampMicrosecond(Some(value), _) + | ScalarValue::TimestampNanosecond(Some(value), _) => Some(*value), + _ => None, + } +} + +fn timestamp_scalar(unit: ArrowTimeUnit, timezone: Option>, value: i64) -> ScalarValue { + match unit { + ArrowTimeUnit::Second => ScalarValue::TimestampSecond(Some(value), timezone), + ArrowTimeUnit::Millisecond => ScalarValue::TimestampMillisecond(Some(value), timezone), + ArrowTimeUnit::Microsecond => ScalarValue::TimestampMicrosecond(Some(value), timezone), + ArrowTimeUnit::Nanosecond => ScalarValue::TimestampNanosecond(Some(value), timezone), + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use arrow_schema::{DataType, TimeUnit as ArrowTimeUnit}; + use async_trait::async_trait; + use common_time::Timestamp; + use common_time::range::TimestampRange; + use common_time::timestamp::TimeUnit; + use datafusion::catalog::Session; + use datafusion::config::ConfigOptions; + use datafusion::datasource::{TableProvider, provider_as_source}; + use datafusion::physical_plan::ExecutionPlan; + use datafusion_common::arrow::datatypes::Field; + use datafusion_common::{DFSchema, ScalarValue, ToDFSchema}; + use datafusion_expr::expr::{Between, Like}; + use datafusion_expr::expr_fn::{cast, col, try_cast}; + use datafusion_expr::{ + Expr, LogicalPlan, LogicalPlanBuilder, TableProviderFilterPushDown, TableScan, TableSource, + TableType, lit, + }; + use datafusion_optimizer::analyzer::AnalyzerRule; + use datafusion_optimizer::optimizer::{Optimizer, OptimizerContext}; + use datafusion_optimizer::push_down_filter::PushDownFilter; + use table::predicate::build_time_range_predicate; + + use super::{ + ConstNormalizationRule, PatternMatchKind, lower_bound_for_ge, try_cast_literal_to_type, + }; + + #[test] + fn test_normalize_direct_integer_cast_comparison() { + assert_filter_plan( + vec![Field::new("v", DataType::Int32, false)], + cast(col("v"), DataType::Int64).gt_eq(lit(42_i64)), + "Filter: t.v >= Int32(42)\n TableScan: t", + ); + } + + #[test] + fn test_normalize_non_column_operand() { + assert_filter_plan( + vec![Field::new("v", DataType::Int32, false)], + cast(col("v") + lit(1_i32), DataType::Int64).gt_eq(lit(42_i64)), + "Filter: t.v + Int32(1) >= Int32(42)\n TableScan: t", + ); + } + + #[test] + fn test_normalize_swapped_binary_comparison() { + assert_filter_plan( + vec![Field::new("v", DataType::Int16, false)], + lit(42_i64).lt_eq(cast(col("v"), DataType::Int64)), + "Filter: t.v >= Int16(42)\n TableScan: t", + ); + } + + #[test] + fn test_normalize_try_cast_target() { + assert_filter_plan( + vec![Field::new("v", DataType::Int16, false)], + try_cast(col("v"), DataType::Int64).gt_eq(lit(42_i64)), + "Filter: t.v >= Int16(42)\n TableScan: t", + ); + } + + #[test] + fn test_normalize_casted_constants() { + let fields = vec![Field::new("v", DataType::Int16, false)]; + let cases = [ + ( + col("v").gt_eq(cast(lit(42_i8), DataType::Int64)), + "Filter: t.v >= Int16(42)\n TableScan: t", + ), + ( + col("v").in_list( + vec![ + cast(lit(1_i8), DataType::Int64), + try_cast(lit(2_i8), DataType::Int64), + ], + false, + ), + "Filter: t.v IN ([Int16(1), Int16(2)])\n TableScan: t", + ), + ]; + + for (predicate, expected) in cases { + assert_filter_plan(fields.clone(), predicate, expected); + } + } + + #[test] + fn test_normalize_plain_integer_literals() { + let fields = vec![Field::new("v", DataType::Int16, false)]; + let cases = [ + ( + col("v").gt_eq(lit(42_i64)), + "Filter: t.v >= Int16(42)\n TableScan: t", + ), + ( + col("v").in_list(vec![lit(1_i64), lit(2_i64)], false), + "Filter: t.v IN ([Int16(1), Int16(2)])\n TableScan: t", + ), + ( + col("v").between(lit(3_i64), lit(5_i64)), + "Filter: t.v BETWEEN Int16(3) AND Int16(5)\n TableScan: t", + ), + ]; + + for (predicate, expected) in cases { + assert_filter_plan(fields.clone(), predicate, expected); + } + } + + #[test] + fn test_normalize_unsigned_to_signed_literals() { + let cases = [ + ( + vec![Field::new("v", DataType::UInt8, false)], + cast(col("v"), DataType::Int16).lt_eq(lit(255_i16)), + "Filter: t.v <= UInt8(255)\n TableScan: t", + ), + ( + vec![Field::new("v", DataType::UInt16, false)], + cast(col("v"), DataType::Int32).gt_eq(lit(42_i32)), + "Filter: t.v >= UInt16(42)\n TableScan: t", + ), + ( + vec![Field::new("v", DataType::UInt32, false)], + cast(col("v"), DataType::Int64).between(lit(3_i64), lit(5_i64)), + "Filter: t.v BETWEEN UInt32(3) AND UInt32(5)\n TableScan: t", + ), + ]; + + for (fields, predicate, expected) in cases { + assert_filter_plan(fields, predicate, expected); + } + } + + #[test] + fn test_normalize_in_list_and_between() { + let fields = vec![Field::new("v", DataType::Int16, false)]; + let cases = [ + ( + cast(col("v"), DataType::Int64).in_list(vec![lit(1_i64), lit(2_i64)], false), + "Filter: t.v IN ([Int16(1), Int16(2)])\n TableScan: t", + ), + ( + cast(col("v"), DataType::Int64).between(lit(3_i64), lit(5_i64)), + "Filter: t.v BETWEEN Int16(3) AND Int16(5)\n TableScan: t", + ), + ]; + + for (predicate, expected) in cases { + assert_filter_plan(fields.clone(), predicate, expected); + } + } + + #[test] + fn test_keep_non_lossless_literal_unchanged() { + assert_filter_plan( + vec![Field::new("v", DataType::Int16, false)], + col("v").gt_eq(lit(100_000_i64)), + "Filter: t.v >= Int64(100000)\n TableScan: t", + ); + } + + #[test] + fn test_normalize_scan_filters() { + let scan = build_scan_plan(test_schema(vec![Field::new("v", DataType::Int16, false)])); + let LogicalPlan::TableScan(scan) = scan else { + panic!("expected table scan"); + }; + let plan = LogicalPlan::TableScan(TableScan { + filters: vec![cast(col("v"), DataType::Int64).gt_eq(lit(42_i64))], + ..scan + }); + + let analyzed = analyze_plan(plan); + + assert_eq!( + vec![col("v").gt_eq(lit(42_i16))], + extract_scan_filters(&analyzed) + ); + } + + #[test] + fn test_normalize_negated_between() { + assert_filter_plan( + vec![Field::new("v", DataType::Int16, false)], + Expr::Between(Between { + expr: Box::new(cast(col("v"), DataType::Int64)), + negated: true, + low: Box::new(lit(3_i64)), + high: Box::new(lit(5_i64)), + }), + "Filter: t.v NOT BETWEEN Int16(3) AND Int16(5)\n TableScan: t", + ); + } + + #[test] + fn test_normalize_like_literal() { + assert_pattern_match_plan( + PatternMatchKind::Like, + ScalarValue::LargeUtf8(Some("api%".to_string())), + "Filter: t.s LIKE Utf8(\"api%\")\n TableScan: t", + ); + } + + #[test] + fn test_normalize_similar_to_literal() { + assert_pattern_match_plan( + PatternMatchKind::SimilarTo, + ScalarValue::LargeUtf8(Some("api.*".to_string())), + "Filter: t.s SIMILAR TO Utf8(\"api.*\")\n TableScan: t", + ); + } + + #[test] + fn test_normalize_direct_timestamp_filter() { + assert_timestamp_pushdown( + vec![ + Field::new( + "ts", + DataType::Timestamp(ArrowTimeUnit::Nanosecond, None), + false, + ), + Field::new("tag", DataType::Utf8, true), + ], + ts_cast_to_ms() + .gt_eq(ts_ms_literal(-299_999)) + .and(ts_cast_to_ms().lt_eq(ts_ms_literal(10_000))) + .and(col("tag").eq(lit("api"))), + "Filter: t.ts >= TimestampNanosecond(-299999999999, None) AND t.ts < TimestampNanosecond(10001000000, None) AND t.tag = Utf8(\"api\")\n TableScan: t", + "TableScan: t, full_filters=[t.ts >= TimestampNanosecond(-299999999999, None), t.ts < TimestampNanosecond(10001000000, None), t.tag = Utf8(\"api\")]", + TimestampRange::new_inclusive( + Some(Timestamp::new_nanosecond(-299_999_999_999)), + Some(Timestamp::new_nanosecond(10_000_999_999)), + ), + ); + } + + #[test] + fn test_normalize_timestamp_between_filter() { + assert_timestamp_pushdown( + vec![Field::new( + "ts", + DataType::Timestamp(ArrowTimeUnit::Nanosecond, None), + false, + )], + ts_cast_to_ms().between(ts_ms_literal(-299_999), ts_ms_literal(10_000)), + "Filter: t.ts >= TimestampNanosecond(-299999999999, None) AND t.ts < TimestampNanosecond(10001000000, None)\n TableScan: t", + "TableScan: t, full_filters=[t.ts >= TimestampNanosecond(-299999999999, None), t.ts < TimestampNanosecond(10001000000, None)]", + TimestampRange::new_inclusive( + Some(Timestamp::new_nanosecond(-299_999_999_999)), + Some(Timestamp::new_nanosecond(10_000_999_999)), + ), + ); + } + + #[test] + fn test_normalize_strict_timestamp_filter() { + assert_timestamp_pushdown( + vec![Field::new( + "ts", + DataType::Timestamp(ArrowTimeUnit::Nanosecond, None), + false, + )], + ts_cast_to_ms() + .gt(ts_ms_literal(10_000)) + .and(ts_cast_to_ms().lt(ts_ms_literal(20_000))), + "Filter: t.ts >= TimestampNanosecond(10001000000, None) AND t.ts < TimestampNanosecond(20000000000, None)\n TableScan: t", + "TableScan: t, full_filters=[t.ts >= TimestampNanosecond(10001000000, None), t.ts < TimestampNanosecond(20000000000, None)]", + TimestampRange::new_inclusive( + Some(Timestamp::new_nanosecond(10_001_000_000)), + Some(Timestamp::new_nanosecond(19_999_999_999)), + ), + ); + } + + #[test] + fn test_normalize_zero_boundary_timestamp_filter() { + let fields = vec![Field::new( + "ts", + DataType::Timestamp(ArrowTimeUnit::Nanosecond, None), + false, + )]; + + assert_timestamp_pushdown( + fields.clone(), + ts_cast_to_ms().gt_eq(ts_ms_literal(0)), + "Filter: t.ts >= TimestampNanosecond(-999999, None)\n TableScan: t", + "TableScan: t, full_filters=[t.ts >= TimestampNanosecond(-999999, None)]", + TimestampRange::from_start(Timestamp::new_nanosecond(-999_999)), + ); + + assert_timestamp_pushdown( + fields.clone(), + ts_cast_to_ms().lt(ts_ms_literal(0)), + "Filter: t.ts < TimestampNanosecond(-999999, None)\n TableScan: t", + "TableScan: t, full_filters=[t.ts < TimestampNanosecond(-999999, None)]", + TimestampRange::until_end(Timestamp::new_nanosecond(-999_999), false), + ); + + assert_timestamp_pushdown( + fields, + ts_cast_to_ms().between(ts_ms_literal(0), ts_ms_literal(0)), + "Filter: t.ts >= TimestampNanosecond(-999999, None) AND t.ts < TimestampNanosecond(1000000, None)\n TableScan: t", + "TableScan: t, full_filters=[t.ts >= TimestampNanosecond(-999999, None), t.ts < TimestampNanosecond(1000000, None)]", + TimestampRange::new_inclusive( + Some(Timestamp::new_nanosecond(-999_999)), + Some(Timestamp::new_nanosecond(999_999)), + ), + ); + } + + #[test] + fn test_timestamp_downcast_contract_matches_datafusion_casts() { + let cases = [ + (-1_000_001, -1), + (-1_000_000, -1), + (-999_999, 0), + (-1, 0), + (0, 0), + (999_999, 0), + (1_000_000, 1), + ]; + + for (source, expected) in cases { + let casted = try_cast_literal_to_type( + &ScalarValue::TimestampNanosecond(Some(source), None), + &DataType::Timestamp(ArrowTimeUnit::Millisecond, None), + ) + .unwrap(); + assert_eq!( + ScalarValue::TimestampMillisecond(Some(expected), None), + casted + ); + } + + assert_eq!( + Some(-1_999_999), + lower_bound_for_ge(-1, ArrowTimeUnit::Nanosecond, ArrowTimeUnit::Millisecond) + ); + assert_eq!( + Some(-999_999), + lower_bound_for_ge(0, ArrowTimeUnit::Nanosecond, ArrowTimeUnit::Millisecond) + ); + assert_eq!( + Some(1_000_000), + lower_bound_for_ge(1, ArrowTimeUnit::Nanosecond, ArrowTimeUnit::Millisecond) + ); + } + + #[test] + fn test_normalize_plain_timestamp_literals() { + assert_timestamp_pushdown( + vec![Field::new( + "ts", + DataType::Timestamp(ArrowTimeUnit::Nanosecond, None), + false, + )], + col("ts") + .gt_eq(ts_ms_literal(-299_999)) + .and(col("ts").lt_eq(ts_ms_literal(10_000))), + "Filter: t.ts >= TimestampNanosecond(-299999000000, None) AND t.ts <= TimestampNanosecond(10000000000, None)\n TableScan: t", + "TableScan: t, full_filters=[t.ts >= TimestampNanosecond(-299999000000, None), t.ts <= TimestampNanosecond(10000000000, None)]", + TimestampRange::new_inclusive( + Some(Timestamp::new_nanosecond(-299_999_000_000)), + Some(Timestamp::new_nanosecond(10_000_000_000)), + ), + ); + } + + #[test] + fn test_keep_timestamp_upcast_filter_unchanged() { + assert_filter_plan( + vec![Field::new( + "ts", + DataType::Timestamp(ArrowTimeUnit::Millisecond, None), + false, + )], + cast( + col("ts"), + DataType::Timestamp(ArrowTimeUnit::Nanosecond, None), + ) + .gt_eq(lit(ScalarValue::TimestampNanosecond(Some(1), None))), + "Filter: CAST(t.ts AS Timestamp(ns)) >= TimestampNanosecond(1, None)\n TableScan: t", + ); + } + + fn assert_pattern_match_plan(kind: PatternMatchKind, pattern: ScalarValue, expected: &str) { + let predicate = match kind { + PatternMatchKind::Like => Expr::Like(Like::new( + false, + Box::new(cast(col("s"), DataType::LargeUtf8)), + Box::new(lit(pattern)), + None, + false, + )), + PatternMatchKind::SimilarTo => Expr::SimilarTo(Like::new( + false, + Box::new(cast(col("s"), DataType::LargeUtf8)), + Box::new(lit(pattern)), + None, + false, + )), + }; + + assert_filter_plan( + vec![Field::new("s", DataType::Utf8, false)], + predicate, + expected, + ); + } + + fn assert_filter_plan(fields: Vec, predicate: Expr, expected: &str) { + assert_eq!(expected, analyze_filter(fields, predicate).to_string()); + } + + fn assert_timestamp_pushdown( + fields: Vec, + predicate: Expr, + expected_analyzed: &str, + expected_pushed: &str, + expected_range: TimestampRange, + ) { + let analyzed = analyze_filter(fields, predicate); + assert_eq!(expected_analyzed, analyzed.to_string()); + + let pushed = push_down_filters(analyzed); + assert_eq!(expected_pushed, pushed.to_string()); + + let range = + build_time_range_predicate("ts", TimeUnit::Nanosecond, &extract_scan_filters(&pushed)); + assert_eq!(expected_range, range); + } + + fn analyze_filter(fields: Vec, predicate: Expr) -> LogicalPlan { + analyze_plan(build_filter_plan(test_schema(fields), predicate)) + } + + fn analyze_plan(plan: LogicalPlan) -> LogicalPlan { + ConstNormalizationRule + .analyze(plan, &ConfigOptions::default()) + .unwrap() + } + + fn build_filter_plan(schema: Arc, predicate: Expr) -> LogicalPlan { + LogicalPlanBuilder::scan("t", test_source(schema), None) + .unwrap() + .filter(predicate) + .unwrap() + .build() + .unwrap() + } + + fn build_scan_plan(schema: Arc) -> LogicalPlan { + LogicalPlanBuilder::scan("t", test_source(schema), None) + .unwrap() + .build() + .unwrap() + } + + fn push_down_filters(plan: LogicalPlan) -> LogicalPlan { + Optimizer::with_rules(vec![Arc::new(PushDownFilter::new())]) + .optimize(plan, &OptimizerContext::new(), |_, _| {}) + .unwrap() + } + + fn ts_cast_to_ms() -> Expr { + cast( + col("ts"), + DataType::Timestamp(ArrowTimeUnit::Millisecond, None), + ) + } + + fn ts_ms_literal(value: i64) -> Expr { + lit(ScalarValue::TimestampMillisecond(Some(value), None)) + } + + fn extract_scan_filters(plan: &LogicalPlan) -> Vec { + match plan { + LogicalPlan::TableScan(scan) => scan.filters.clone(), + _ => plan + .inputs() + .into_iter() + .flat_map(extract_scan_filters) + .collect(), + } + } + + fn test_schema(fields: Vec) -> Arc { + arrow_schema::Schema::new(fields).to_dfschema_ref().unwrap() + } + + fn test_source(schema: Arc) -> Arc { + let table = ExactPushdownProvider { + schema: Arc::new(schema.as_ref().as_arrow().clone()), + }; + provider_as_source(Arc::new(table)) + } + + #[derive(Debug)] + struct ExactPushdownProvider { + schema: arrow_schema::SchemaRef, + } + + #[async_trait] + impl TableProvider for ExactPushdownProvider { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn schema(&self) -> arrow_schema::SchemaRef { + self.schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + async fn scan( + &self, + _state: &dyn Session, + _projection: Option<&Vec>, + _filters: &[Expr], + _limit: Option, + ) -> datafusion::error::Result> { + unreachable!("scan should not be called in const_normalization tests") + } + + fn supports_filters_pushdown( + &self, + filters: &[&Expr], + ) -> datafusion::error::Result> { + Ok(vec![TableProviderFilterPushDown::Exact; filters.len()]) + } + } +} diff --git a/src/query/src/optimizer/windowed_sort.rs b/src/query/src/optimizer/windowed_sort.rs index 3d3993d454..55a48c1869 100644 --- a/src/query/src/optimizer/windowed_sort.rs +++ b/src/query/src/optimizer/windowed_sort.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; use std::sync::Arc; +use arrow_schema::DataType; use datafusion::physical_optimizer::PhysicalOptimizerRule; use datafusion::physical_plan::ExecutionPlan; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -26,7 +26,8 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_common::Result as DataFusionResult; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_physical_expr::expressions::Column as PhysicalColumn; +use datafusion_physical_expr::expressions::{CastExpr, Column as PhysicalColumn}; +use datafusion_physical_expr::{PhysicalExpr, ScalarFunctionExpr}; use store_api::region_engine::PartitionRange; use table::table::scan::RegionScanExec; @@ -91,9 +92,11 @@ impl WindowedSortPhysicalRule { .expr .as_any() .downcast_ref::() - && scanner_info - .time_index - .contains(input_schema.field(column_expr.index()).name()) + && matches!( + input_schema.field(column_expr.index()).data_type(), + DataType::Timestamp(_, _) + ) + && is_time_index_expr(&sort_input, &first_sort_expr.expr)? && sort_exec.fetch().is_none() // skip if there is a limit, as dyn filter along is good enough in this case { @@ -154,14 +157,11 @@ impl WindowedSortPhysicalRule { #[derive(Debug)] struct ScannerInfo { partition_ranges: Vec>, - time_index: HashSet, tag_columns: Vec, } fn fetch_partition_range(input: Arc) -> DataFusionResult> { let mut partition_ranges = None; - let mut time_index = HashSet::new(); - let mut alias_map = Vec::new(); let mut tag_columns = None; input.transform_up(|plan| { @@ -184,18 +184,6 @@ fn fetch_partition_range(input: Arc) -> DataFusionResult() { - for expr in projection.expr() { - if let Some(column_expr) = expr.expr.as_any().downcast_ref::() { - alias_map.push((column_expr.name().to_string(), expr.alias.clone())); - } - } - // resolve alias properly - time_index = resolve_alias(&alias_map, &time_index); - } - if let Some(region_scan_exec) = plan.as_any().downcast_ref::() { // `PerSeries` distribution is not supported in windowed sort. if region_scan_exec.distribution() @@ -206,8 +194,6 @@ fn fetch_partition_range(input: Arc) -> DataFusionResult) -> DataFusionResult) -> DataFusionResult, + expr: &Arc, +) -> DataFusionResult { + if let Some(column_expr) = expr.as_any().downcast_ref::() { + return is_time_index_column(plan, column_expr); + } + + if let Some(cast_expr) = expr.as_any().downcast_ref::() { + return if matches!(cast_expr.cast_type(), DataType::Timestamp(_, _)) { + is_time_index_expr(plan, cast_expr.expr()) + } else { + Ok(false) + }; + } + + if let Some(scalar_function_expr) = expr.as_any().downcast_ref::() { + return if is_supported_time_index_wrapper(scalar_function_expr) + && scalar_function_expr.args().len() == 1 + { + is_time_index_expr(plan, &scalar_function_expr.args()[0]) + } else { + Ok(false) + }; + } + + Ok(false) +} + +fn is_time_index_column( + plan: &Arc, + column_expr: &PhysicalColumn, +) -> DataFusionResult { + if let Some(projection) = plan.as_any().downcast_ref::() { + let Some(projection_expr) = projection.expr().get(column_expr.index()) else { + return Ok(false); + }; + return is_time_index_expr(projection.input(), &projection_expr.expr); + } + + if let Some(filter) = plan.as_any().downcast_ref::() { + let child_column_expr = filter + .projection() + .as_ref() + .and_then(|projection| projection.get(column_expr.index()).copied()) + .map(|input_index| { + PhysicalColumn::new( + filter.input().schema().field(input_index).name(), + input_index, + ) + }) + .unwrap_or_else(|| column_expr.clone()); + let child_expr = Arc::new(child_column_expr) as Arc; + return is_time_index_expr(filter.input(), &child_expr); + } + + if let Some(region_scan_exec) = plan.as_any().downcast_ref::() { + let schema = plan.schema(); + let column_field = schema.field(column_expr.index()); + return Ok( + matches!(column_field.data_type(), DataType::Timestamp(_, _)) + && *column_field.name() == region_scan_exec.time_index(), + ); + } + + let Some(child) = passthrough_child(plan.as_ref()) else { + return Ok(false); + }; + let child_expr = Arc::new(column_expr.clone()) as Arc; + is_time_index_expr(&child, &child_expr) +} + +fn passthrough_child(plan: &dyn ExecutionPlan) -> Option> { + if plan.as_any().is::() + || plan.as_any().is::() + || plan.as_any().is::() + { + return schema_preserving_child(plan); + } + + None +} + +fn schema_preserving_child(plan: &dyn ExecutionPlan) -> Option> { + let child = plan.children().first().cloned().cloned()?; + (plan.schema().as_ref() == child.schema().as_ref()).then_some(child) +} + +fn is_supported_time_index_wrapper(expr: &ScalarFunctionExpr) -> bool { + (expr.name().eq_ignore_ascii_case("to_timestamp") + || expr.name().eq_ignore_ascii_case("to_timestamp_seconds") + || expr.name().eq_ignore_ascii_case("to_timestamp_millis") + || expr.name().eq_ignore_ascii_case("to_timestamp_micros") + || expr.name().eq_ignore_ascii_case("to_timestamp_nanos")) + && matches!(expr.return_type(), DataType::Timestamp(_, _)) +} + /// Removes the repartition plan between the filter and region scan. fn remove_repartition( plan: Arc, @@ -248,80 +330,321 @@ fn remove_repartition( }) } -/// Resolves alias of the time index column. -/// -/// i.e if a is time index, alias= {a:b, b:c}, then result should be {a, b}(not {a, c}) because projection is not transitive -/// if alias={b:a} and a is time index, then return empty -fn resolve_alias(alias_map: &[(String, String)], time_index: &HashSet) -> HashSet { - // available old name for time index - let mut avail_old_name = time_index.clone(); - let mut new_time_index = HashSet::new(); - for (old, new) in alias_map { - if time_index.contains(old) { - new_time_index.insert(new.clone()); - } else if time_index.contains(new) && old != new { - // other alias to time index, remove the old name - avail_old_name.remove(new); - continue; - } - } - // add the remaining time index that is not in alias map - new_time_index.extend(avail_old_name); - new_time_index -} - #[cfg(test)] mod test { - use itertools::Itertools; + use std::sync::Arc; + + use api::v1::SemanticType; + use arrow_schema::{Field, TimeUnit}; + use common_recordbatch::RecordBatches; + use datafusion::config::ConfigOptions; + use datafusion::physical_plan::filter::FilterExecBuilder; + use datafusion_common::ScalarValue; + use datafusion_functions::datetime::to_timestamp_millis; + use datafusion_physical_expr::expressions::{CastExpr, Literal}; + use datatypes::data_type::ConcreteDataType; + use datatypes::schema::{ColumnSchema, Schema}; + use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder}; + use store_api::region_engine::SinglePartitionScanner; + use store_api::storage::{RegionId, ScanRequest}; use super::*; #[test] - fn test_alias() { - let testcases = [ - // notice the old name is still in the result - ( - vec![("a", "b"), ("b", "c")], - HashSet::from(["a"]), - HashSet::from(["a", "b"]), - ), - // alias swap - ( - vec![("b", "a"), ("a", "b")], - HashSet::from(["a"]), - HashSet::from(["b"]), - ), - ( - vec![("b", "a"), ("b", "c")], - HashSet::from(["a"]), - HashSet::from([]), - ), - // not in alias map - ( - vec![("c", "d"), ("d", "c")], - HashSet::from(["a"]), - HashSet::from(["a"]), - ), - // no alias - (vec![], HashSet::from(["a"]), HashSet::from(["a"])), - // empty time index - (vec![], HashSet::from([]), HashSet::from([])), - ]; - for (alias_map, time_index, expected) in testcases { - let alias_map = alias_map - .into_iter() - .map(|(k, v)| (k.to_string(), v.to_string())) - .collect_vec(); - let time_index = time_index.into_iter().map(|i| i.to_string()).collect(); - let expected: HashSet = expected.into_iter().map(|i| i.to_string()).collect(); + fn test_is_time_index_expr_tracks_aliases_through_projection() { + let scan = new_region_scan(); + let projection = Arc::new( + ProjectionExec::try_new( + vec![( + Arc::new(PhysicalColumn::new("ts", 1)) as Arc, + "alias_ts".to_string(), + )], + scan, + ) + .unwrap(), + ) as Arc; + let expr = Arc::new(PhysicalColumn::new("alias_ts", 0)) as Arc; - assert_eq!( - expected, - resolve_alias(&alias_map, &time_index), - "alias_map={:?}, time_index={:?}", - alias_map, - time_index - ); - } + assert!(is_time_index_expr(&projection, &expr).unwrap()); + } + + #[test] + fn test_is_time_index_expr_tracks_multi_level_aliases() { + let scan = new_region_scan(); + let first_projection = Arc::new( + ProjectionExec::try_new( + vec![( + Arc::new(PhysicalColumn::new("ts", 1)) as Arc, + "alias_1".to_string(), + )], + scan, + ) + .unwrap(), + ) as Arc; + let second_projection = Arc::new( + ProjectionExec::try_new( + vec![( + Arc::new(PhysicalColumn::new("alias_1", 0)) as Arc, + "alias_2".to_string(), + )], + first_projection, + ) + .unwrap(), + ) as Arc; + let expr = Arc::new(PhysicalColumn::new("alias_2", 0)) as Arc; + + assert!(is_time_index_expr(&second_projection, &expr).unwrap()); + } + + #[test] + fn test_is_time_index_expr_tracks_wrapped_aliases_through_projection() { + let scan = new_region_scan(); + let config = Arc::new(ConfigOptions::default()); + let return_field = Arc::new(Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + )); + let projection = Arc::new( + ProjectionExec::try_new( + vec![( + Arc::new(ScalarFunctionExpr::new( + "to_timestamp_millis", + to_timestamp_millis(config.as_ref()), + vec![Arc::new(PhysicalColumn::new("ts", 1))], + return_field, + config, + )) as Arc, + "ts".to_string(), + )], + scan, + ) + .unwrap(), + ) as Arc; + let expr = Arc::new(PhysicalColumn::new("ts", 0)) as Arc; + + assert!(is_time_index_expr(&projection, &expr).unwrap()); + } + + #[test] + fn test_is_time_index_expr_tracks_cast_aliases_through_projection() { + let scan = new_region_scan(); + let projection = Arc::new( + ProjectionExec::try_new( + vec![( + Arc::new(CastExpr::new( + Arc::new(PhysicalColumn::new("ts", 1)), + DataType::Timestamp(TimeUnit::Millisecond, None), + None, + )) as Arc, + "ts_ms".to_string(), + )], + scan, + ) + .unwrap(), + ) as Arc; + let expr = Arc::new(PhysicalColumn::new("ts_ms", 0)) as Arc; + + assert!(is_time_index_expr(&projection, &expr).unwrap()); + } + + #[test] + fn test_is_time_index_expr_rejects_unsupported_wrappers() { + let scan = new_region_scan(); + let config = Arc::new(ConfigOptions::default()); + let return_field = Arc::new(Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + )); + let projection = Arc::new( + ProjectionExec::try_new( + vec![( + Arc::new(ScalarFunctionExpr::new( + "date_trunc", + to_timestamp_millis(config.as_ref()), + vec![Arc::new(PhysicalColumn::new("ts", 1))], + return_field, + config, + )) as Arc, + "ts".to_string(), + )], + scan, + ) + .unwrap(), + ) as Arc; + let expr = Arc::new(PhysicalColumn::new("ts", 0)) as Arc; + + assert!(!is_time_index_expr(&projection, &expr).unwrap()); + } + + #[test] + fn test_is_supported_time_index_wrapper_ignores_function_name_case() { + let config = Arc::new(ConfigOptions::default()); + let return_field = Arc::new(Field::new( + "ts", + DataType::Timestamp(TimeUnit::Millisecond, None), + false, + )); + let expr = ScalarFunctionExpr::new( + "To_Timestamp_Millis", + to_timestamp_millis(config.as_ref()), + vec![Arc::new(PhysicalColumn::new("ts", 1))], + return_field, + config, + ); + + assert!(is_supported_time_index_wrapper(&expr)); + } + + #[test] + fn test_is_time_index_expr_rejects_non_timestamp_casts() { + let scan = new_region_scan(); + let cast_expr = Arc::new(CastExpr::new( + Arc::new(PhysicalColumn::new("ts", 1)), + DataType::Timestamp(TimeUnit::Millisecond, None), + None, + )) as Arc; + assert!(is_time_index_expr(&scan, &cast_expr).unwrap()); + + let non_timestamp_cast = Arc::new(CastExpr::new( + Arc::new(PhysicalColumn::new("ts", 1)), + DataType::Int64, + None, + )) as Arc; + assert!(!is_time_index_expr(&scan, &non_timestamp_cast).unwrap()); + } + + #[test] + fn test_is_time_index_expr_tracks_time_index_through_filter() { + let scan = new_region_scan(); + let filter = Arc::new( + FilterExec::try_new( + Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))), + scan, + ) + .unwrap(), + ) as Arc; + let expr = Arc::new(PhysicalColumn::new("ts", 1)) as Arc; + + assert!(is_time_index_expr(&filter, &expr).unwrap()); + } + + #[test] + fn test_is_time_index_expr_tracks_time_index_through_passthrough_wrapper_and_filter_projection() + { + let scan = new_region_scan(); + let projected_filter = Arc::new( + FilterExecBuilder::new( + Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))), + scan, + ) + .apply_projection(Some(vec![1])) + .unwrap() + .build() + .unwrap(), + ) as Arc; + let cooperative = + Arc::new(CooperativeExec::new(projected_filter)) as Arc; + let expr = Arc::new(PhysicalColumn::new("ts", 0)) as Arc; + + assert!(is_time_index_expr(&cooperative, &expr).unwrap()); + } + + #[test] + fn test_schema_preserving_child_rejects_schema_changing_projection() { + let scan = new_region_scan(); + let projection = ProjectionExec::try_new( + vec![( + Arc::new(PhysicalColumn::new("ts", 1)) as Arc, + "ts".to_string(), + )], + scan, + ) + .unwrap(); + + assert!(schema_preserving_child(&projection).is_none()); + } + + #[test] + fn test_cooperative_exec_satisfies_passthrough_schema_contract() { + let child = new_region_scan(); + let plan = Arc::new(CooperativeExec::new(child.clone())) as Arc; + + assert_passthrough_schema_contract(plan, child); + } + + #[test] + fn test_repartition_exec_satisfies_passthrough_schema_contract() { + let child = new_region_scan(); + let plan = Arc::new( + RepartitionExec::try_new( + child.clone(), + datafusion_physical_expr::Partitioning::RoundRobinBatch(2), + ) + .unwrap(), + ) as Arc; + + assert_passthrough_schema_contract(plan, child); + } + + #[test] + fn test_coalesce_partitions_exec_satisfies_passthrough_schema_contract() { + let child = new_region_scan(); + let plan = Arc::new(CoalescePartitionsExec::new(child.clone())) as Arc; + + assert_passthrough_schema_contract(plan, child); + } + + fn assert_passthrough_schema_contract( + plan: Arc, + child: Arc, + ) { + assert_eq!(plan.schema().as_ref(), child.schema().as_ref()); + + let passthrough = passthrough_child(plan.as_ref()).expect("wrapper should preserve schema"); + assert_eq!(passthrough.schema().as_ref(), child.schema().as_ref()); + } + + fn new_region_scan() -> Arc { + let schema = Arc::new(Schema::new(vec![ + ColumnSchema::new("value", ConcreteDataType::int32_datatype(), false), + ColumnSchema::new( + "ts", + ConcreteDataType::timestamp_nanosecond_datatype(), + false, + ), + ])); + let recordbatches = RecordBatches::try_new(schema.clone(), vec![]).unwrap(); + let stream = recordbatches.as_stream(); + + let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1)); + builder + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "value", + ConcreteDataType::int32_datatype(), + false, + ), + semantic_type: SemanticType::Field, + column_id: 1, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "ts", + ConcreteDataType::timestamp_nanosecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 2, + }); + + let scanner = Box::new(SinglePartitionScanner::new( + stream, + false, + Arc::new(builder.build().unwrap()), + None, + )); + Arc::new(RegionScanExec::new(scanner, ScanRequest::default(), None).unwrap()) } } diff --git a/src/query/src/part_sort.rs b/src/query/src/part_sort.rs index 19a114c8ce..933f0e47cf 100644 --- a/src/query/src/part_sort.rs +++ b/src/query/src/part_sort.rs @@ -56,7 +56,7 @@ use snafu::location; use store_api::region_engine::PartitionRange; use crate::error::Result; -use crate::window_sort::check_partition_range_monotonicity; +use crate::window_sort::{check_partition_range_monotonicity, project_partition_range_for_sort}; use crate::{array_iter_helper, downcast_ts_array}; /// Get the primary end of a `PartitionRange` based on sort direction. @@ -473,6 +473,7 @@ impl PartSortStream { snafu::location!() )? }; + let cur_range = project_partition_range_for_sort(cur_range, sort_column.data_type())?; downcast_ts_array!( sort_column.data_type() => (array_check_helper, sort_column, cur_range, min_max_idx), @@ -506,7 +507,10 @@ impl PartSortStream { snafu::location!() )?; } - let cur_range = self.partition_ranges[self.cur_part_idx]; + let cur_range = project_partition_range_for_sort( + self.partition_ranges[self.cur_part_idx], + sort_column.data_type(), + )?; let sort_column_iter = downcast_ts_array!( sort_column.data_type() => (array_iter_helper, sort_column), diff --git a/src/query/src/planner.rs b/src/query/src/planner.rs index 278058974a..b95803a52b 100644 --- a/src/query/src/planner.rs +++ b/src/query/src/planner.rs @@ -26,8 +26,8 @@ use common_telemetry::tracing; use datafusion::common::{DFSchema, plan_err}; use datafusion::execution::context::SessionState; use datafusion::sql::planner::PlannerContext; -use datafusion_common::ToDFSchema; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; +use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::expr::{Exists, InSubquery}; use datafusion_expr::{ Analyze, Explain, ExplainFormat, Expr as DfExpr, LogicalPlan, LogicalPlanBuilder, PlanType, @@ -451,6 +451,19 @@ impl DfLogicalPlanner { casted_placeholders.insert(ph.id.clone()); } + // Handle arrow_cast(Placeholder, 'type_string') generated by SQL rewriter + if let DfExpr::ScalarFunction(scalar_func) = e + && scalar_func.name() == "arrow_cast" + && scalar_func.args.len() == 2 + && let DfExpr::Placeholder(ph) = &scalar_func.args[0] + && let DfExpr::Literal(ScalarValue::Utf8(Some(type_str)), _) = + &scalar_func.args[1] + && let Ok(data_type) = type_str.parse::() + { + placeholder_types.insert(ph.id.clone(), Some(data_type)); + casted_placeholders.insert(ph.id.clone()); + } + // Handle bare (non-casted) placeholders if let DfExpr::Placeholder(ph) = e && !casted_placeholders.contains(&ph.id) @@ -869,4 +882,25 @@ mod tests { assert_eq!(types.get("$3"), Some(&Some(DataType::Int32))); assert_eq!(types.get("$4"), Some(&Some(DataType::Utf8))); } + + #[tokio::test] + async fn test_get_inferred_parameter_types_arrow_cast() { + let plan = parse_sql_to_plan("SELECT $1::INT64, $2::FLOAT64, $3::INT16, $4::INT32, $5::UINT8, $6::UINT16, $7::UINT32").await; + let types = DfLogicalPlanner::get_inferred_parameter_types(&plan).unwrap(); + + assert_eq!(types.get("$1"), Some(&Some(DataType::Int64))); + assert_eq!(types.get("$2"), Some(&Some(DataType::Float64))); + assert_eq!(types.get("$3"), Some(&Some(DataType::Int16))); + assert_eq!(types.get("$4"), Some(&Some(DataType::Int32))); + assert_eq!(types.get("$5"), Some(&Some(DataType::UInt8))); + assert_eq!(types.get("$6"), Some(&Some(DataType::UInt16))); + assert_eq!(types.get("$7"), Some(&Some(DataType::UInt32))); + + let plan = parse_sql_to_plan("SELECT $1::INT8, $2::FLOAT8, $3::INT2, $4::INT8").await; + let types = DfLogicalPlanner::get_inferred_parameter_types(&plan).unwrap(); + + assert_eq!(types.get("$1"), Some(&Some(DataType::Int64))); + assert_eq!(types.get("$2"), Some(&Some(DataType::Float64))); + assert_eq!(types.get("$3"), Some(&Some(DataType::Int16))); + } } diff --git a/src/query/src/query_engine/state.rs b/src/query/src/query_engine/state.rs index f696c8b53e..3646f468f4 100644 --- a/src/query/src/query_engine/state.rs +++ b/src/query/src/query_engine/state.rs @@ -59,6 +59,7 @@ use crate::dist_plan::{ }; use crate::metrics::{QUERY_MEMORY_POOL_REJECTED_TOTAL, QUERY_MEMORY_POOL_USAGE_BYTES}; use crate::optimizer::ExtensionAnalyzerRule; +use crate::optimizer::const_normalization::ConstNormalizationRule; use crate::optimizer::constant_term::MatchesConstantTermOptimizer; use crate::optimizer::count_nest_aggr::CountNestAggrRule; use crate::optimizer::count_wildcard::CountWildcardToTimeIndexRule; @@ -156,6 +157,7 @@ impl QueryEngineState { analyzer .rules .insert(0, Arc::new(CountWildcardToTimeIndexRule)); + analyzer.rules.push(Arc::new(ConstNormalizationRule)); // Add ApplyFunctionRewrites rule, // Note we cannot use `analyzer.add_function_rewrite` diff --git a/src/query/src/range_select/plan_rewrite.rs b/src/query/src/range_select/plan_rewrite.rs index 7cf8ef98b5..4d795565f0 100644 --- a/src/query/src/range_select/plan_rewrite.rs +++ b/src/query/src/range_select/plan_rewrite.rs @@ -35,6 +35,7 @@ use datafusion_expr::{ }; use datafusion_optimizer::simplify_expressions::ExprSimplifier; use datatypes::prelude::ConcreteDataType; +use datatypes::schema::TIME_INDEX_KEY; use promql_parser::util::parse_duration; use session::context::QueryContextRef; use snafu::{OptionExt, ResultExt, ensure}; @@ -376,7 +377,7 @@ impl RangePlanRewriter { } .fail(); }; - let (time_index, default_by) = self.get_index_by(input.schema()).await?; + let query_ctx = self.query_ctx.clone(); let mut range_rewriter = RangeExprRewriter { input_plan: &input, align: Duration::default(), @@ -384,13 +385,16 @@ impl RangePlanRewriter { by: vec![], range_fn: BTreeSet::new(), sub_aggr: aggr_plan, - query_ctx: &self.query_ctx, + query_ctx: &query_ctx, }; let new_expr = expr .iter() .map(|expr| expr.clone().rewrite(&mut range_rewriter).map(|x| x.data)) .collect::>>()?; - if range_rewriter.by.is_empty() { + let need_default_by = range_rewriter.by.is_empty(); + let (time_index, default_by) = + self.get_index_by(input.schema(), need_default_by).await?; + if need_default_by { range_rewriter.by = default_by; } let range_select = RangeSelect::try_new( @@ -481,25 +485,58 @@ impl RangePlanRewriter { } } - /// this function use to find the time_index column and row columns from input schema, - /// return `(time_index, [row_columns])` to the rewriter. - /// If the user does not explicitly use the `by` keyword to indicate time series, - /// `[row_columns]` will be use as default time series - async fn get_index_by(&mut self, schema: &Arc) -> Result<(Expr, Vec)> { + /// Finds the time index column and default row-key grouping from the input schema. + /// + /// Returns `(time_index, [row_columns])` to the rewriter. If the user omits `BY`, + /// `[row_columns]` is used as the default time-series grouping. + /// + /// For derived inputs such as subqueries, joins, or set operations, the source table + /// qualifier may no longer resolve back to a table provider. In that case we can still + /// recover the time index from column metadata, but we cannot safely reconstruct the + /// original row-key columns, so omitted `BY` must be rejected by the caller. + async fn get_index_by( + &mut self, + schema: &Arc, + need_default_by: bool, + ) -> Result<(Expr, Vec)> { #[allow(deprecated)] let mut time_index_expr = Expr::Wildcard { qualifier: None, options: Box::new(WildcardOptions::default()), }; let mut default_by = vec![]; + let metadata_time_index_expr = (0..schema.fields().len()).find_map(|i| { + let (qualifier, field) = schema.qualified_field(i); + if field.metadata().contains_key(TIME_INDEX_KEY) + && matches!(field.data_type(), DataType::Timestamp(_, _)) + { + Some(Expr::Column(Column::new( + qualifier.cloned(), + field.name().clone(), + ))) + } else { + None + } + }); for i in 0..schema.fields().len() { let (qualifier, _) = schema.qualified_field(i); if let Some(table_ref) = qualifier { - let table = self - .table_provider - .resolve_table(table_ref.clone()) - .await - .context(CatalogSnafu)? + let table_source = match self.table_provider.resolve_table(table_ref.clone()).await + { + Ok(table_source) => table_source, + Err(error) => { + // `TableNotExist` here usually means the qualifier now refers to a derived + // input instead of a base table. We can still salvage the time index from + // field metadata, but only when such metadata is present. + if matches!(&error, catalog::error::Error::TableNotExist { .. }) + && metadata_time_index_expr.is_some() + { + continue; + } + return Err(error).context(CatalogSnafu); + } + }; + let table = table_source .as_any() .downcast_ref::() .context(UnknownTableSnafu)? @@ -537,6 +574,22 @@ impl RangePlanRewriter { } } #[allow(deprecated)] + if matches!(time_index_expr, Expr::Wildcard { .. }) + && let Some(expr) = metadata_time_index_expr + { + common_telemetry::debug!( + "Range query falling back to time-index metadata for derived input schema: {}", + schema + ); + ensure!( + !need_default_by, + RangeQuerySnafu { + msg: "Cannot infer default BY columns from derived range query input" + } + ); + time_index_expr = expr; + } + #[allow(deprecated)] if matches!(time_index_expr, Expr::Wildcard { .. }) { TimeIndexNotFoundSnafu { table: schema.to_string(), @@ -614,6 +667,7 @@ mod test { use datatypes::schema::{ColumnSchema, Schema}; use session::context::QueryContext; use table::metadata::{TableInfoBuilder, TableMetaBuilder}; + use table::table::TableRef; use table::test_util::EmptyTable; use super::*; @@ -622,7 +676,45 @@ mod test { use crate::{QueryEngineFactory, QueryEngineRef}; async fn create_test_engine() -> QueryEngineRef { - let table_name = "test".to_string(); + create_test_engine_with_tables(&["test"], false).await + } + + async fn create_union_test_engine() -> QueryEngineRef { + create_test_engine_with_tables(&["test_0", "test_1"], true).await + } + + async fn create_test_engine_with_tables( + table_names: &[&str], + with_extra_timestamp: bool, + ) -> QueryEngineRef { + let catalog_list = MemoryCatalogManager::with_default_setup(); + for (i, table_name) in table_names.iter().enumerate() { + let table = create_test_table(table_name, with_extra_timestamp); + assert!( + catalog_list + .register_table_sync(RegisterTableRequest { + catalog: DEFAULT_CATALOG_NAME.to_string(), + schema: DEFAULT_SCHEMA_NAME.to_string(), + table_name: (*table_name).to_string(), + table_id: 1024 + i as u32, + table, + }) + .is_ok() + ); + } + QueryEngineFactory::new( + catalog_list, + None, + None, + None, + None, + false, + QueryOptions::default(), + ) + .query_engine() + } + + fn create_test_table(table_name: &str, with_extra_timestamp: bool) -> TableRef { let mut columns = vec![]; for i in 0..5 { columns.push(ColumnSchema::new( @@ -639,6 +731,13 @@ mod test { ) .with_time_index(true), ); + if with_extra_timestamp { + columns.push(ColumnSchema::new( + "timestamp_2".to_string(), + ConcreteDataType::timestamp_millisecond_datatype(), + true, + )); + } for i in 0..5 { columns.push(ColumnSchema::new( format!("field_{i}"), @@ -650,38 +749,20 @@ mod test { let table_meta = TableMetaBuilder::empty() .schema(schema) .primary_key_indices((0..5).collect()) - .value_indices((6..11).collect()) + .value_indices(if with_extra_timestamp { + (6..12).collect() + } else { + (6..11).collect() + }) .next_column_id(1024) .build() .unwrap(); let table_info = TableInfoBuilder::default() - .name(&table_name) + .name(table_name) .meta(table_meta) .build() .unwrap(); - let table = EmptyTable::from_table_info(&table_info); - let catalog_list = MemoryCatalogManager::with_default_setup(); - assert!( - catalog_list - .register_table_sync(RegisterTableRequest { - catalog: DEFAULT_CATALOG_NAME.to_string(), - schema: DEFAULT_SCHEMA_NAME.to_string(), - table_name, - table_id: 1024, - table, - }) - .is_ok() - ); - QueryEngineFactory::new( - catalog_list, - None, - None, - None, - None, - false, - QueryOptions::default(), - ) - .query_engine() + EmptyTable::from_table_info(&table_info) } async fn do_query(sql: &str) -> Result { @@ -690,6 +771,12 @@ mod test { engine.planner().plan(&stmt, QueryContext::arc()).await } + async fn do_union_query(sql: &str) -> Result { + let stmt = QueryLanguageParser::parse_sql(sql, &QueryContext::arc()).unwrap(); + let engine = create_union_test_engine().await; + engine.planner().plan(&stmt, QueryContext::arc()).await + } + async fn query_plan_compare(sql: &str, expected: String) { let plan = do_query(sql).await.unwrap(); assert_eq!(plan.display_indent_schema().to_string(), expected); @@ -765,6 +852,69 @@ mod test { query_plan_compare(query, expected).await; } + #[tokio::test] + async fn range_from_union_query() { + let queries = [ + r#"SELECT timestamp, tag_0, avg(field_0) RANGE '5m' + FROM ( + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_0 + UNION ALL + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_1 + ) + WHERE timestamp >= '1970-01-01 00:00:00' + ALIGN '1h' by (tag_0)"#, + r#"SELECT tmp.timestamp, tmp.tag_0, avg(tmp.field_0) RANGE '5m' + FROM ( + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_0 + UNION ALL + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_1 + ) AS tmp + WHERE tmp.timestamp >= '1970-01-01 00:00:00' + ALIGN '1h' by (tmp.tag_0)"#, + ]; + + for query in queries { + let plan = do_union_query(query) + .await + .unwrap() + .display_indent_schema() + .to_string(); + + assert!(plan.contains("RangeSelect")); + assert!(plan.contains("Union")); + assert!(plan.contains("time_index=timestamp")); + } + } + + #[tokio::test] + async fn range_from_derived_query_without_by_err() { + let queries = [ + r#"SELECT timestamp, tag_0, avg(field_0) RANGE '5m' + FROM ( + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_0 + UNION ALL + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_1 + ) + WHERE timestamp >= '1970-01-01 00:00:00' + ALIGN '1h'"#, + r#"SELECT tmp.timestamp, tmp.tag_0, avg(tmp.field_0) RANGE '5m' + FROM ( + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_0 + UNION ALL + SELECT timestamp, tag_0, field_0, timestamp_2 FROM test_1 + ) AS tmp + WHERE tmp.timestamp >= '1970-01-01 00:00:00' + ALIGN '1h'"#, + ]; + + for query in queries { + assert_eq!( + do_union_query(query).await.unwrap_err().to_string(), + "Range Query: Cannot infer default BY columns from derived range query input" + ); + } + } + #[tokio::test] async fn range_in_expr() { let query = r#"SELECT sin(avg(field_0 + field_1) RANGE '5m' + 1) FROM test ALIGN '1h' by (tag_0,tag_1);"#; diff --git a/src/query/src/window_sort.rs b/src/query/src/window_sort.rs index 83feee6f30..7267de3dab 100644 --- a/src/query/src/window_sort.rs +++ b/src/query/src/window_sort.rs @@ -30,6 +30,7 @@ use common_error::status_code::StatusCode; use common_recordbatch::{DfRecordBatch, DfSendableRecordBatchStream}; use common_telemetry::error; use common_time::Timestamp; +use common_time::timestamp::TimeUnit as TimestampUnit; use datafusion::execution::memory_pool::{MemoryConsumer, MemoryPool}; use datafusion::execution::{RecordBatchStream, TaskContext}; use datafusion::physical_plan::memory::MemoryStream; @@ -778,18 +779,11 @@ fn find_slice_from_range( sort_column: &SortColumn, range: &TimeRange, ) -> datafusion_common::Result<(usize, usize)> { - let ty = sort_column.values.data_type(); - let time_unit = if let DataType::Timestamp(unit, _) = ty { - unit - } else { - return Err(DataFusionError::Internal(format!( - "Unsupported sort column type: {}", - sort_column.values.data_type() - ))); - }; + let time_unit = sort_timestamp_unit(sort_column.values.data_type())?; let array = &sort_column.values; let opt = &sort_column.options.unwrap_or_default(); let descending = opt.descending; + let range = convert_time_range_for_sort(range, time_unit)?; let typed_sorted_range = [range.start, range.end] .iter() @@ -836,6 +830,72 @@ fn find_slice_from_range( Ok((start, end - start)) } +fn sort_timestamp_unit(data_type: &DataType) -> datafusion_common::Result { + if let DataType::Timestamp(unit, _) = data_type { + Ok(*unit) + } else { + Err(DataFusionError::Internal(format!( + "Unsupported sort column type: {data_type}" + ))) + } +} + +#[derive(Debug, Clone, Copy)] +enum RangeBoundKind { + InclusiveStart, + ExclusiveEnd, +} + +fn convert_time_range_for_sort( + range: &TimeRange, + time_unit: arrow_schema::TimeUnit, +) -> datafusion_common::Result { + let target_unit = time_unit.into(); + Ok(TimeRange::new( + convert_timestamp_range_bound(range.start, target_unit, RangeBoundKind::InclusiveStart)?, + convert_timestamp_range_bound(range.end, target_unit, RangeBoundKind::ExclusiveEnd)?, + )) +} + +fn convert_timestamp_range_bound( + timestamp: Timestamp, + target_unit: TimestampUnit, + bound_kind: RangeBoundKind, +) -> datafusion_common::Result { + let converted = match bound_kind { + RangeBoundKind::InclusiveStart => timestamp.convert_to(target_unit), + RangeBoundKind::ExclusiveEnd => timestamp.convert_to_ceil(target_unit), + }; + + converted.ok_or_else(|| { + DataFusionError::Internal(format!( + "Failed to convert timestamp from {:?} to {:?}", + timestamp.unit(), + target_unit + )) + }) +} + +pub(crate) fn project_partition_range_for_sort( + range: PartitionRange, + sort_data_type: &DataType, +) -> datafusion_common::Result { + let target_unit = sort_timestamp_unit(sort_data_type)?.into(); + Ok(PartitionRange { + start: convert_timestamp_range_bound( + range.start, + target_unit, + RangeBoundKind::InclusiveStart, + )?, + end: convert_timestamp_range_bound(range.end, target_unit, RangeBoundKind::ExclusiveEnd)?, + ..range + }) +} + +fn discrete_exclusive_end(timestamp: Timestamp) -> Timestamp { + Timestamp::new(timestamp.value() + 1, timestamp.unit()) +} + /// Get an iterator from a primitive array. /// /// Used with `downcast_ts_array`. The returned iter is wrapped with `.enumerate()`. @@ -905,7 +965,7 @@ impl SucRun { let end = self .first_val .max(self.last_val) - .map(|i| Timestamp::new(i.value() + 1, i.unit())); + .map(discrete_exclusive_end); start.zip(end).map(|(s, e)| TimeRange::new(s, e)) } } @@ -1498,6 +1558,49 @@ mod test { } } + #[test] + fn test_project_partition_range_for_sort_uses_ceil_on_exclusive_end() { + let range = PartitionRange { + start: Timestamp::new_nanosecond(1_000_000), + end: Timestamp::new_nanosecond(1_000_001), + num_rows: 1, + identifier: 0, + }; + + let projected = project_partition_range_for_sort( + range, + &DataType::Timestamp(TimeUnit::Millisecond, None), + ) + .unwrap(); + + assert_eq!(Timestamp::new_millisecond(1), projected.start); + assert_eq!(Timestamp::new_millisecond(2), projected.end); + } + + #[test] + fn test_find_slice_from_range_preserves_last_row_after_precision_drop() { + let sort_column = SortColumn { + values: Arc::new(TimestampMillisecondArray::from_iter_values([1])) as ArrayRef, + options: Some(SortOptions::default()), + }; + let range = TimeRange::new( + Timestamp::new_nanosecond(1_000_000), + Timestamp::new_nanosecond(1_000_001), + ); + + assert_eq!((0, 1), find_slice_from_range(&sort_column, &range).unwrap()); + } + + #[test] + fn test_discrete_exclusive_end_creates_half_open_upper_bound() { + let timestamp = Timestamp::new_millisecond(42); + + assert_eq!( + Timestamp::new_millisecond(43), + discrete_exclusive_end(timestamp) + ); + } + #[allow(clippy::type_complexity)] fn run_compute_working_ranges_test( testcases: Vec<( diff --git a/src/servers/src/pending_rows_batcher.rs b/src/servers/src/pending_rows_batcher.rs index 4cd8331636..307311a4e2 100644 --- a/src/servers/src/pending_rows_batcher.rs +++ b/src/servers/src/pending_rows_batcher.rs @@ -228,6 +228,7 @@ struct PendingBatch { tables: HashMap, created_at: Instant, total_row_count: usize, + db_string: String, ctx: QueryContextRef, waiters: Vec, } @@ -240,6 +241,7 @@ struct FlushWaiter { struct FlushBatch { table_batches: Vec, total_row_count: usize, + db_string: String, ctx: QueryContextRef, waiters: Vec, } @@ -821,10 +823,12 @@ impl Drop for PendingRowsBatcher { impl PendingBatch { fn new(ctx: QueryContextRef) -> Self { + let db_string = ctx.get_db_string(); Self { tables: HashMap::new(), created_at: Instant::now(), total_row_count: 0, + db_string, ctx, waiters: Vec::new(), } @@ -995,6 +999,7 @@ fn drain_batch(batch: &mut Option) -> Option { Some(FlushBatch { table_batches, total_row_count, + db_string: batch.db_string, ctx: batch.ctx, waiters, }) @@ -1169,15 +1174,19 @@ fn strip_partition_columns_from_batch(batch: RecordBatch) -> Result async fn flush_region_writes_concurrently( node_manager: &(impl PhysicalFlushNodeRequester + ?Sized), writes: Vec, -) -> Result<()> { +) -> Result { + let mut affected_rows = 0; if !should_dispatch_concurrently(writes.len()) { for write in writes { let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED .with_label_values(&["flush_write_region"]) .start_timer(); - node_manager.handle(&write.datanode, write.request).await?; + affected_rows += node_manager + .handle(&write.datanode, write.request) + .await? + .affected_rows; } - return Ok(()); + return Ok(affected_rows); } let write_futures = writes.into_iter().map(|write| async move { @@ -1185,13 +1194,16 @@ async fn flush_region_writes_concurrently( .with_label_values(&["flush_write_region"]) .start_timer(); - node_manager.handle(&write.datanode, write.request).await?; - Ok::<_, Error>(()) + let response = node_manager.handle(&write.datanode, write.request).await?; + Ok::<_, Error>(response.affected_rows) }); // todo(hl): should be bounded. - futures::future::try_join_all(write_futures).await?; - Ok(()) + let affected_rows = futures::future::try_join_all(write_futures) + .await? + .into_iter() + .sum(); + Ok(affected_rows) } async fn flush_batch( @@ -1203,6 +1215,7 @@ async fn flush_batch( let FlushBatch { table_batches, total_row_count, + db_string, ctx, waiters, } = flush; @@ -1233,9 +1246,12 @@ async fn flush_batch( if result.is_err() { FLUSH_FAILURES.inc(); FLUSH_DROPPED_ROWS.inc_by(total_row_count as u64); - } else { + } else if let Ok(affected_rows) = &result { FLUSH_TOTAL.inc(); FLUSH_ROWS.observe(total_row_count as f64); + operator::metrics::DIST_INGEST_ROW_COUNT + .with_label_values(&[db_string.as_str()]) + .inc_by(*affected_rows as u64); } debug!( @@ -1243,7 +1259,7 @@ async fn flush_batch( total_row_count, elapsed ); - notify_waiters(waiters, result); + notify_waiters(waiters, result.map(|_| ())); } /// Flushes a batch of logical table rows by transforming them into the physical table format @@ -1263,7 +1279,7 @@ pub async fn flush_batch_physical( partition_manager: &(impl PhysicalFlushPartitionProvider + ?Sized), node_manager: &(impl PhysicalFlushNodeRequester + ?Sized), catalog_manager: &(impl PhysicalFlushCatalogProvider + ?Sized), -) -> Result<()> { +) -> Result { // 1. Resolve the physical table and get column ID mapping let physical_table = { let _timer = PENDING_ROWS_BATCH_FLUSH_STAGE_ELAPSED @@ -1856,6 +1872,7 @@ mod tests { )]), created_at: Instant::now(), total_row_count: 1, + db_string: ctx.get_db_string(), ctx: ctx.clone(), waiters: vec![FlushWaiter { response_tx, @@ -1868,6 +1885,7 @@ mod tests { assert!(batch.is_none()); assert_eq!(1, flush.total_row_count); assert_eq!(1, flush.table_batches.len()); + assert_eq!(ctx.get_db_string(), flush.db_string); assert_eq!(ctx.current_catalog(), flush.ctx.current_catalog()); } @@ -2480,6 +2498,51 @@ mod tests { assert_eq!(1, node.writes.load(Ordering::SeqCst)); } + #[derive(Default)] + struct AffectedRowsFlushNodeRequester { + affected_rows: usize, + } + + #[async_trait] + impl PhysicalFlushNodeRequester for AffectedRowsFlushNodeRequester { + async fn handle( + &self, + _peer: &Peer, + _request: RegionRequest, + ) -> error::Result { + Ok(RegionResponse::new(self.affected_rows)) + } + } + + #[tokio::test] + async fn test_flush_batch_physical_returns_actual_affected_rows() { + let table_batches = vec![TableBatch { + table_name: "t1".to_string(), + table_id: 11, + batches: vec![mock_tag_batch("tag1", "host-1", 1000, 1.0)], + row_count: 1, + }]; + let ctx = session::context::QueryContext::arc(); + + let affected_rows = flush_batch_physical( + &table_batches, + "phy", + &ctx, + &MockFlushPartitionProvider { + partition_rule_calls: Arc::new(AtomicUsize::new(0)), + region_leader_calls: Arc::new(AtomicUsize::new(0)), + }, + &AffectedRowsFlushNodeRequester { affected_rows: 7 }, + &MockFlushCatalogProvider { + table: Some(mock_physical_table_metadata(1024)), + }, + ) + .await + .unwrap(); + + assert_eq!(7, affected_rows); + } + #[tokio::test] async fn test_flush_batch_physical_stops_before_partition_and_node_when_table_missing() { let table_batches = vec![TableBatch { diff --git a/src/servers/src/postgres/types.rs b/src/servers/src/postgres/types.rs index 203e477c6f..ec1b084d35 100644 --- a/src/servers/src/postgres/types.rs +++ b/src/servers/src/postgres/types.rs @@ -391,8 +391,8 @@ pub(super) fn parameters_to_scalar_values( return Err(invalid_parameter_error( "unknown_parameter_type", Some(format!( - "Cannot get parameter type information for parameter {}", - idx + "Cannot get type for parameter {}, try to provide a type using ${}::", + idx, idx )), )); }; diff --git a/tests-integration/tests/sql.rs b/tests-integration/tests/sql.rs index aa0062e4fd..662ccde097 100644 --- a/tests-integration/tests/sql.rs +++ b/tests-integration/tests/sql.rs @@ -83,6 +83,7 @@ macro_rules! sql_tests { test_postgres_intervalstyle, test_postgres_parameter_inference, test_postgres_uint64_parameter, + test_postgres_explain_bind_parameter, test_postgres_array_types, test_mysql_prepare_stmt_insert_timestamp, test_declare_fetch_close_cursor, @@ -1352,6 +1353,65 @@ pub async fn test_postgres_uint64_parameter(store_type: StorageType) { guard.remove_all().await; } +pub async fn test_postgres_explain_bind_parameter(store_type: StorageType) { + // Regression test for #8029: EXPLAIN / EXPLAIN ANALYZE must accept bind + // parameters over the Postgres extended query protocol. + let (mut guard, fe_pg_server) = + setup_pg_server(store_type, "test_postgres_explain_bind_parameter").await; + let addr = fe_pg_server.bind_addr().unwrap().to_string(); + + let (client, connection) = tokio_postgres::connect(&format!("postgres://{addr}/public"), NoTls) + .await + .unwrap(); + + let (tx, rx) = tokio::sync::oneshot::channel(); + tokio::spawn(async move { + connection.await.unwrap(); + tx.send(()).unwrap(); + }); + + let _ = client + .simple_query( + "create table t (k varchar(36) not null, ts timestamp(3) not null, time index(ts))", + ) + .await + .unwrap(); + let _ = client + .simple_query("insert into t (k, ts) values ('a', 1), ('b', 2), ('c', 3)") + .await + .unwrap(); + + // Sanity check: the underlying SELECT with a bind parameter works. + let rows = client + .query("SELECT k FROM t WHERE k = $1", &[&"a"]) + .await + .unwrap(); + assert_eq!(1, rows.len()); + + // EXPLAIN with a bind parameter must succeed. + let rows = client + .query("EXPLAIN SELECT k FROM t WHERE k = $1", &[&"a"]) + .await + .unwrap(); + assert!(!rows.is_empty(), "EXPLAIN should produce at least one row"); + + // EXPLAIN ANALYZE with a bind parameter must also succeed. + let rows = client + .query("EXPLAIN ANALYZE SELECT k FROM t WHERE k = $1", &[&"a"]) + .await + .unwrap(); + assert!( + !rows.is_empty(), + "EXPLAIN ANALYZE should produce at least one row" + ); + + drop(client); + rx.await.unwrap(); + + let _ = fe_pg_server.shutdown().await; + guard.remove_all().await; +} + pub async fn test_mysql_async_timestamp(store_type: StorageType) { use mysql_async::prelude::*; use time::PrimitiveDateTime; diff --git a/tests/cases/distributed/optimizer/windowed_sort_advance.result b/tests/cases/distributed/optimizer/windowed_sort_advance.result new file mode 100644 index 0000000000..e3c92a0a08 --- /dev/null +++ b/tests/cases/distributed/optimizer/windowed_sort_advance.result @@ -0,0 +1,118 @@ +create table `a` (`value` double, `status` bigint, ts timestamp(9) time index); + +Affected Rows: 0 + +INSERT INTO `a` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.000000000+08:00'), +(46.84, 200, '2026-03-12T08:00:15.000000000+08:00'), +(46.85, 200, '2026-03-12T08:00:25.000000000+08:00'), +(46.86, 200, '2026-03-12T08:00:35.000000000+08:00'), +(46.88, 200, '2026-03-12T08:00:45.000000000+08:00'), +(46.89, 200, '2026-03-12T08:00:55.000000000+08:00'), +(46.91, 200, '2026-03-12T08:01:05.000000000+08:00'), +(46.90, 200, '2026-03-12T08:01:15.000000000+08:00'), +(46.87, 200, '2026-03-12T08:01:25.000000000+08:00'), +(46.85, 200, '2026-03-12T08:01:35.000000000+08:00'); + +Affected Rows: 10 + +select ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++---------------------+--------+-------+ +| ts | status | value | ++---------------------+--------+-------+ +| 2026-03-12T00:00:05 | 200 | 46.82 | +| 2026-03-12T00:00:15 | 200 | 46.84 | +| 2026-03-12T00:00:25 | 200 | 46.85 | +| 2026-03-12T00:00:35 | 200 | 46.86 | +| 2026-03-12T00:00:45 | 200 | 46.88 | +| 2026-03-12T00:00:55 | 200 | 46.89 | +| 2026-03-12T00:01:05 | 200 | 46.91 | +| 2026-03-12T00:01:15 | 200 | 46.9 | +| 2026-03-12T00:01:25 | 200 | 46.87 | +| 2026-03-12T00:01:35 | 200 | 46.85 | ++---------------------+--------+-------+ + +select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++---------------------+--------+-------+ +| ts | status | value | ++---------------------+--------+-------+ +| 2026-03-12T00:00:05 | 200 | 46.82 | +| 2026-03-12T00:00:15 | 200 | 46.84 | +| 2026-03-12T00:00:25 | 200 | 46.85 | +| 2026-03-12T00:00:35 | 200 | 46.86 | +| 2026-03-12T00:00:45 | 200 | 46.88 | +| 2026-03-12T00:00:55 | 200 | 46.89 | +| 2026-03-12T00:01:05 | 200 | 46.91 | +| 2026-03-12T00:01:15 | 200 | 46.9 | +| 2026-03-12T00:01:25 | 200 | 46.87 | +| 2026-03-12T00:01:35 | 200 | 46.85 | ++---------------------+--------+-------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED +|_|_|_WindowedSortExec: expr=ts@0 ASC NULLS LAST num_ranges=REDACTED REDACTED +|_|_|_ProjectionExec: expr=[ts@2 as ts, status@1 as status, value@0 as value] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 10_| ++-+-+-+ + +select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++---------------------+--------+-------+ +| ts | status | value | ++---------------------+--------+-------+ +| 2026-03-12T00:00:05 | 200 | 46.82 | +| 2026-03-12T00:00:15 | 200 | 46.84 | +| 2026-03-12T00:00:25 | 200 | 46.85 | +| 2026-03-12T00:00:35 | 200 | 46.86 | +| 2026-03-12T00:00:45 | 200 | 46.88 | +| 2026-03-12T00:00:55 | 200 | 46.89 | +| 2026-03-12T00:01:05 | 200 | 46.91 | +| 2026-03-12T00:01:15 | 200 | 46.9 | +| 2026-03-12T00:01:25 | 200 | 46.87 | +| 2026-03-12T00:01:35 | 200 | 46.85 | ++---------------------+--------+-------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_ProjectionExec: expr=[to_timestamp_millis(a.ts)@0 as ts, status@1 as status, value@2 as value] REDACTED +|_|_|_SortPreservingMergeExec: [to_timestamp_millis(a.ts)@0 ASC NULLS LAST] REDACTED +|_|_|_WindowedSortExec: expr=to_timestamp_millis(a.ts)@0 ASC NULLS LAST num_ranges=REDACTED REDACTED +|_|_|_ProjectionExec: expr=[to_timestamp_millis(ts@2) as to_timestamp_millis(a.ts), status@1 as status, value@0 as value] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 10_| ++-+-+-+ + +DROP TABLE `a`; + +Affected Rows: 0 + diff --git a/tests/cases/distributed/optimizer/windowed_sort_advance.sql b/tests/cases/distributed/optimizer/windowed_sort_advance.sql new file mode 100644 index 0000000000..fa73b97933 --- /dev/null +++ b/tests/cases/distributed/optimizer/windowed_sort_advance.sql @@ -0,0 +1,37 @@ +create table `a` (`value` double, `status` bigint, ts timestamp(9) time index); + +INSERT INTO `a` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.000000000+08:00'), +(46.84, 200, '2026-03-12T08:00:15.000000000+08:00'), +(46.85, 200, '2026-03-12T08:00:25.000000000+08:00'), +(46.86, 200, '2026-03-12T08:00:35.000000000+08:00'), +(46.88, 200, '2026-03-12T08:00:45.000000000+08:00'), +(46.89, 200, '2026-03-12T08:00:55.000000000+08:00'), +(46.91, 200, '2026-03-12T08:01:05.000000000+08:00'), +(46.90, 200, '2026-03-12T08:01:15.000000000+08:00'), +(46.87, 200, '2026-03-12T08:01:25.000000000+08:00'), +(46.85, 200, '2026-03-12T08:01:35.000000000+08:00'); + +select ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +DROP TABLE `a`; diff --git a/tests/cases/distributed/optimizer/windowed_sort_nanos.result b/tests/cases/distributed/optimizer/windowed_sort_nanos.result new file mode 100644 index 0000000000..2310ec8f28 --- /dev/null +++ b/tests/cases/distributed/optimizer/windowed_sort_nanos.result @@ -0,0 +1,79 @@ +create table `a_ms` (`value` double, `status` bigint, ts timestamp(3) time index); + +Affected Rows: 0 + +INSERT INTO `a_ms` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.123+08:00'), +(46.84, 200, '2026-03-12T08:00:15.234+08:00'), +(46.85, 200, '2026-03-12T08:00:25.345+08:00'), +(46.86, 200, '2026-03-12T08:00:35.456+08:00'), +(46.88, 200, '2026-03-12T08:00:45.567+08:00'), +(46.89, 200, '2026-03-12T08:00:55.678+08:00'), +(46.91, 200, '2026-03-12T08:01:05.789+08:00'), +(46.90, 200, '2026-03-12T08:01:15.890+08:00'), +(46.87, 200, '2026-03-12T08:01:25.901+08:00'), +(46.85, 200, '2026-03-12T08:01:35.999+08:00'); + +Affected Rows: 10 + +select ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + ++-------------------------+--------+-------+ +| ts | status | value | ++-------------------------+--------+-------+ +| 2026-03-12T00:00:05.123 | 200 | 46.82 | +| 2026-03-12T00:00:15.234 | 200 | 46.84 | +| 2026-03-12T00:00:25.345 | 200 | 46.85 | +| 2026-03-12T00:00:35.456 | 200 | 46.86 | +| 2026-03-12T00:00:45.567 | 200 | 46.88 | +| 2026-03-12T00:00:55.678 | 200 | 46.89 | +| 2026-03-12T00:01:05.789 | 200 | 46.91 | +| 2026-03-12T00:01:15.890 | 200 | 46.9 | +| 2026-03-12T00:01:25.901 | 200 | 46.87 | +| 2026-03-12T00:01:35.999 | 200 | 46.85 | ++-------------------------+--------+-------+ + +select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + ++-------------------------+--------+-------+ +| ts | status | value | ++-------------------------+--------+-------+ +| 2026-03-12T00:00:05.123 | 200 | 46.82 | +| 2026-03-12T00:00:15.234 | 200 | 46.84 | +| 2026-03-12T00:00:25.345 | 200 | 46.85 | +| 2026-03-12T00:00:35.456 | 200 | 46.86 | +| 2026-03-12T00:00:45.567 | 200 | 46.88 | +| 2026-03-12T00:00:55.678 | 200 | 46.89 | +| 2026-03-12T00:01:05.789 | 200 | 46.91 | +| 2026-03-12T00:01:15.890 | 200 | 46.9 | +| 2026-03-12T00:01:25.901 | 200 | 46.87 | +| 2026-03-12T00:01:35.999 | 200 | 46.85 | ++-------------------------+--------+-------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_ProjectionExec: expr=[to_timestamp_nanos(a_ms.ts)@0 as ts, status@1 as status, value@2 as value] REDACTED +|_|_|_SortPreservingMergeExec: [to_timestamp_nanos(a_ms.ts)@0 ASC NULLS LAST] REDACTED +|_|_|_WindowedSortExec: expr=to_timestamp_nanos(a_ms.ts)@0 ASC NULLS LAST num_ranges=REDACTED REDACTED +|_|_|_ProjectionExec: expr=[to_timestamp_nanos(ts@2) as to_timestamp_nanos(a_ms.ts), status@1 as status, value@0 as value] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 10_| ++-+-+-+ + +DROP TABLE `a_ms`; + +Affected Rows: 0 + diff --git a/tests/cases/distributed/optimizer/windowed_sort_nanos.sql b/tests/cases/distributed/optimizer/windowed_sort_nanos.sql new file mode 100644 index 0000000000..6a6cf51c4b --- /dev/null +++ b/tests/cases/distributed/optimizer/windowed_sort_nanos.sql @@ -0,0 +1,27 @@ +create table `a_ms` (`value` double, `status` bigint, ts timestamp(3) time index); + +INSERT INTO `a_ms` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.123+08:00'), +(46.84, 200, '2026-03-12T08:00:15.234+08:00'), +(46.85, 200, '2026-03-12T08:00:25.345+08:00'), +(46.86, 200, '2026-03-12T08:00:35.456+08:00'), +(46.88, 200, '2026-03-12T08:00:45.567+08:00'), +(46.89, 200, '2026-03-12T08:00:55.678+08:00'), +(46.91, 200, '2026-03-12T08:01:05.789+08:00'), +(46.90, 200, '2026-03-12T08:01:15.890+08:00'), +(46.87, 200, '2026-03-12T08:01:25.901+08:00'), +(46.85, 200, '2026-03-12T08:01:35.999+08:00'); + +select ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + +select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + +DROP TABLE `a_ms`; diff --git a/tests/cases/standalone/common/alter/alter_table.result b/tests/cases/standalone/common/alter/alter_table.result index b1969bbaf7..5ba9fc0ed0 100644 --- a/tests/cases/standalone/common/alter/alter_table.result +++ b/tests/cases/standalone/common/alter/alter_table.result @@ -508,42 +508,42 @@ Affected Rows: 0 ALTER TABLE t1 ADD - COLUMN `at` STRING; + COLUMN `at` STRING PRIMARY KEY; Affected Rows: 0 ALTER TABLE t2 ADD - COLUMN at3 STRING; + COLUMN at3 STRING PRIMARY KEY; Affected Rows: 0 ALTER TABLE t2 ADD - COLUMN `at` STRING; + COLUMN `at` STRING PRIMARY KEY; Affected Rows: 0 ALTER TABLE t2 ADD - COLUMN at2 STRING; + COLUMN at2 STRING PRIMARY KEY; Affected Rows: 0 ALTER TABLE t2 ADD - COLUMN at4 UINT16; + COLUMN at4 STRING PRIMARY KEY; Affected Rows: 0 INSERT INTO t2 VALUES - ("loc_1", "loc_2", "loc_3", 2, 'job1', 0, 1); + ("loc_1", "loc_2", "loc_3", "loc_4", 'job1', 0, 1); Affected Rows: 1 @@ -552,11 +552,11 @@ SELECT FROM t2; -+-------+-------+-------+-----+------+---------------------+-----+ -| at | at2 | at3 | at4 | job | ts | val | -+-------+-------+-------+-----+------+---------------------+-----+ -| loc_1 | loc_2 | loc_3 | 2 | job1 | 1970-01-01T00:00:00 | 1.0 | -+-------+-------+-------+-----+------+---------------------+-----+ ++-------+-------+-------+-------+------+---------------------+-----+ +| at | at2 | at3 | at4 | job | ts | val | ++-------+-------+-------+-------+------+---------------------+-----+ +| loc_1 | loc_2 | loc_3 | loc_4 | job1 | 1970-01-01T00:00:00 | 1.0 | ++-------+-------+-------+-------+------+---------------------+-----+ DROP TABLE t1; diff --git a/tests/cases/standalone/common/alter/alter_table.sql b/tests/cases/standalone/common/alter/alter_table.sql index 6088ea3bd2..767aca0510 100644 --- a/tests/cases/standalone/common/alter/alter_table.sql +++ b/tests/cases/standalone/common/alter/alter_table.sql @@ -184,32 +184,32 @@ CREATE TABLE t2 ( ALTER TABLE t1 ADD - COLUMN `at` STRING; + COLUMN `at` STRING PRIMARY KEY; ALTER TABLE t2 ADD - COLUMN at3 STRING; + COLUMN at3 STRING PRIMARY KEY; ALTER TABLE t2 ADD - COLUMN `at` STRING; + COLUMN `at` STRING PRIMARY KEY; ALTER TABLE t2 ADD - COLUMN at2 STRING; + COLUMN at2 STRING PRIMARY KEY; ALTER TABLE t2 ADD - COLUMN at4 UINT16; + COLUMN at4 STRING PRIMARY KEY; INSERT INTO t2 VALUES - ("loc_1", "loc_2", "loc_3", 2, 'job1', 0, 1); + ("loc_1", "loc_2", "loc_3", "loc_4", 'job1', 0, 1); SELECT * diff --git a/tests/cases/standalone/common/create/create_with_options.result b/tests/cases/standalone/common/create/create_with_options.result index bb350a78aa..7c76ac6c63 100644 --- a/tests/cases/standalone/common/create/create_with_options.result +++ b/tests/cases/standalone/common/create/create_with_options.result @@ -81,6 +81,22 @@ drop table test_mito_options; Affected Rows: 0 +create table if not exists test_compaction_override_without_type( + host string, + ts timestamp, + memory double, + TIME INDEX (ts), + PRIMARY KEY(host) +) +engine=mito +with('compaction.override'='true'); + +Affected Rows: 0 + +drop table test_compaction_override_without_type; + +Affected Rows: 0 + create table if not exists invalid_compaction( host string, ts timestamp, diff --git a/tests/cases/standalone/common/create/create_with_options.sql b/tests/cases/standalone/common/create/create_with_options.sql index 13824f76aa..2ab8cee1b8 100644 --- a/tests/cases/standalone/common/create/create_with_options.sql +++ b/tests/cases/standalone/common/create/create_with_options.sql @@ -67,6 +67,18 @@ with( drop table test_mito_options; +create table if not exists test_compaction_override_without_type( + host string, + ts timestamp, + memory double, + TIME INDEX (ts), + PRIMARY KEY(host) +) +engine=mito +with('compaction.override'='true'); + +drop table test_compaction_override_without_type; + create table if not exists invalid_compaction( host string, ts timestamp, diff --git a/tests/cases/standalone/common/insert/logical_metric_table.result b/tests/cases/standalone/common/insert/logical_metric_table.result index 80e765ccd6..32085175ea 100644 --- a/tests/cases/standalone/common/insert/logical_metric_table.result +++ b/tests/cases/standalone/common/insert/logical_metric_table.result @@ -77,6 +77,40 @@ DROP TABLE phy; Affected Rows: 0 +CREATE TABLE phy_default (ts timestamp time index, val double default 42) engine=metric with ("physical_metric_table" = ""); + +Affected Rows: 0 + +CREATE TABLE t_default (ts timestamp time index, val double default 42, host string primary key) engine = metric with ("on_physical_table" = "phy_default"); + +Affected Rows: 0 + +INSERT INTO t_default (host, ts) VALUES ('host1', 0), ('host2', 1); + +Affected Rows: 2 + +SELECT host, ts, val FROM t_default ORDER BY host; + ++-------+-------------------------+------+ +| host | ts | val | ++-------+-------------------------+------+ +| host1 | 1970-01-01T00:00:00 | 42.0 | +| host2 | 1970-01-01T00:00:00.001 | 42.0 | ++-------+-------------------------+------+ + +-- SQLNESS REPLACE (region\s\d+\(\d+\,\s\d+\)) region +INSERT INTO t_default (host, val) VALUES ('host3', 3); + +Error: 1004(InvalidArguments), Invalid request for region, reason: missing required time index column ts + +DROP TABLE t_default; + +Affected Rows: 0 + +DROP TABLE phy_default; + +Affected Rows: 0 + CREATE TABLE phy ( ts timestamp time index, val double diff --git a/tests/cases/standalone/common/insert/logical_metric_table.sql b/tests/cases/standalone/common/insert/logical_metric_table.sql index 9899699c66..6a3bed401d 100644 --- a/tests/cases/standalone/common/insert/logical_metric_table.sql +++ b/tests/cases/standalone/common/insert/logical_metric_table.sql @@ -24,6 +24,21 @@ SELECT ts, val, __tsid, host, job FROM phy; DROP TABLE phy; +CREATE TABLE phy_default (ts timestamp time index, val double default 42) engine=metric with ("physical_metric_table" = ""); + +CREATE TABLE t_default (ts timestamp time index, val double default 42, host string primary key) engine = metric with ("on_physical_table" = "phy_default"); + +INSERT INTO t_default (host, ts) VALUES ('host1', 0), ('host2', 1); + +SELECT host, ts, val FROM t_default ORDER BY host; + +-- SQLNESS REPLACE (region\s\d+\(\d+\,\s\d+\)) region +INSERT INTO t_default (host, val) VALUES ('host3', 3); + +DROP TABLE t_default; + +DROP TABLE phy_default; + CREATE TABLE phy ( ts timestamp time index, val double diff --git a/tests/cases/standalone/common/range/error.result b/tests/cases/standalone/common/range/error.result index e3f12646e7..ff0795f450 100644 --- a/tests/cases/standalone/common/range/error.result +++ b/tests/cases/standalone/common/range/error.result @@ -60,6 +60,12 @@ SELECT min(val) RANGE '5s' FILL PREV FROM host; Error: 2000(InvalidSyntax), Invalid SQL syntax: sql parser error: ALIGN argument cannot be omitted in the range select query +SELECT tmp.ts, tmp.host, min(tmp.val) RANGE '5s' +FROM (SELECT ts, host, val FROM host) AS tmp +ALIGN '5s'; + +Error: 2000(InvalidSyntax), Range Query: Cannot infer default BY columns from derived range query input + -- 2.3 type mismatch SELECT covar(ceil(val), floor(val)) RANGE '20s' FROM host ALIGN '10s'; diff --git a/tests/cases/standalone/common/range/error.sql b/tests/cases/standalone/common/range/error.sql index 3659be1c79..d43b67ea4d 100644 --- a/tests/cases/standalone/common/range/error.sql +++ b/tests/cases/standalone/common/range/error.sql @@ -42,6 +42,10 @@ SELECT min(val) RANGE '5s' FROM host; SELECT min(val) RANGE '5s' FILL PREV FROM host; +SELECT tmp.ts, tmp.host, min(tmp.val) RANGE '5s' +FROM (SELECT ts, host, val FROM host) AS tmp +ALIGN '5s'; + -- 2.3 type mismatch SELECT covar(ceil(val), floor(val)) RANGE '20s' FROM host ALIGN '10s'; diff --git a/tests/cases/standalone/common/range/nest.result b/tests/cases/standalone/common/range/nest.result index 0730a0b997..7bc3e39a7b 100644 --- a/tests/cases/standalone/common/range/nest.result +++ b/tests/cases/standalone/common/range/nest.result @@ -45,6 +45,82 @@ SELECT ts, b, min(c) RANGE '5s' FROM (SELECT ts, host AS b, val AS c FROM host W | 1970-01-01T00:00:20 | host1 | 2 | +---------------------+-------+-----------------+ +CREATE TABLE host_union_0 ( + ts timestamp(3) time index, + host STRING PRIMARY KEY, + val BIGINT, + ts2 timestamp(3), +); + +Affected Rows: 0 + +CREATE TABLE host_union_1 ( + ts timestamp(3) time index, + host STRING PRIMARY KEY, + val BIGINT, + ts2 timestamp(3), +); + +Affected Rows: 0 + +INSERT INTO TABLE host_union_0 VALUES + (0, 'host1', 3, 0), + (5000, 'host1', 2, 5000), + (10000, 'host1', 1, 10000); + +Affected Rows: 3 + +INSERT INTO TABLE host_union_1 VALUES + (0, 'host1', 6, 0), + (5000, 'host1', 5, 5000), + (10000, 'host1', 4, 10000); + +Affected Rows: 3 + +SELECT ts, host, min(val ORDER BY ts ASC) RANGE '5s' +FROM ( + SELECT ts, host, val, ts2 FROM host_union_0 + UNION ALL + SELECT ts, host, val, ts2 FROM host_union_1 +) +WHERE ts >= '1970-01-01 00:00:00' +ALIGN '5s' BY (host) +ORDER BY host, ts; + ++---------------------+-------+------------------------------------------------+ +| ts | host | min(val) ORDER BY [ts ASC NULLS LAST] RANGE 5s | ++---------------------+-------+------------------------------------------------+ +| 1970-01-01T00:00:00 | host1 | 3 | +| 1970-01-01T00:00:05 | host1 | 2 | +| 1970-01-01T00:00:10 | host1 | 1 | ++---------------------+-------+------------------------------------------------+ + +SELECT tmp.ts, tmp.host, min(tmp.val ORDER BY tmp.ts ASC) RANGE '5s' +FROM ( + SELECT ts, host, val, ts2 FROM host_union_0 + UNION ALL + SELECT ts, host, val, ts2 FROM host_union_1 +) AS tmp +WHERE tmp.ts >= '1970-01-01 00:00:00' +ALIGN '5s' BY (tmp.host) +ORDER BY tmp.host, tmp.ts; + ++---------------------+-------+--------------------------------------------------------+ +| ts | host | min(tmp.val) ORDER BY [tmp.ts ASC NULLS LAST] RANGE 5s | ++---------------------+-------+--------------------------------------------------------+ +| 1970-01-01T00:00:00 | host1 | 3 | +| 1970-01-01T00:00:05 | host1 | 2 | +| 1970-01-01T00:00:10 | host1 | 1 | ++---------------------+-------+--------------------------------------------------------+ + +DROP TABLE host_union_0; + +Affected Rows: 0 + +DROP TABLE host_union_1; + +Affected Rows: 0 + -- Test EXPLAIN and ANALYZE -- SQLNESS REPLACE (-+) - -- SQLNESS REPLACE (\s\s+) _ diff --git a/tests/cases/standalone/common/range/nest.sql b/tests/cases/standalone/common/range/nest.sql index 4ee447cfbd..d92f9f1345 100644 --- a/tests/cases/standalone/common/range/nest.sql +++ b/tests/cases/standalone/common/range/nest.sql @@ -22,6 +22,54 @@ SELECT ts, host, foo FROM (SELECT ts, host, min(val) RANGE '5s' AS foo FROM host SELECT ts, b, min(c) RANGE '5s' FROM (SELECT ts, host AS b, val AS c FROM host WHERE host = 'host1') ALIGN '5s' BY (b) ORDER BY b, ts; +CREATE TABLE host_union_0 ( + ts timestamp(3) time index, + host STRING PRIMARY KEY, + val BIGINT, + ts2 timestamp(3), +); + +CREATE TABLE host_union_1 ( + ts timestamp(3) time index, + host STRING PRIMARY KEY, + val BIGINT, + ts2 timestamp(3), +); + +INSERT INTO TABLE host_union_0 VALUES + (0, 'host1', 3, 0), + (5000, 'host1', 2, 5000), + (10000, 'host1', 1, 10000); + +INSERT INTO TABLE host_union_1 VALUES + (0, 'host1', 6, 0), + (5000, 'host1', 5, 5000), + (10000, 'host1', 4, 10000); + +SELECT ts, host, min(val ORDER BY ts ASC) RANGE '5s' +FROM ( + SELECT ts, host, val, ts2 FROM host_union_0 + UNION ALL + SELECT ts, host, val, ts2 FROM host_union_1 +) +WHERE ts >= '1970-01-01 00:00:00' +ALIGN '5s' BY (host) +ORDER BY host, ts; + +SELECT tmp.ts, tmp.host, min(tmp.val ORDER BY tmp.ts ASC) RANGE '5s' +FROM ( + SELECT ts, host, val, ts2 FROM host_union_0 + UNION ALL + SELECT ts, host, val, ts2 FROM host_union_1 +) AS tmp +WHERE tmp.ts >= '1970-01-01 00:00:00' +ALIGN '5s' BY (tmp.host) +ORDER BY tmp.host, tmp.ts; + +DROP TABLE host_union_0; + +DROP TABLE host_union_1; + -- Test EXPLAIN and ANALYZE diff --git a/tests/cases/standalone/common/tql-explain-analyze/explain.result b/tests/cases/standalone/common/tql-explain-analyze/explain.result index 1e4cf18b40..29f3dbf1b3 100644 --- a/tests/cases/standalone/common/tql-explain-analyze/explain.result +++ b/tests/cases/standalone/common/tql-explain-analyze/explain.result @@ -74,6 +74,7 @@ TQL EXPLAIN ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp -- SQLNESS REPLACE (elapsed_compute.*) REDACTED -- SQLNESS REPLACE (peers.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED TQL EXPLAIN VERBOSE (0, 10, '5s') test; +-+-+ @@ -90,6 +91,7 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test; | logical_plan after TranscribeAtatRule_| SAME TEXT AS ABOVE_| | logical_plan after resolve_grouping_function_| SAME TEXT AS ABOVE_| | logical_plan after type_coercion_| SAME TEXT AS ABOVE_| +| logical_plan after ConstNormalizationRule_| SAME TEXT AS ABOVE_| | logical_plan after DistPlannerAnalyzer_| Projection: test.i, test.j, test.k_| |_|_MergeScan [is_placeholder=false, remote_input=[_| |_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| @@ -216,6 +218,7 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test; -- SQLNESS REPLACE (elapsed_compute.*) REDACTED -- SQLNESS REPLACE (peers.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED TQL EXPLAIN VERBOSE (0, 10, '5s') test AS series; +-+-+ @@ -233,6 +236,7 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test AS series; | logical_plan after TranscribeAtatRule_| SAME TEXT AS ABOVE_| | logical_plan after resolve_grouping_function_| SAME TEXT AS ABOVE_| | logical_plan after type_coercion_| SAME TEXT AS ABOVE_| +| logical_plan after ConstNormalizationRule_| SAME TEXT AS ABOVE_| | logical_plan after DistPlannerAnalyzer_| Projection: series, test.k, test.j_| |_|_MergeScan [is_placeholder=false, remote_input=[_| |_| Projection: test.i AS series, test.k, test.j_| @@ -356,6 +360,391 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test AS series; |_|_| +-+-+ +CREATE TABLE test_nano(i DOUBLE, j TIMESTAMP(9) TIME INDEX, k STRING PRIMARY KEY); + +Affected Rows: 0 + +INSERT INTO test_nano VALUES (1, 1000000, "a"), (1, 1000000, "b"), (2, 2000000, "a"); + +Affected Rows: 3 + +-- explain at 0s, 5s and 10s for a nanosecond time index. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED +TQL EXPLAIN (0, 10, '5s') test_nano; + ++---------------+-----------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-----------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] | +| | PromSeriesDivide: tags=["k"] | +| | Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST | +| | Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j | +| | Projection: test_nano.i, test_nano.j, test_nano.k | +| | Filter: __common_expr_3 >= TimestampMillisecond(-299999, None) AND __common_expr_3 <= TimestampMillisecond(10000, None) | +| | Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_3, test_nano.i, test_nano.j, test_nano.k | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | TableScan: test_nano | +| | ]] | +| physical_plan | PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] | +| | PromSeriesDivideExec: tags=["k"] | +| | SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true] | +| | RepartitionExec: REDACTED +| | ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j] | +| | FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, projection=[i@1, j@2, k@3] | +| | ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k] | +| | MergeScanExec: REDACTED +| | | ++---------------+-----------------------------------------------------------------------------------------------------------------------------------+ + +-- explain verbose at 0s, 5s and 10s for a nanosecond time index. +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (elapsed_compute.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED +TQL EXPLAIN VERBOSE (0, 10, '5s') test_nano; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| initial_logical_plan_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Filter: test_nano.j >= TimestampMillisecond(-299999, None) AND test_nano.j <= TimestampMillisecond(10000, None)_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_TableScan: test_nano_| +| logical_plan after apply_function_rewrites_| SAME TEXT AS ABOVE_| +| logical_plan after count_wildcard_to_time_index_rule_| SAME TEXT AS ABOVE_| +| logical_plan after StringNormalizationRule_| SAME TEXT AS ABOVE_| +| logical_plan after TranscribeAtatRule_| SAME TEXT AS ABOVE_| +| logical_plan after resolve_grouping_function_| SAME TEXT AS ABOVE_| +| logical_plan after type_coercion_| SAME TEXT AS ABOVE_| +| logical_plan after ConstNormalizationRule_| SAME TEXT AS ABOVE_| +| logical_plan after DistPlannerAnalyzer_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Filter: test_nano.j >= TimestampMillisecond(-299999, None) AND test_nano.j <= TimestampMillisecond(10000, None)_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after FixStateUdafOrderingAnalyzer_| SAME TEXT AS ABOVE_| +| analyzed_logical_plan_| SAME TEXT AS ABOVE_| +| logical_plan after rewrite_set_comparison_| SAME TEXT AS ABOVE_| +| logical_plan after optimize_unions_| SAME TEXT AS ABOVE_| +| logical_plan after simplify_expressions_| SAME TEXT AS ABOVE_| +| logical_plan after replace_distinct_aggregate_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_join_| SAME TEXT AS ABOVE_| +| logical_plan after decorrelate_predicate_subquery_| SAME TEXT AS ABOVE_| +| logical_plan after scalar_subquery_to_join_| SAME TEXT AS ABOVE_| +| logical_plan after decorrelate_lateral_join_| SAME TEXT AS ABOVE_| +| logical_plan after extract_equijoin_predicate_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_duplicated_expr_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_filter_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_cross_join_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_limit_| SAME TEXT AS ABOVE_| +| logical_plan after propagate_empty_relation_| SAME TEXT AS ABOVE_| +| logical_plan after filter_null_join_keys_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_outer_join_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_limit_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_filter_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: CAST(test_nano.j AS Timestamp(ms)) >= TimestampMillisecond(-299999, None) AND CAST(test_nano.j AS Timestamp(ms)) <= TimestampMillisecond(10000, None)_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after single_distinct_aggregation_to_group_by | SAME TEXT AS ABOVE_| +| logical_plan after eliminate_group_by_constant_| SAME TEXT AS ABOVE_| +| logical_plan after common_sub_expression_eliminate_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_1 >= TimestampMillisecond(-299999, None) AND __common_expr_1 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_1, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after extract_leaf_expressions_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_leaf_projections_| SAME TEXT AS ABOVE_| +| logical_plan after optimize_projections_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_1 >= TimestampMillisecond(-299999, None) AND __common_expr_1 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_1, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after ScanHintRule_| SAME TEXT AS ABOVE_| +| logical_plan after rewrite_set_comparison_| SAME TEXT AS ABOVE_| +| logical_plan after optimize_unions_| SAME TEXT AS ABOVE_| +| logical_plan after simplify_expressions_| SAME TEXT AS ABOVE_| +| logical_plan after replace_distinct_aggregate_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_join_| SAME TEXT AS ABOVE_| +| logical_plan after decorrelate_predicate_subquery_| SAME TEXT AS ABOVE_| +| logical_plan after scalar_subquery_to_join_| SAME TEXT AS ABOVE_| +| logical_plan after decorrelate_lateral_join_| SAME TEXT AS ABOVE_| +| logical_plan after extract_equijoin_predicate_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_duplicated_expr_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_filter_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_cross_join_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_limit_| SAME TEXT AS ABOVE_| +| logical_plan after propagate_empty_relation_| SAME TEXT AS ABOVE_| +| logical_plan after filter_null_join_keys_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_outer_join_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_limit_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_filter_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_1, test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: CAST(test_nano.j AS Timestamp(ms)) >= TimestampMillisecond(-299999, None) AND CAST(test_nano.j AS Timestamp(ms)) <= TimestampMillisecond(10000, None)_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after single_distinct_aggregation_to_group_by | SAME TEXT AS ABOVE_| +| logical_plan after eliminate_group_by_constant_| SAME TEXT AS ABOVE_| +| logical_plan after common_sub_expression_eliminate_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_1, test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_2 >= TimestampMillisecond(-299999, None) AND __common_expr_2 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_2, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after extract_leaf_expressions_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_leaf_projections_| SAME TEXT AS ABOVE_| +| logical_plan after optimize_projections_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_2 >= TimestampMillisecond(-299999, None) AND __common_expr_2 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_2, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after ScanHintRule_| SAME TEXT AS ABOVE_| +| logical_plan after rewrite_set_comparison_| SAME TEXT AS ABOVE_| +| logical_plan after optimize_unions_| SAME TEXT AS ABOVE_| +| logical_plan after simplify_expressions_| SAME TEXT AS ABOVE_| +| logical_plan after replace_distinct_aggregate_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_join_| SAME TEXT AS ABOVE_| +| logical_plan after decorrelate_predicate_subquery_| SAME TEXT AS ABOVE_| +| logical_plan after scalar_subquery_to_join_| SAME TEXT AS ABOVE_| +| logical_plan after decorrelate_lateral_join_| SAME TEXT AS ABOVE_| +| logical_plan after extract_equijoin_predicate_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_duplicated_expr_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_filter_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_cross_join_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_limit_| SAME TEXT AS ABOVE_| +| logical_plan after propagate_empty_relation_| SAME TEXT AS ABOVE_| +| logical_plan after filter_null_join_keys_| SAME TEXT AS ABOVE_| +| logical_plan after eliminate_outer_join_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_limit_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_filter_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_2, test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: CAST(test_nano.j AS Timestamp(ms)) >= TimestampMillisecond(-299999, None) AND CAST(test_nano.j AS Timestamp(ms)) <= TimestampMillisecond(10000, None)_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after single_distinct_aggregation_to_group_by | SAME TEXT AS ABOVE_| +| logical_plan after eliminate_group_by_constant_| SAME TEXT AS ABOVE_| +| logical_plan after common_sub_expression_eliminate_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_2, test_nano.i, test_nano.j, test_nano.k_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_3 >= TimestampMillisecond(-299999, None) AND __common_expr_3 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_3, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after extract_leaf_expressions_| SAME TEXT AS ABOVE_| +| logical_plan after push_down_leaf_projections_| SAME TEXT AS ABOVE_| +| logical_plan after optimize_projections_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_3 >= TimestampMillisecond(-299999, None) AND __common_expr_3 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_3, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| logical_plan after ScanHintRule_| SAME TEXT AS ABOVE_| +| logical_plan_| PromInstantManipulate: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test_nano.k ASC NULLS FIRST, test_nano.j ASC NULLS FIRST_| +|_|_Projection: test_nano.i, test_nano.k, CAST(test_nano.j AS Timestamp(ms)) AS j_| +|_|_Projection: test_nano.i, test_nano.j, test_nano.k_| +|_|_Filter: __common_expr_3 >= TimestampMillisecond(-299999, None) AND __common_expr_3 <= TimestampMillisecond(10000, None)_| +|_|_Projection: CAST(test_nano.j AS Timestamp(ms)) AS __common_expr_3, test_nano.i, test_nano.j, test_nano.k_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: test_nano_| +|_| ]]_| +| initial_physical_plan_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[false]_| +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| initial_physical_plan_with_stats_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j], statistics=[Rows=Inexact(2), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_PromSeriesDivideExec: tags=["k"], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[false], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:)]]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:)]] | +|_|_MergeScanExec: REDACTED +|_|_| +| initial_physical_plan_with_schema_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_PromSeriesDivideExec: tags=["k"], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[false], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k], schema=[i:Float64;N, j:Timestamp(ns), k:Utf8;N]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, schema=[__common_expr_3:Timestamp(ms), i:Float64;N, j:Timestamp(ns), k:Utf8;N]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k], schema=[__common_expr_3:Timestamp(ms), i:Float64;N, j:Timestamp(ns), k:Utf8;N]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after OutputRequirements_| OutputRequirementExec: order_by=[], dist_by=Unspecified_| +|_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[false]_| +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after aggregate_statistics_| SAME TEXT AS ABOVE_| +| physical_plan after join_selection_| SAME TEXT AS ABOVE_| +| physical_plan after LimitedDistinctAggregation_| SAME TEXT AS ABOVE_| +| physical_plan after FilterPushdown_| SAME TEXT AS ABOVE_| +| physical_plan after parallelize_scan_| SAME TEXT AS ABOVE_| +| physical_plan after PassDistributionRule_| SAME TEXT AS ABOVE_| +| physical_plan after EnforceSorting_| OutputRequirementExec: order_by=[], dist_by=Unspecified_| +|_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after EnforceDistribution_| OutputRequirementExec: order_by=[], dist_by=Unspecified_| +|_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_RepartitionExec: REDACTED +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after CombinePartialFinalAggregate_| SAME TEXT AS ABOVE_| +| physical_plan after EnforceSorting_| OutputRequirementExec: order_by=[], dist_by=Unspecified_| +|_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_RepartitionExec: REDACTED +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_ProjectionExec: expr=[i@1 as i, j@2 as j, k@3 as k]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after OptimizeAggregateOrder_| SAME TEXT AS ABOVE_| +| physical_plan after ProjectionPushdown_| OutputRequirementExec: order_by=[], dist_by=Unspecified_| +|_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_RepartitionExec: REDACTED +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, projection=[i@1, j@2, k@3]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after OutputRequirements_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_RepartitionExec: REDACTED +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, projection=[i@1, j@2, k@3]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan after LimitAggregation_| SAME TEXT AS ABOVE_| +| physical_plan after LimitPushPastWindows_| SAME TEXT AS ABOVE_| +| physical_plan after LimitPushdown_| SAME TEXT AS ABOVE_| +| physical_plan after ProjectionPushdown_| SAME TEXT AS ABOVE_| +| physical_plan after PushdownSort_| SAME TEXT AS ABOVE_| +| physical_plan after EnsureCooperative_| SAME TEXT AS ABOVE_| +| physical_plan after FilterPushdown(Post)_| SAME TEXT AS ABOVE_| +| physical_plan after WindowedSortRule_| SAME TEXT AS ABOVE_| +| physical_plan after MatchesConstantTerm_| SAME TEXT AS ABOVE_| +| physical_plan after RemoveDuplicateRule_| SAME TEXT AS ABOVE_| +| physical_plan after SanityCheckPlan_| SAME TEXT AS ABOVE_| +| physical_plan_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j]_| +|_|_PromSeriesDivideExec: tags=["k"]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true]_| +|_|_RepartitionExec: REDACTED +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, projection=[i@1, j@2, k@3]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k]_| +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan_with_stats_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j], statistics=[Rows=Inexact(2), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_PromSeriesDivideExec: tags=["k"], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_RepartitionExec: REDACTED +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, projection=[i@1, j@2, k@3], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:)]] | +|_|_MergeScanExec: REDACTED +|_|_| +| physical_plan_with_schema_| PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_PromSeriesDivideExec: tags=["k"], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_SortExec: expr=[k@1 ASC, j@2 ASC], preserve_partitioning=[true], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_RepartitionExec: REDACTED +|_|_ProjectionExec: expr=[i@0 as i, k@2 as k, CAST(j@1 AS Timestamp(ms)) as j], schema=[i:Float64;N, k:Utf8;N, j:Timestamp(ms)]_| +|_|_FilterExec: __common_expr_3@0 >= -299999 AND __common_expr_3@0 <= 10000, projection=[i@1, j@2, k@3], schema=[i:Float64;N, j:Timestamp(ns), k:Utf8;N]_| +|_|_ProjectionExec: expr=[CAST(j@1 AS Timestamp(ms)) as __common_expr_3, i@0 as i, j@1 as j, k@2 as k], schema=[__common_expr_3:Timestamp(ms), i:Float64;N, j:Timestamp(ns), k:Utf8;N]_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +DROP TABLE test_nano; + +Affected Rows: 0 + DROP TABLE test; Affected Rows: 0 diff --git a/tests/cases/standalone/common/tql-explain-analyze/explain.sql b/tests/cases/standalone/common/tql-explain-analyze/explain.sql index 5e296d4b6b..62685b0952 100644 --- a/tests/cases/standalone/common/tql-explain-analyze/explain.sql +++ b/tests/cases/standalone/common/tql-explain-analyze/explain.sql @@ -25,6 +25,7 @@ TQL EXPLAIN ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp -- SQLNESS REPLACE (elapsed_compute.*) REDACTED -- SQLNESS REPLACE (peers.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED TQL EXPLAIN VERBOSE (0, 10, '5s') test; -- explain verbose at 0s, 5s and 10s. No point at 0s. @@ -33,6 +34,28 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test; -- SQLNESS REPLACE (elapsed_compute.*) REDACTED -- SQLNESS REPLACE (peers.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED TQL EXPLAIN VERBOSE (0, 10, '5s') test AS series; +CREATE TABLE test_nano(i DOUBLE, j TIMESTAMP(9) TIME INDEX, k STRING PRIMARY KEY); + +INSERT INTO test_nano VALUES (1, 1000000, "a"), (1, 1000000, "b"), (2, 2000000, "a"); + +-- explain at 0s, 5s and 10s for a nanosecond time index. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED +TQL EXPLAIN (0, 10, '5s') test_nano; + +-- explain verbose at 0s, 5s and 10s for a nanosecond time index. +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (elapsed_compute.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (RepartitionExec:.*) RepartitionExec: REDACTED +TQL EXPLAIN VERBOSE (0, 10, '5s') test_nano; + +DROP TABLE test_nano; + DROP TABLE test; diff --git a/tests/cases/standalone/optimizer/time_index_filter_pushdown.result b/tests/cases/standalone/optimizer/time_index_filter_pushdown.result index ea10cfc416..1f281c684a 100644 --- a/tests/cases/standalone/optimizer/time_index_filter_pushdown.result +++ b/tests/cases/standalone/optimizer/time_index_filter_pushdown.result @@ -4,6 +4,7 @@ CREATE TABLE IF NOT EXISTS `cpu` ( `rack` STRING NULL, `os` STRING NULL, + `usage_small` SMALLINT NULL, `usage_user` BIGINT NULL, `greptime_timestamp` TIMESTAMP(9) NOT NULL, TIME INDEX (`greptime_timestamp`), @@ -25,11 +26,11 @@ Affected Rows: 0 INSERT INTO cpu VALUES - ("1", "linux", 10, "2023-06-12 01:04:49"), - ("1", "linux", 15, "2023-06-12 01:04:50"), - ("3", "windows", 25, "2023-06-12 01:05:00"), - ("5", "mac", 30, "2023-06-12 01:03:00"), - ("7", "linux", 45, "2023-06-12 02:00:00"); + ("1", "linux", 10, 10, "2023-06-12 01:04:49"), + ("1", "linux", 15, 15, "2023-06-12 01:04:50"), + ("3", "windows", 25, 25, "2023-06-12 01:05:00"), + ("5", "mac", 30, 30, "2023-06-12 01:03:00"), + ("7", "linux", 45, 45, "2023-06-12 02:00:00"); Affected Rows: 5 @@ -44,14 +45,108 @@ ADMIN FLUSH_TABLE ('cpu'); INSERT INTO cpu VALUES - ("2", "linux", 20, "2023-06-12 01:04:51"), - ("2", "windows", 22, "2023-06-12 01:06:00"), - ("4", "mac", 12, "2023-06-12 00:59:00"), - ("6", "linux", 35, "2023-06-12 01:04:55"), - ("8", "windows", 50, "2023-06-12 02:10:00"); + ("2", "linux", 20, 20, "2023-06-12 01:04:51"), + ("2", "windows", 22, 22, "2023-06-12 01:06:00"), + ("4", "mac", 12, 12, "2023-06-12 00:59:00"), + ("6", "linux", 35, 35, "2023-06-12 01:04:55"), + ("8", "windows", 50, 50, "2023-06-12 02:10:00"); Affected Rows: 5 +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN SELECT + rack +FROM + cpu +WHERE + usage_small IN (10, 20); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: cpu.rack | +| | Filter: cpu.usage_small = Int16(10) OR cpu.usage_small = Int16(20) | +| | TableScan: cpu | +| | ]] | +| physical_plan | CooperativeExec | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN SELECT + rack +FROM + cpu +WHERE + usage_small BETWEEN 10 AND 20; + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: cpu.rack | +| | Filter: cpu.usage_small >= Int16(10) AND cpu.usage_small <= Int16(20) | +| | TableScan: cpu | +| | ]] | +| physical_plan | CooperativeExec | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------+ + +CREATE TABLE + IF NOT EXISTS `cpu_single` ( + `rack` STRING NULL, + `usage_small` SMALLINT NULL, + `greptime_timestamp` TIMESTAMP(9) NOT NULL, + TIME INDEX (`greptime_timestamp`), + ) ENGINE = mito +WITH + (append_mode = 'true', sst_format = 'flat'); + +Affected Rows: 0 + +INSERT INTO + cpu_single +VALUES + ("1", 10, "2023-06-12 01:04:49"), + ("2", 20, "2023-06-12 01:04:50"), + ("3", 25, "2023-06-12 01:05:00"); + +Affected Rows: 3 + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE "partition_count":\{(.*?)\} "partition_count":REDACTED +-- SQLNESS REPLACE "flat_format":\s\w+, "flat_format": REDACTED, +EXPLAIN ANALYZE VERBOSE SELECT + rack +FROM + cpu_single +WHERE + 10 <= usage_small; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_FilterExec: usage_small@1 >= 10, projection=[rack@0] REDACTED +|_|_|_CooperativeExec REDACTED +|_|_|_UnorderedScan: region=REDACTED, {"partition_count":REDACTED, "projection": ["rack", "usage_small"], "filters": ["usage_small >= Int16(10)"], "flat_format": REDACTED, "REDACTED +|_|_|_| +|_|_| Total rows: 3_| ++-+-+-+ + +drop table cpu_single; + +Affected Rows: 0 + -- SQLNESS SORT_RESULT 3 1 select count(*) diff --git a/tests/cases/standalone/optimizer/time_index_filter_pushdown.sql b/tests/cases/standalone/optimizer/time_index_filter_pushdown.sql index 28f4180a2c..c0636e26a4 100644 --- a/tests/cases/standalone/optimizer/time_index_filter_pushdown.sql +++ b/tests/cases/standalone/optimizer/time_index_filter_pushdown.sql @@ -4,6 +4,7 @@ CREATE TABLE IF NOT EXISTS `cpu` ( `rack` STRING NULL, `os` STRING NULL, + `usage_small` SMALLINT NULL, `usage_user` BIGINT NULL, `greptime_timestamp` TIMESTAMP(9) NOT NULL, TIME INDEX (`greptime_timestamp`), @@ -23,22 +24,71 @@ WITH INSERT INTO cpu VALUES - ("1", "linux", 10, "2023-06-12 01:04:49"), - ("1", "linux", 15, "2023-06-12 01:04:50"), - ("3", "windows", 25, "2023-06-12 01:05:00"), - ("5", "mac", 30, "2023-06-12 01:03:00"), - ("7", "linux", 45, "2023-06-12 02:00:00"); + ("1", "linux", 10, 10, "2023-06-12 01:04:49"), + ("1", "linux", 15, 15, "2023-06-12 01:04:50"), + ("3", "windows", 25, 25, "2023-06-12 01:05:00"), + ("5", "mac", 30, 30, "2023-06-12 01:03:00"), + ("7", "linux", 45, 45, "2023-06-12 02:00:00"); ADMIN FLUSH_TABLE ('cpu'); INSERT INTO cpu VALUES - ("2", "linux", 20, "2023-06-12 01:04:51"), - ("2", "windows", 22, "2023-06-12 01:06:00"), - ("4", "mac", 12, "2023-06-12 00:59:00"), - ("6", "linux", 35, "2023-06-12 01:04:55"), - ("8", "windows", 50, "2023-06-12 02:10:00"); + ("2", "linux", 20, 20, "2023-06-12 01:04:51"), + ("2", "windows", 22, 22, "2023-06-12 01:06:00"), + ("4", "mac", 12, 12, "2023-06-12 00:59:00"), + ("6", "linux", 35, 35, "2023-06-12 01:04:55"), + ("8", "windows", 50, 50, "2023-06-12 02:10:00"); + +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN SELECT + rack +FROM + cpu +WHERE + usage_small IN (10, 20); + +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN SELECT + rack +FROM + cpu +WHERE + usage_small BETWEEN 10 AND 20; + +CREATE TABLE + IF NOT EXISTS `cpu_single` ( + `rack` STRING NULL, + `usage_small` SMALLINT NULL, + `greptime_timestamp` TIMESTAMP(9) NOT NULL, + TIME INDEX (`greptime_timestamp`), + ) ENGINE = mito +WITH + (append_mode = 'true', sst_format = 'flat'); + +INSERT INTO + cpu_single +VALUES + ("1", 10, "2023-06-12 01:04:49"), + ("2", 20, "2023-06-12 01:04:50"), + ("3", 25, "2023-06-12 01:05:00"); + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE "partition_count":\{(.*?)\} "partition_count":REDACTED +-- SQLNESS REPLACE "flat_format":\s\w+, "flat_format": REDACTED, +EXPLAIN ANALYZE VERBOSE SELECT + rack +FROM + cpu_single +WHERE + 10 <= usage_small; + +drop table cpu_single; -- SQLNESS SORT_RESULT 3 1 select @@ -70,4 +120,4 @@ where group by os; -drop table cpu; \ No newline at end of file +drop table cpu; diff --git a/tests/cases/standalone/optimizer/windowed_sort_advance.result b/tests/cases/standalone/optimizer/windowed_sort_advance.result new file mode 100644 index 0000000000..c0be1bece9 --- /dev/null +++ b/tests/cases/standalone/optimizer/windowed_sort_advance.result @@ -0,0 +1,117 @@ +create table `a` (`value` double, `status` bigint, ts timestamp(9) time index); + +Affected Rows: 0 + +INSERT INTO `a` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.000000000+08:00'), +(46.84, 200, '2026-03-12T08:00:15.000000000+08:00'), +(46.85, 200, '2026-03-12T08:00:25.000000000+08:00'), +(46.86, 200, '2026-03-12T08:00:35.000000000+08:00'), +(46.88, 200, '2026-03-12T08:00:45.000000000+08:00'), +(46.89, 200, '2026-03-12T08:00:55.000000000+08:00'), +(46.91, 200, '2026-03-12T08:01:05.000000000+08:00'), +(46.90, 200, '2026-03-12T08:01:15.000000000+08:00'), +(46.87, 200, '2026-03-12T08:01:25.000000000+08:00'), +(46.85, 200, '2026-03-12T08:01:35.000000000+08:00'); + +Affected Rows: 10 + +select ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++---------------------+--------+-------+ +| ts | status | value | ++---------------------+--------+-------+ +| 2026-03-12T00:00:05 | 200 | 46.82 | +| 2026-03-12T00:00:15 | 200 | 46.84 | +| 2026-03-12T00:00:25 | 200 | 46.85 | +| 2026-03-12T00:00:35 | 200 | 46.86 | +| 2026-03-12T00:00:45 | 200 | 46.88 | +| 2026-03-12T00:00:55 | 200 | 46.89 | +| 2026-03-12T00:01:05 | 200 | 46.91 | +| 2026-03-12T00:01:15 | 200 | 46.9 | +| 2026-03-12T00:01:25 | 200 | 46.87 | +| 2026-03-12T00:01:35 | 200 | 46.85 | ++---------------------+--------+-------+ + +select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++---------------------+--------+-------+ +| ts | status | value | ++---------------------+--------+-------+ +| 2026-03-12T00:00:05 | 200 | 46.82 | +| 2026-03-12T00:00:15 | 200 | 46.84 | +| 2026-03-12T00:00:25 | 200 | 46.85 | +| 2026-03-12T00:00:35 | 200 | 46.86 | +| 2026-03-12T00:00:45 | 200 | 46.88 | +| 2026-03-12T00:00:55 | 200 | 46.89 | +| 2026-03-12T00:01:05 | 200 | 46.91 | +| 2026-03-12T00:01:15 | 200 | 46.9 | +| 2026-03-12T00:01:25 | 200 | 46.87 | +| 2026-03-12T00:01:35 | 200 | 46.85 | ++---------------------+--------+-------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED +|_|_|_WindowedSortExec: expr=ts@0 ASC NULLS LAST num_ranges=REDACTED REDACTED +|_|_|_ProjectionExec: expr=[ts@2 as ts, status@1 as status, value@0 as value] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 10_| ++-+-+-+ + +select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++---------------------+--------+-------+ +| ts | status | value | ++---------------------+--------+-------+ +| 2026-03-12T00:00:05 | 200 | 46.82 | +| 2026-03-12T00:00:15 | 200 | 46.84 | +| 2026-03-12T00:00:25 | 200 | 46.85 | +| 2026-03-12T00:00:35 | 200 | 46.86 | +| 2026-03-12T00:00:45 | 200 | 46.88 | +| 2026-03-12T00:00:55 | 200 | 46.89 | +| 2026-03-12T00:01:05 | 200 | 46.91 | +| 2026-03-12T00:01:15 | 200 | 46.9 | +| 2026-03-12T00:01:25 | 200 | 46.87 | +| 2026-03-12T00:01:35 | 200 | 46.85 | ++---------------------+--------+-------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED +|_|_|_WindowedSortExec: expr=ts@0 ASC NULLS LAST num_ranges=REDACTED REDACTED +|_|_|_ProjectionExec: expr=[to_timestamp_millis(ts@2) as ts, status@1 as status, value@0 as value] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 10_| ++-+-+-+ + +DROP TABLE `a`; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/optimizer/windowed_sort_advance.sql b/tests/cases/standalone/optimizer/windowed_sort_advance.sql new file mode 100644 index 0000000000..fa73b97933 --- /dev/null +++ b/tests/cases/standalone/optimizer/windowed_sort_advance.sql @@ -0,0 +1,37 @@ +create table `a` (`value` double, `status` bigint, ts timestamp(9) time index); + +INSERT INTO `a` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.000000000+08:00'), +(46.84, 200, '2026-03-12T08:00:15.000000000+08:00'), +(46.85, 200, '2026-03-12T08:00:25.000000000+08:00'), +(46.86, 200, '2026-03-12T08:00:35.000000000+08:00'), +(46.88, 200, '2026-03-12T08:00:45.000000000+08:00'), +(46.89, 200, '2026-03-12T08:00:55.000000000+08:00'), +(46.91, 200, '2026-03-12T08:01:05.000000000+08:00'), +(46.90, 200, '2026-03-12T08:01:15.000000000+08:00'), +(46.87, 200, '2026-03-12T08:01:25.000000000+08:00'), +(46.85, 200, '2026-03-12T08:01:35.000000000+08:00'); + +select ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select ts as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_millis(ts) as ts, status, value from `a` where ts >= '2026-03-12T08:00:00+08:00' and ts < '2026-03-12T08:02:01+08:00' order by ts asc; + +DROP TABLE `a`; diff --git a/tests/cases/standalone/optimizer/windowed_sort_nanos.result b/tests/cases/standalone/optimizer/windowed_sort_nanos.result new file mode 100644 index 0000000000..f673636307 --- /dev/null +++ b/tests/cases/standalone/optimizer/windowed_sort_nanos.result @@ -0,0 +1,78 @@ +create table `a_ms` (`value` double, `status` bigint, ts timestamp(3) time index); + +Affected Rows: 0 + +INSERT INTO `a_ms` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.123+08:00'), +(46.84, 200, '2026-03-12T08:00:15.234+08:00'), +(46.85, 200, '2026-03-12T08:00:25.345+08:00'), +(46.86, 200, '2026-03-12T08:00:35.456+08:00'), +(46.88, 200, '2026-03-12T08:00:45.567+08:00'), +(46.89, 200, '2026-03-12T08:00:55.678+08:00'), +(46.91, 200, '2026-03-12T08:01:05.789+08:00'), +(46.90, 200, '2026-03-12T08:01:15.890+08:00'), +(46.87, 200, '2026-03-12T08:01:25.901+08:00'), +(46.85, 200, '2026-03-12T08:01:35.999+08:00'); + +Affected Rows: 10 + +select ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + ++-------------------------+--------+-------+ +| ts | status | value | ++-------------------------+--------+-------+ +| 2026-03-12T00:00:05.123 | 200 | 46.82 | +| 2026-03-12T00:00:15.234 | 200 | 46.84 | +| 2026-03-12T00:00:25.345 | 200 | 46.85 | +| 2026-03-12T00:00:35.456 | 200 | 46.86 | +| 2026-03-12T00:00:45.567 | 200 | 46.88 | +| 2026-03-12T00:00:55.678 | 200 | 46.89 | +| 2026-03-12T00:01:05.789 | 200 | 46.91 | +| 2026-03-12T00:01:15.890 | 200 | 46.9 | +| 2026-03-12T00:01:25.901 | 200 | 46.87 | +| 2026-03-12T00:01:35.999 | 200 | 46.85 | ++-------------------------+--------+-------+ + +select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + ++-------------------------+--------+-------+ +| ts | status | value | ++-------------------------+--------+-------+ +| 2026-03-12T00:00:05.123 | 200 | 46.82 | +| 2026-03-12T00:00:15.234 | 200 | 46.84 | +| 2026-03-12T00:00:25.345 | 200 | 46.85 | +| 2026-03-12T00:00:35.456 | 200 | 46.86 | +| 2026-03-12T00:00:45.567 | 200 | 46.88 | +| 2026-03-12T00:00:55.678 | 200 | 46.89 | +| 2026-03-12T00:01:05.789 | 200 | 46.91 | +| 2026-03-12T00:01:15.890 | 200 | 46.9 | +| 2026-03-12T00:01:25.901 | 200 | 46.87 | +| 2026-03-12T00:01:35.999 | 200 | 46.85 | ++-------------------------+--------+-------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED +|_|_|_WindowedSortExec: expr=ts@0 ASC NULLS LAST num_ranges=REDACTED REDACTED +|_|_|_ProjectionExec: expr=[to_timestamp_nanos(ts@2) as ts, status@1 as status, value@0 as value] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 10_| ++-+-+-+ + +DROP TABLE `a_ms`; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/optimizer/windowed_sort_nanos.sql b/tests/cases/standalone/optimizer/windowed_sort_nanos.sql new file mode 100644 index 0000000000..6a6cf51c4b --- /dev/null +++ b/tests/cases/standalone/optimizer/windowed_sort_nanos.sql @@ -0,0 +1,27 @@ +create table `a_ms` (`value` double, `status` bigint, ts timestamp(3) time index); + +INSERT INTO `a_ms` (`value`, `status`, `ts`) VALUES +(46.82, 200, '2026-03-12T08:00:05.123+08:00'), +(46.84, 200, '2026-03-12T08:00:15.234+08:00'), +(46.85, 200, '2026-03-12T08:00:25.345+08:00'), +(46.86, 200, '2026-03-12T08:00:35.456+08:00'), +(46.88, 200, '2026-03-12T08:00:45.567+08:00'), +(46.89, 200, '2026-03-12T08:00:55.678+08:00'), +(46.91, 200, '2026-03-12T08:01:05.789+08:00'), +(46.90, 200, '2026-03-12T08:01:15.890+08:00'), +(46.87, 200, '2026-03-12T08:01:25.901+08:00'), +(46.85, 200, '2026-03-12T08:01:35.999+08:00'); + +select ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + +select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- SQLNESS REPLACE num_ranges=\d+ num_ranges=REDACTED +EXPLAIN ANALYZE select to_timestamp_nanos(ts) as ts, status, value from `a_ms` where ts >= '2026-03-12T08:00:00.000+08:00' and ts < '2026-03-12T08:02:01.000+08:00' order by ts asc; + +DROP TABLE `a_ms`;