diff --git a/src/common/meta/src/cache/flow/table_flownode.rs b/src/common/meta/src/cache/flow/table_flownode.rs index c2ccd9fd72..152d384643 100644 --- a/src/common/meta/src/cache/flow/table_flownode.rs +++ b/src/common/meta/src/cache/flow/table_flownode.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; +use common_telemetry::info; use futures::future::BoxFuture; use moka::future::Cache; use moka::ops::compute::Op; @@ -89,6 +90,12 @@ fn init_factory(table_flow_manager: TableFlowManagerRef) -> Initializer( match ident { CacheIdent::CreateFlow(create_flow) => handle_create_flow(cache, create_flow).await, CacheIdent::DropFlow(drop_flow) => handle_drop_flow(cache, drop_flow).await, + CacheIdent::FlowNodeAddressChange(node_id) => { + info!( + "Invalidate flow node cache for node_id in table_flownode: {}", + node_id + ); + cache.invalidate_all(); + } _ => {} } Ok(()) @@ -174,7 +188,10 @@ fn invalidator<'a>( } fn filter(ident: &CacheIdent) -> bool { - matches!(ident, CacheIdent::CreateFlow(_) | CacheIdent::DropFlow(_)) + matches!( + ident, + CacheIdent::CreateFlow(_) | CacheIdent::DropFlow(_) | CacheIdent::FlowNodeAddressChange(_) + ) } #[cfg(test)] diff --git a/src/common/meta/src/cache_invalidator.rs b/src/common/meta/src/cache_invalidator.rs index 41098ed69b..13ad6dff36 100644 --- a/src/common/meta/src/cache_invalidator.rs +++ b/src/common/meta/src/cache_invalidator.rs @@ -22,6 +22,7 @@ use crate::key::flow::flow_name::FlowNameKey; use crate::key::flow::flow_route::FlowRouteKey; use crate::key::flow::flownode_flow::FlownodeFlowKey; use crate::key::flow::table_flow::TableFlowKey; +use crate::key::node_address::NodeAddressKey; use crate::key::schema_name::SchemaNameKey; use crate::key::table_info::TableInfoKey; use crate::key::table_name::TableNameKey; @@ -53,6 +54,10 @@ pub struct Context { #[async_trait::async_trait] pub trait CacheInvalidator: Send + Sync { async fn invalidate(&self, ctx: &Context, caches: &[CacheIdent]) -> Result<()>; + + fn name(&self) -> &'static str { + std::any::type_name::() + } } pub type CacheInvalidatorRef = Arc; @@ -137,6 +142,13 @@ where let key = FlowInfoKey::new(*flow_id); self.invalidate_key(&key.to_bytes()).await; } + CacheIdent::FlowNodeAddressChange(node_id) => { + // other caches doesn't need to be invalidated + // since this is only for flownode address change not id change + common_telemetry::info!("Invalidate flow node cache for node_id: {}", node_id); + let key = NodeAddressKey::with_flownode(*node_id); + self.invalidate_key(&key.to_bytes()).await; + } } } Ok(()) diff --git a/src/common/meta/src/instruction.rs b/src/common/meta/src/instruction.rs index 203262a001..1d2259310c 100644 --- a/src/common/meta/src/instruction.rs +++ b/src/common/meta/src/instruction.rs @@ -174,6 +174,8 @@ pub struct UpgradeRegion { /// The identifier of cache. pub enum CacheIdent { FlowId(FlowId), + /// Indicate change of address of flownode. + FlowNodeAddressChange(u64), FlowName(FlowName), TableId(TableId), TableName(TableName), diff --git a/src/meta-srv/src/handler/remap_flow_peer_handler.rs b/src/meta-srv/src/handler/remap_flow_peer_handler.rs index 494af777fe..ec6cdb7b5a 100644 --- a/src/meta-srv/src/handler/remap_flow_peer_handler.rs +++ b/src/meta-srv/src/handler/remap_flow_peer_handler.rs @@ -13,6 +13,7 @@ // limitations under the License. use api::v1::meta::{HeartbeatRequest, Peer, Role}; +use common_meta::instruction::CacheIdent; use common_meta::key::node_address::{NodeAddressKey, NodeAddressValue}; use common_meta::key::{MetadataKey, MetadataValue}; use common_meta::rpc::store::PutRequest; @@ -80,7 +81,19 @@ async fn rewrite_node_address(ctx: &mut Context, peer: &Peer) { match ctx.leader_cached_kv_backend.put(put).await { Ok(_) => { info!("Successfully updated flow `NodeAddressValue`: {:?}", peer); - // TODO(discord): broadcast invalidating cache to all frontends + // broadcast invalidating cache to all frontends + let cache_idents = vec![CacheIdent::FlowNodeAddressChange(peer.id)]; + info!( + "Invalidate flow node cache for new address with cache idents: {:?}", + cache_idents + ); + if let Err(e) = ctx + .cache_invalidator + .invalidate(&Default::default(), &cache_idents) + .await + { + error!(e; "Failed to invalidate {} `NodeAddressKey` cache, peer: {:?}", cache_idents.len(), peer); + } } Err(e) => { error!(e; "Failed to update flow `NodeAddressValue`: {:?}", peer); diff --git a/src/mito2/src/compaction.rs b/src/mito2/src/compaction.rs index ca50f6c6eb..37c9a8a845 100644 --- a/src/mito2/src/compaction.rs +++ b/src/mito2/src/compaction.rs @@ -62,7 +62,7 @@ use crate::read::BoxedBatchReader; use crate::region::options::MergeMode; use crate::region::version::VersionControlRef; use crate::region::ManifestContextRef; -use crate::request::{OptionOutputTx, OutputTx, WorkerRequest}; +use crate::request::{OptionOutputTx, OutputTx, WorkerRequestWithTime}; use crate::schedule::remote_job_scheduler::{ CompactionJob, DefaultNotifier, RemoteJob, RemoteJobSchedulerRef, }; @@ -77,7 +77,7 @@ pub struct CompactionRequest { pub(crate) current_version: CompactionVersion, pub(crate) access_layer: AccessLayerRef, /// Sender to send notification to the region worker. - pub(crate) request_sender: mpsc::Sender, + pub(crate) request_sender: mpsc::Sender, /// Waiters of the compaction request. pub(crate) waiters: Vec, /// Start time of compaction task. @@ -101,7 +101,7 @@ pub(crate) struct CompactionScheduler { /// Compacting regions. region_status: HashMap, /// Request sender of the worker that this scheduler belongs to. - request_sender: Sender, + request_sender: Sender, cache_manager: CacheManagerRef, engine_config: Arc, listener: WorkerListener, @@ -112,7 +112,7 @@ pub(crate) struct CompactionScheduler { impl CompactionScheduler { pub(crate) fn new( scheduler: SchedulerRef, - request_sender: Sender, + request_sender: Sender, cache_manager: CacheManagerRef, engine_config: Arc, listener: WorkerListener, @@ -559,7 +559,7 @@ impl CompactionStatus { #[allow(clippy::too_many_arguments)] fn new_compaction_request( &mut self, - request_sender: Sender, + request_sender: Sender, mut waiter: OptionOutputTx, engine_config: Arc, cache_manager: CacheManagerRef, diff --git a/src/mito2/src/compaction/task.rs b/src/mito2/src/compaction/task.rs index 2508c3f6ae..e193665e7a 100644 --- a/src/mito2/src/compaction/task.rs +++ b/src/mito2/src/compaction/task.rs @@ -27,6 +27,7 @@ use crate::manifest::action::RegionEdit; use crate::metrics::{COMPACTION_FAILURE_COUNT, COMPACTION_STAGE_ELAPSED}; use crate::request::{ BackgroundNotify, CompactionFailed, CompactionFinished, OutputTx, WorkerRequest, + WorkerRequestWithTime, }; use crate::worker::WorkerListener; use crate::{error, metrics}; @@ -37,7 +38,7 @@ pub const MAX_PARALLEL_COMPACTION: usize = 1; pub(crate) struct CompactionTaskImpl { pub compaction_region: CompactionRegion, /// Request sender to notify the worker. - pub(crate) request_sender: mpsc::Sender, + pub(crate) request_sender: mpsc::Sender, /// Senders that are used to notify waiters waiting for pending compaction tasks. pub waiters: Vec, /// Start time of compaction task @@ -135,7 +136,11 @@ impl CompactionTaskImpl { /// Notifies region worker to handle post-compaction tasks. async fn send_to_worker(&self, request: WorkerRequest) { - if let Err(e) = self.request_sender.send(request).await { + if let Err(e) = self + .request_sender + .send(WorkerRequestWithTime::new(request)) + .await + { error!( "Failed to notify compaction job status for region {}, request: {:?}", self.compaction_region.region_id, e.0 diff --git a/src/mito2/src/flush.rs b/src/mito2/src/flush.rs index b84aaedc27..3c8f8352ac 100644 --- a/src/mito2/src/flush.rs +++ b/src/mito2/src/flush.rs @@ -42,7 +42,7 @@ use crate::region::version::{VersionControlData, VersionControlRef}; use crate::region::{ManifestContextRef, RegionLeaderState}; use crate::request::{ BackgroundNotify, FlushFailed, FlushFinished, OptionOutputTx, OutputTx, SenderBulkRequest, - SenderDdlRequest, SenderWriteRequest, WorkerRequest, + SenderDdlRequest, SenderWriteRequest, WorkerRequest, WorkerRequestWithTime, }; use crate::schedule::scheduler::{Job, SchedulerRef}; use crate::sst::file::FileMeta; @@ -223,7 +223,7 @@ pub(crate) struct RegionFlushTask { /// Flush result senders. pub(crate) senders: Vec, /// Request sender to notify the worker. - pub(crate) request_sender: mpsc::Sender, + pub(crate) request_sender: mpsc::Sender, pub(crate) access_layer: AccessLayerRef, pub(crate) listener: WorkerListener, @@ -441,7 +441,11 @@ impl RegionFlushTask { /// Notify flush job status. async fn send_worker_request(&self, request: WorkerRequest) { - if let Err(e) = self.request_sender.send(request).await { + if let Err(e) = self + .request_sender + .send(WorkerRequestWithTime::new(request)) + .await + { error!( "Failed to notify flush job status for region {}, request: {:?}", self.region_id, e.0 diff --git a/src/mito2/src/memtable/bulk/part.rs b/src/mito2/src/memtable/bulk/part.rs index 0756bc9231..1addda5af9 100644 --- a/src/mito2/src/memtable/bulk/part.rs +++ b/src/mito2/src/memtable/bulk/part.rs @@ -126,7 +126,12 @@ impl From<&BulkPart> for BulkWalEntry { impl BulkPart { pub(crate) fn estimated_size(&self) -> usize { - self.batch.get_array_memory_size() + self.batch + .columns() + .iter() + // If can not get slice memory size, assume 0 here. + .map(|c| c.to_data().get_slice_memory_size().unwrap_or(0)) + .sum() } /// Converts [BulkPart] to [Mutation] for fallback `write_bulk` implementation. diff --git a/src/mito2/src/metrics.rs b/src/mito2/src/metrics.rs index c9ebbc0fbf..745fc3bec4 100644 --- a/src/mito2/src/metrics.rs +++ b/src/mito2/src/metrics.rs @@ -94,12 +94,7 @@ lazy_static! { // ------ Write related metrics - /// Number of stalled write requests in each worker. - pub static ref WRITE_STALL_TOTAL: IntGaugeVec = register_int_gauge_vec!( - "greptime_mito_write_stall_total", - "mito stalled write request in each worker", - &[WORKER_LABEL] - ).unwrap(); + // /// Counter of rejected write requests. pub static ref WRITE_REJECT_TOTAL: IntCounter = register_int_counter!("greptime_mito_write_reject_total", "mito write reject total").unwrap(); @@ -402,6 +397,7 @@ lazy_static! { } +// Use another block to avoid reaching the recursion limit. lazy_static! { /// Counter for compaction input file size. pub static ref COMPACTION_INPUT_BYTES: Counter = register_counter!( @@ -426,6 +422,27 @@ lazy_static! { "greptime_mito_memtable_field_builder_count", "active field builder count in TimeSeriesMemtable", ).unwrap(); + + /// Number of stalling write requests in each worker. + pub static ref WRITE_STALLING: IntGaugeVec = register_int_gauge_vec!( + "greptime_mito_write_stalling_count", + "mito stalled write request in each worker", + &[WORKER_LABEL] + ).unwrap(); + /// Total number of stalled write requests. + pub static ref WRITE_STALL_TOTAL: IntCounter = register_int_counter!( + "greptime_mito_write_stall_total", + "Total number of stalled write requests" + ).unwrap(); + /// Time waiting for requests to be handled by the region worker. + pub static ref REQUEST_WAIT_TIME: HistogramVec = register_histogram_vec!( + "greptime_mito_request_wait_time", + "mito request wait time before being handled by region worker", + &[WORKER_LABEL], + // 0.001 ~ 10000 + exponential_buckets(0.001, 10.0, 8).unwrap(), + ) + .unwrap(); } /// Stager notifier to collect metrics. diff --git a/src/mito2/src/request.rs b/src/mito2/src/request.rs index de86d32a37..46c00c90bc 100644 --- a/src/mito2/src/request.rs +++ b/src/mito2/src/request.rs @@ -542,6 +542,22 @@ pub(crate) struct SenderBulkRequest { pub(crate) region_metadata: RegionMetadataRef, } +/// Request sent to a worker with timestamp +#[derive(Debug)] +pub(crate) struct WorkerRequestWithTime { + pub(crate) request: WorkerRequest, + pub(crate) created_at: Instant, +} + +impl WorkerRequestWithTime { + pub(crate) fn new(request: WorkerRequest) -> Self { + Self { + request, + created_at: Instant::now(), + } + } +} + /// Request sent to a worker #[derive(Debug)] pub(crate) enum WorkerRequest { diff --git a/src/mito2/src/schedule/remote_job_scheduler.rs b/src/mito2/src/schedule/remote_job_scheduler.rs index bfe31ef041..3e977709e0 100644 --- a/src/mito2/src/schedule/remote_job_scheduler.rs +++ b/src/mito2/src/schedule/remote_job_scheduler.rs @@ -30,6 +30,7 @@ use crate::manifest::action::RegionEdit; use crate::metrics::{COMPACTION_FAILURE_COUNT, INFLIGHT_COMPACTION_COUNT}; use crate::request::{ BackgroundNotify, CompactionFailed, CompactionFinished, OutputTx, WorkerRequest, + WorkerRequestWithTime, }; pub type RemoteJobSchedulerRef = Arc; @@ -130,7 +131,7 @@ pub struct CompactionJobResult { /// DefaultNotifier is a default implementation of Notifier that sends WorkerRequest to the mito engine. pub(crate) struct DefaultNotifier { /// The sender to send WorkerRequest to the mito engine. This is used to notify the mito engine when a remote job is completed. - pub(crate) request_sender: Sender, + pub(crate) request_sender: Sender, } impl DefaultNotifier { @@ -173,10 +174,10 @@ impl Notifier for DefaultNotifier { if let Err(e) = self .request_sender - .send(WorkerRequest::Background { + .send(WorkerRequestWithTime::new(WorkerRequest::Background { region_id: result.region_id, notify, - }) + })) .await { error!( diff --git a/src/mito2/src/test_util/scheduler_util.rs b/src/mito2/src/test_util/scheduler_util.rs index 6f864ef00e..204aac4531 100644 --- a/src/mito2/src/test_util/scheduler_util.rs +++ b/src/mito2/src/test_util/scheduler_util.rs @@ -32,7 +32,7 @@ use crate::error::Result; use crate::flush::FlushScheduler; use crate::manifest::manager::{RegionManifestManager, RegionManifestOptions}; use crate::region::{ManifestContext, ManifestContextRef, RegionLeaderState, RegionRoleState}; -use crate::request::WorkerRequest; +use crate::request::{WorkerRequest, WorkerRequestWithTime}; use crate::schedule::scheduler::{Job, LocalScheduler, Scheduler, SchedulerRef}; use crate::sst::index::intermediate::IntermediateManager; use crate::sst::index::puffin_manager::PuffinManagerFactory; @@ -85,7 +85,7 @@ impl SchedulerEnv { /// Creates a new compaction scheduler. pub(crate) fn mock_compaction_scheduler( &self, - request_sender: Sender, + request_sender: Sender, ) -> CompactionScheduler { let scheduler = self.get_scheduler(); diff --git a/src/mito2/src/worker.rs b/src/mito2/src/worker.rs index 6d9a2153dc..06fd2a86fe 100644 --- a/src/mito2/src/worker.rs +++ b/src/mito2/src/worker.rs @@ -39,7 +39,7 @@ use common_runtime::JoinHandle; use common_telemetry::{error, info, warn}; use futures::future::try_join_all; use object_store::manager::ObjectStoreManagerRef; -use prometheus::IntGauge; +use prometheus::{Histogram, IntGauge}; use rand::{rng, Rng}; use snafu::{ensure, ResultExt}; use store_api::logstore::LogStore; @@ -58,11 +58,11 @@ use crate::error; use crate::error::{CreateDirSnafu, JoinSnafu, Result, WorkerStoppedSnafu}; use crate::flush::{FlushScheduler, WriteBufferManagerImpl, WriteBufferManagerRef}; use crate::memtable::MemtableBuilderProvider; -use crate::metrics::{REGION_COUNT, WRITE_STALL_TOTAL}; +use crate::metrics::{REGION_COUNT, REQUEST_WAIT_TIME, WRITE_STALLING}; use crate::region::{MitoRegionRef, OpeningRegions, OpeningRegionsRef, RegionMap, RegionMapRef}; use crate::request::{ BackgroundNotify, DdlRequest, SenderBulkRequest, SenderDdlRequest, SenderWriteRequest, - WorkerRequest, + WorkerRequest, WorkerRequestWithTime, }; use crate::schedule::scheduler::{LocalScheduler, SchedulerRef}; use crate::sst::file::FileId; @@ -469,8 +469,9 @@ impl WorkerStarter { last_periodical_check_millis: now, flush_sender: self.flush_sender, flush_receiver: self.flush_receiver, - stalled_count: WRITE_STALL_TOTAL.with_label_values(&[&id_string]), + stalling_count: WRITE_STALLING.with_label_values(&[&id_string]), region_count: REGION_COUNT.with_label_values(&[&id_string]), + request_wait_time: REQUEST_WAIT_TIME.with_label_values(&[&id_string]), region_edit_queues: RegionEditQueues::default(), schema_metadata_manager: self.schema_metadata_manager, }; @@ -498,7 +499,7 @@ pub(crate) struct RegionWorker { /// The opening regions. opening_regions: OpeningRegionsRef, /// Request sender. - sender: Sender, + sender: Sender, /// Handle to the worker thread. handle: Mutex>>, /// Whether to run the worker thread. @@ -509,7 +510,8 @@ impl RegionWorker { /// Submits request to background worker thread. async fn submit_request(&self, request: WorkerRequest) -> Result<()> { ensure!(self.is_running(), WorkerStoppedSnafu { id: self.id }); - if self.sender.send(request).await.is_err() { + let request_with_time = WorkerRequestWithTime::new(request); + if self.sender.send(request_with_time).await.is_err() { warn!( "Worker {} is already exited but the running flag is still true", self.id @@ -531,7 +533,12 @@ impl RegionWorker { info!("Stop region worker {}", self.id); self.set_running(false); - if self.sender.send(WorkerRequest::Stop).await.is_err() { + if self + .sender + .send(WorkerRequestWithTime::new(WorkerRequest::Stop)) + .await + .is_err() + { warn!("Worker {} is already exited before stop", self.id); } @@ -669,9 +676,9 @@ struct RegionWorkerLoop { /// Regions that are opening. opening_regions: OpeningRegionsRef, /// Request sender. - sender: Sender, + sender: Sender, /// Request receiver. - receiver: Receiver, + receiver: Receiver, /// WAL of the engine. wal: Wal, /// Manages object stores for manifest and SSTs. @@ -706,10 +713,12 @@ struct RegionWorkerLoop { flush_sender: watch::Sender<()>, /// Watch channel receiver to wait for background flush job. flush_receiver: watch::Receiver<()>, - /// Gauge of stalled request count. - stalled_count: IntGauge, + /// Gauge of stalling request count. + stalling_count: IntGauge, /// Gauge of regions in the worker. region_count: IntGauge, + /// Histogram of request wait time for this worker. + request_wait_time: Histogram, /// Queues for region edit requests. region_edit_queues: RegionEditQueues, /// Database level metadata manager. @@ -749,10 +758,16 @@ impl RegionWorkerLoop { tokio::select! { request_opt = self.receiver.recv() => { match request_opt { - Some(request) => match request { - WorkerRequest::Write(sender_req) => write_req_buffer.push(sender_req), - WorkerRequest::Ddl(sender_req) => ddl_req_buffer.push(sender_req), - _ => general_req_buffer.push(request), + Some(request_with_time) => { + // Observe the wait time + let wait_time = request_with_time.created_at.elapsed(); + self.request_wait_time.observe(wait_time.as_secs_f64()); + + match request_with_time.request { + WorkerRequest::Write(sender_req) => write_req_buffer.push(sender_req), + WorkerRequest::Ddl(sender_req) => ddl_req_buffer.push(sender_req), + req => general_req_buffer.push(req), + } }, // The channel is disconnected. None => break, @@ -791,11 +806,17 @@ impl RegionWorkerLoop { for _ in 1..self.config.worker_request_batch_size { // We have received one request so we start from 1. match self.receiver.try_recv() { - Ok(req) => match req { - WorkerRequest::Write(sender_req) => write_req_buffer.push(sender_req), - WorkerRequest::Ddl(sender_req) => ddl_req_buffer.push(sender_req), - _ => general_req_buffer.push(req), - }, + Ok(request_with_time) => { + // Observe the wait time + let wait_time = request_with_time.created_at.elapsed(); + self.request_wait_time.observe(wait_time.as_secs_f64()); + + match request_with_time.request { + WorkerRequest::Write(sender_req) => write_req_buffer.push(sender_req), + WorkerRequest::Ddl(sender_req) => ddl_req_buffer.push(sender_req), + req => general_req_buffer.push(req), + } + } // We still need to handle remaining requests. Err(_) => break, } diff --git a/src/mito2/src/worker/handle_manifest.rs b/src/mito2/src/worker/handle_manifest.rs index 3edf3c326d..e9c745e476 100644 --- a/src/mito2/src/worker/handle_manifest.rs +++ b/src/mito2/src/worker/handle_manifest.rs @@ -34,7 +34,7 @@ use crate::region::version::VersionBuilder; use crate::region::{MitoRegionRef, RegionLeaderState, RegionRoleState}; use crate::request::{ BackgroundNotify, OptionOutputTx, RegionChangeResult, RegionEditRequest, RegionEditResult, - RegionSyncRequest, TruncateResult, WorkerRequest, + RegionSyncRequest, TruncateResult, WorkerRequest, WorkerRequestWithTime, }; use crate::sst::location; use crate::worker::{RegionWorkerLoop, WorkerListener}; @@ -230,7 +230,10 @@ impl RegionWorkerLoop { }), }; // We don't set state back as the worker loop is already exited. - if let Err(res) = request_sender.send(notify).await { + if let Err(res) = request_sender + .send(WorkerRequestWithTime::new(notify)) + .await + { warn!( "Failed to send region edit result back to the worker, region_id: {}, res: {:?}", region_id, res @@ -318,10 +321,10 @@ impl RegionWorkerLoop { truncated_sequence: truncate.truncated_sequence, }; let _ = request_sender - .send(WorkerRequest::Background { + .send(WorkerRequestWithTime::new(WorkerRequest::Background { region_id: truncate.region_id, notify: BackgroundNotify::Truncate(truncate_result), - }) + })) .await .inspect_err(|_| warn!("failed to send truncate result")); }); @@ -364,7 +367,10 @@ impl RegionWorkerLoop { .on_notify_region_change_result_begin(region.region_id) .await; - if let Err(res) = request_sender.send(notify).await { + if let Err(res) = request_sender + .send(WorkerRequestWithTime::new(notify)) + .await + { warn!( "Failed to send region change result back to the worker, region_id: {}, res: {:?}", region.region_id, res diff --git a/src/mito2/src/worker/handle_write.rs b/src/mito2/src/worker/handle_write.rs index d3389ccefa..6ed9477687 100644 --- a/src/mito2/src/worker/handle_write.rs +++ b/src/mito2/src/worker/handle_write.rs @@ -27,7 +27,9 @@ use store_api::storage::RegionId; use crate::error::{InvalidRequestSnafu, RegionStateSnafu, RejectWriteSnafu, Result}; use crate::metrics; -use crate::metrics::{WRITE_REJECT_TOTAL, WRITE_ROWS_TOTAL, WRITE_STAGE_ELAPSED}; +use crate::metrics::{ + WRITE_REJECT_TOTAL, WRITE_ROWS_TOTAL, WRITE_STAGE_ELAPSED, WRITE_STALL_TOTAL, +}; use crate::region::{RegionLeaderState, RegionRoleState}; use crate::region_write_ctx::RegionWriteCtx; use crate::request::{SenderBulkRequest, SenderWriteRequest, WriteRequest}; @@ -57,8 +59,9 @@ impl RegionWorkerLoop { } if self.write_buffer_manager.should_stall() && allow_stall { - self.stalled_count - .add((write_requests.len() + bulk_requests.len()) as i64); + let stalled_count = (write_requests.len() + bulk_requests.len()) as i64; + self.stalling_count.add(stalled_count); + WRITE_STALL_TOTAL.inc_by(stalled_count as u64); self.stalled_requests.append(write_requests, bulk_requests); self.listener.on_write_stall(); return; @@ -161,7 +164,7 @@ impl RegionWorkerLoop { pub(crate) async fn handle_stalled_requests(&mut self) { // Handle stalled requests. let stalled = std::mem::take(&mut self.stalled_requests); - self.stalled_count.sub(stalled.stalled_count() as i64); + self.stalling_count.sub(stalled.stalled_count() as i64); // We already stalled these requests, don't stall them again. for (_, (_, mut requests, mut bulk)) in stalled.requests { self.handle_write_requests(&mut requests, &mut bulk, false) @@ -172,7 +175,7 @@ impl RegionWorkerLoop { /// Rejects all stalled requests. pub(crate) fn reject_stalled_requests(&mut self) { let stalled = std::mem::take(&mut self.stalled_requests); - self.stalled_count.sub(stalled.stalled_count() as i64); + self.stalling_count.sub(stalled.stalled_count() as i64); for (_, (_, mut requests, mut bulk)) in stalled.requests { reject_write_requests(&mut requests, &mut bulk); } @@ -182,7 +185,8 @@ impl RegionWorkerLoop { pub(crate) fn reject_region_stalled_requests(&mut self, region_id: &RegionId) { debug!("Rejects stalled requests for region {}", region_id); let (mut requests, mut bulk) = self.stalled_requests.remove(region_id); - self.stalled_count.sub((requests.len() + bulk.len()) as i64); + self.stalling_count + .sub((requests.len() + bulk.len()) as i64); reject_write_requests(&mut requests, &mut bulk); } @@ -190,7 +194,8 @@ impl RegionWorkerLoop { pub(crate) async fn handle_region_stalled_requests(&mut self, region_id: &RegionId) { debug!("Handles stalled requests for region {}", region_id); let (mut requests, mut bulk) = self.stalled_requests.remove(region_id); - self.stalled_count.sub((requests.len() + bulk.len()) as i64); + self.stalling_count + .sub((requests.len() + bulk.len()) as i64); self.handle_write_requests(&mut requests, &mut bulk, true) .await; } @@ -251,7 +256,8 @@ impl RegionWorkerLoop { "Region {} is altering, add request to pending writes", region.region_id ); - self.stalled_count.add(1); + self.stalling_count.add(1); + WRITE_STALL_TOTAL.inc(); self.stalled_requests.push(sender_req); continue; } @@ -353,7 +359,8 @@ impl RegionWorkerLoop { "Region {} is altering, add request to pending writes", region.region_id ); - self.stalled_count.add(1); + self.stalling_count.add(1); + WRITE_STALL_TOTAL.inc(); self.stalled_requests.push_bulk(bulk_req); continue; } diff --git a/src/query/src/analyze.rs b/src/query/src/analyze.rs index 6f87e91267..ec5a30f556 100644 --- a/src/query/src/analyze.rs +++ b/src/query/src/analyze.rs @@ -237,7 +237,8 @@ fn create_output_batch( for (node, metric) in sub_stage_metrics.into_iter().enumerate() { builder.append_metric(1, node as _, metrics_to_string(metric, format)?); } - return Ok(TreeNodeRecursion::Stop); + // might have multiple merge scans, so continue + return Ok(TreeNodeRecursion::Continue); } Ok(TreeNodeRecursion::Continue) })?; diff --git a/src/query/src/dist_plan/analyzer.rs b/src/query/src/dist_plan/analyzer.rs index d755050653..1a4b3c6694 100644 --- a/src/query/src/dist_plan/analyzer.rs +++ b/src/query/src/dist_plan/analyzer.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use common_telemetry::debug; @@ -38,6 +38,13 @@ use crate::dist_plan::merge_scan::MergeScanLogicalPlan; use crate::plan::ExtractExpr; use crate::query_engine::DefaultSerializer; +#[cfg(test)] +mod test; + +mod utils; + +pub(crate) use utils::{AliasMapping, AliasTracker}; + #[derive(Debug)] pub struct DistPlannerAnalyzer; @@ -154,7 +161,33 @@ struct PlanRewriter { status: RewriterStatus, /// Partition columns of the table in current pass partition_cols: Option>, - column_requirements: HashSet, + alias_tracker: Option, + /// use stack count as scope to determine column requirements is needed or not + /// i.e for a logical plan like: + /// ```ignore + /// 1: Projection: t.number + /// 2: Sort: t.pk1+t.pk2 + /// 3. Projection: t.number, t.pk1, t.pk2 + /// ``` + /// `Sort` will make a column requirement for `t.pk1` at level 2. + /// Which making `Projection` at level 1 need to add a ref to `t.pk1` as well. + /// So that the expanded plan will be + /// ```ignore + /// Projection: t.number + /// MergeSort: t.pk1 + /// MergeScan: remote_input= + /// Projection: t.number, "t.pk1+t.pk2" <--- the original `Projection` at level 1 get added with `t.pk1+t.pk2` + /// Sort: t.pk1+t.pk2 + /// Projection: t.number, t.pk1, t.pk2 + /// ``` + /// Making `MergeSort` can have `t.pk1` as input. + /// Meanwhile `Projection` at level 3 doesn't need to add any new column because 3 > 2 + /// and col requirements at level 2 is not applicable for level 3. + /// + /// see more details in test `expand_proj_step_aggr` and `expand_proj_sort_proj` + /// + /// TODO(discord9): a simpler solution to track column requirements for merge scan + column_requirements: Vec<(HashSet, usize)>, /// Whether to expand on next call /// This is used to handle the case where a plan is transformed, but need to be expanded from it's /// parent node. For example a Aggregate plan is split into two parts in frontend and datanode, and need @@ -164,7 +197,7 @@ struct PlanRewriter { /// This is used to handle the case where a plan is transformed, but still /// need to push down as many node as possible before next partial/conditional/transformed commutative /// plan. I.e. - /// ``` + /// ```ignore /// Limit: /// Sort: /// ``` @@ -187,6 +220,15 @@ impl PlanRewriter { /// Return true if should stop and expand. The input plan is the parent node of current node fn should_expand(&mut self, plan: &LogicalPlan) -> bool { + debug!( + "Check should_expand at level: {} with Stack:\n{}, ", + self.level, + self.stack + .iter() + .map(|(p, l)| format!("{l}:{}{}", " ".repeat(l - 1), p.display())) + .collect::>() + .join("\n"), + ); if DFLogicalSubstraitConvertor .encode(plan, DefaultSerializer) .is_err() @@ -200,18 +242,21 @@ impl PlanRewriter { } if self.expand_on_next_part_cond_trans_commutative { - let comm = Categorizer::check_plan(plan, self.partition_cols.clone()); + let comm = Categorizer::check_plan(plan, self.get_aliased_partition_columns()); match comm { Commutativity::PartialCommutative => { // a small difference is that for partial commutative, we still need to - // expand on next call(so `Limit` can be pushed down) + // push down it(so `Limit` can be pushed down) + + // notice how limit needed to be expanded as well to make sure query is correct + // i.e. `Limit fetch=10` need to be pushed down to the leaf node self.expand_on_next_part_cond_trans_commutative = false; self.expand_on_next_call = true; } Commutativity::ConditionalCommutative(_) | Commutativity::TransformedCommutative { .. } => { - // for conditional commutative and transformed commutative, we can - // expand now + // again a new node that can be push down, we should just + // do push down now and avoid further expansion self.expand_on_next_part_cond_trans_commutative = false; return true; } @@ -219,11 +264,12 @@ impl PlanRewriter { } } - match Categorizer::check_plan(plan, self.partition_cols.clone()) { + match Categorizer::check_plan(plan, self.get_aliased_partition_columns()) { Commutativity::Commutative => {} Commutativity::PartialCommutative => { if let Some(plan) = partial_commutative_transformer(plan) { - self.update_column_requirements(&plan); + // notice this plan is parent of current node, so `self.level - 1` when updating column requirements + self.update_column_requirements(&plan, self.level - 1); self.expand_on_next_part_cond_trans_commutative = true; self.stage.push(plan) } @@ -232,7 +278,8 @@ impl PlanRewriter { if let Some(transformer) = transformer && let Some(plan) = transformer(plan) { - self.update_column_requirements(&plan); + // notice this plan is parent of current node, so `self.level - 1` when updating column requirements + self.update_column_requirements(&plan, self.level - 1); self.expand_on_next_part_cond_trans_commutative = true; self.stage.push(plan) } @@ -242,12 +289,22 @@ impl PlanRewriter { && let Some(transformer_actions) = transformer(plan) { debug!( - "PlanRewriter: transformed plan: {:?}\n from {plan}", - transformer_actions.extra_parent_plans + "PlanRewriter: transformed plan: {}\n from {plan}", + transformer_actions + .extra_parent_plans + .iter() + .enumerate() + .map(|(i, p)| format!( + "Extra {i}-th parent plan from parent to child = {}", + p.display() + )) + .collect::>() + .join("\n") ); if let Some(last_stage) = transformer_actions.extra_parent_plans.last() { // update the column requirements from the last stage - self.update_column_requirements(last_stage); + // notice current plan's parent plan is where we need to apply the column requirements + self.update_column_requirements(last_stage, self.level - 1); } self.stage .extend(transformer_actions.extra_parent_plans.into_iter().rev()); @@ -265,9 +322,12 @@ impl PlanRewriter { false } - fn update_column_requirements(&mut self, plan: &LogicalPlan) { + /// Update the column requirements for the current plan, plan_level is the level of the plan + /// in the stack, which is used to determine if the column requirements are applicable + /// for other plans in the stack. + fn update_column_requirements(&mut self, plan: &LogicalPlan, plan_level: usize) { debug!( - "PlanRewriter: update column requirements for plan: {plan}\n withcolumn_requirements: {:?}", + "PlanRewriter: update column requirements for plan: {plan}\n with old column_requirements: {:?}", self.column_requirements ); let mut container = HashSet::new(); @@ -276,9 +336,7 @@ impl PlanRewriter { let _ = expr_to_columns(&expr, &mut container); } - for col in container { - self.column_requirements.insert(col); - } + self.column_requirements.push((container, plan_level)); debug!( "PlanRewriter: updated column requirements: {:?}", self.column_requirements @@ -297,6 +355,45 @@ impl PlanRewriter { self.status = RewriterStatus::Unexpanded; } + /// Maybe update alias for original table columns in the plan + fn maybe_update_alias(&mut self, node: &LogicalPlan) { + if let Some(alias_tracker) = &mut self.alias_tracker { + alias_tracker.update_alias(node); + debug!( + "Current partition columns are: {:?}", + self.get_aliased_partition_columns() + ); + } else if let LogicalPlan::TableScan(table_scan) = node { + self.alias_tracker = AliasTracker::new(table_scan); + debug!( + "Initialize partition columns: {:?} with table={}", + self.get_aliased_partition_columns(), + table_scan.table_name + ); + } + } + + fn get_aliased_partition_columns(&self) -> Option { + if let Some(part_cols) = self.partition_cols.as_ref() { + let Some(alias_tracker) = &self.alias_tracker else { + // no alias tracker meaning no table scan encountered + return None; + }; + let mut aliased = HashMap::new(); + for part_col in part_cols { + let all_alias = alias_tracker + .get_all_alias_for_col(part_col) + .cloned() + .unwrap_or_default(); + + aliased.insert(part_col.clone(), all_alias); + } + Some(aliased) + } else { + None + } + } + fn maybe_set_partitions(&mut self, plan: &LogicalPlan) { if self.partition_cols.is_some() { // only need to set once @@ -342,10 +439,15 @@ impl PlanRewriter { } // store schema before expand let schema = on_node.schema().clone(); - let mut rewriter = EnforceDistRequirementRewriter { - column_requirements: std::mem::take(&mut self.column_requirements), - }; + let mut rewriter = EnforceDistRequirementRewriter::new( + std::mem::take(&mut self.column_requirements), + self.level, + ); + debug!("PlanRewriter: enforce column requirements for node: {on_node} with rewriter: {rewriter:?}"); on_node = on_node.rewrite(&mut rewriter)?.data; + debug!( + "PlanRewriter: after enforced column requirements for node: {on_node} with rewriter: {rewriter:?}" + ); // add merge scan as the new root let mut node = MergeScanLogicalPlan::new( @@ -364,7 +466,8 @@ impl PlanRewriter { } self.set_expanded(); - // recover the schema + // recover the schema, this make sure after expand the schema is the same as old node + // because after expand the raw top node might have extra columns i.e. sorting columns for `Sort` node let node = LogicalPlanBuilder::from(node) .project(schema.iter().map(|(qualifier, field)| { Expr::Column(Column::new(qualifier.cloned(), field.name())) @@ -381,42 +484,96 @@ impl PlanRewriter { /// Requirements enforced by this rewriter: /// - Enforce column requirements for `LogicalPlan::Projection` nodes. Makes sure the /// required columns are available in the sub plan. +/// +#[derive(Debug)] struct EnforceDistRequirementRewriter { - column_requirements: HashSet, + /// only enforce column requirements after the expanding node in question, + /// meaning only for node with `cur_level` <= `level` will consider adding those column requirements + /// TODO(discord9): a simpler solution to track column requirements for merge scan + column_requirements: Vec<(HashSet, usize)>, + /// only apply column requirements >= `cur_level` + /// this is used to avoid applying column requirements that are not needed + /// for the current node, i.e. the node is not in the scope of the column requirements + /// i.e, for this plan: + /// ```ignore + /// Aggregate: min(t.number) + /// Projection: t.number + /// ``` + /// when on `Projection` node, we don't need to apply the column requirements of `Aggregate` node + /// because the `Projection` node is not in the scope of the `Aggregate` node + cur_level: usize, +} + +impl EnforceDistRequirementRewriter { + fn new(column_requirements: Vec<(HashSet, usize)>, cur_level: usize) -> Self { + Self { + column_requirements, + cur_level, + } + } } impl TreeNodeRewriter for EnforceDistRequirementRewriter { type Node = LogicalPlan; fn f_down(&mut self, node: Self::Node) -> DfResult> { - if let LogicalPlan::Projection(ref projection) = node { - let mut column_requirements = std::mem::take(&mut self.column_requirements); - if column_requirements.is_empty() { - return Ok(Transformed::no(node)); - } - - for expr in &projection.expr { - let (qualifier, name) = expr.qualified_name(); - let column = Column::new(qualifier, name); - column_requirements.remove(&column); - } - if column_requirements.is_empty() { - return Ok(Transformed::no(node)); - } - - let mut new_exprs = projection.expr.clone(); - for col in &column_requirements { - new_exprs.push(Expr::Column(col.clone())); - } - let new_node = - node.with_new_exprs(new_exprs, node.inputs().into_iter().cloned().collect())?; - return Ok(Transformed::yes(new_node)); + // check that node doesn't have multiple children, i.e. join/subquery + if node.inputs().len() > 1 { + return Err(datafusion_common::DataFusionError::Internal( + "EnforceDistRequirementRewriter: node with multiple inputs is not supported" + .to_string(), + )); } - + self.cur_level += 1; Ok(Transformed::no(node)) } fn f_up(&mut self, node: Self::Node) -> DfResult> { + self.cur_level -= 1; + // first get all applicable column requirements + let mut applicable_column_requirements = self + .column_requirements + .iter() + .filter(|(_, level)| *level >= self.cur_level) + .map(|(cols, _)| cols.clone()) + .reduce(|mut acc, cols| { + acc.extend(cols); + acc + }) + .unwrap_or_default(); + + debug!( + "EnforceDistRequirementRewriter: applicable column requirements at level {} = {:?} for node {}", + self.cur_level, + applicable_column_requirements, + node.display() + ); + + // make sure all projection applicable scope has the required columns + if let LogicalPlan::Projection(ref projection) = node { + for expr in &projection.expr { + let (qualifier, name) = expr.qualified_name(); + let column = Column::new(qualifier, name); + applicable_column_requirements.remove(&column); + } + if applicable_column_requirements.is_empty() { + return Ok(Transformed::no(node)); + } + + let mut new_exprs = projection.expr.clone(); + for col in &applicable_column_requirements { + new_exprs.push(Expr::Column(col.clone())); + } + let new_node = + node.with_new_exprs(new_exprs, node.inputs().into_iter().cloned().collect())?; + debug!( + "EnforceDistRequirementRewriter: added missing columns {:?} to projection node from old node: \n{node}\n Making new node: \n{new_node}", + applicable_column_requirements + ); + + // still need to continue for next projection if applicable + return Ok(Transformed::yes(new_node)); + } Ok(Transformed::no(node)) } } @@ -432,6 +589,7 @@ impl TreeNodeRewriter for PlanRewriter { self.stage.clear(); self.set_unexpanded(); self.partition_cols = None; + self.alias_tracker = None; Ok(Transformed::no(node)) } @@ -454,8 +612,19 @@ impl TreeNodeRewriter for PlanRewriter { self.maybe_set_partitions(&node); + self.maybe_update_alias(&node); + let Some(parent) = self.get_parent() else { - let node = self.expand(node)?; + debug!("Plan Rewriter: expand now for no parent found for node: {node}"); + let node = self.expand(node); + debug!( + "PlanRewriter: expanded plan: {}", + match &node { + Ok(n) => n.to_string(), + Err(e) => format!("Error expanding plan: {e}"), + } + ); + let node = node?; self.pop_stack(); return Ok(Transformed::yes(node)); }; @@ -483,160 +652,3 @@ impl TreeNodeRewriter for PlanRewriter { Ok(Transformed::no(node)) } } - -#[cfg(test)] -mod test { - use std::sync::Arc; - - use datafusion::datasource::DefaultTableSource; - use datafusion::functions_aggregate::expr_fn::avg; - use datafusion_common::JoinType; - use datafusion_expr::{col, lit, Expr, LogicalPlanBuilder}; - use table::table::adapter::DfTableProviderAdapter; - use table::table::numbers::NumbersTable; - - use super::*; - - #[ignore = "Projection is disabled for https://github.com/apache/arrow-datafusion/issues/6489"] - #[test] - fn transform_simple_projection_filter() { - let numbers_table = NumbersTable::table(0); - let table_source = Arc::new(DefaultTableSource::new(Arc::new( - DfTableProviderAdapter::new(numbers_table), - ))); - - let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) - .unwrap() - .filter(col("number").lt(lit(10))) - .unwrap() - .project(vec![col("number")]) - .unwrap() - .distinct() - .unwrap() - .build() - .unwrap(); - - let config = ConfigOptions::default(); - let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); - let expected = [ - "Distinct:", - " MergeScan [is_placeholder=false]", - " Distinct:", - " Projection: t.number", - " Filter: t.number < Int32(10)", - " TableScan: t", - ] - .join("\n"); - assert_eq!(expected, result.to_string()); - } - - #[test] - fn transform_aggregator() { - let numbers_table = NumbersTable::table(0); - let table_source = Arc::new(DefaultTableSource::new(Arc::new( - DfTableProviderAdapter::new(numbers_table), - ))); - - let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) - .unwrap() - .aggregate(Vec::::new(), vec![avg(col("number"))]) - .unwrap() - .build() - .unwrap(); - - let config = ConfigOptions::default(); - let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); - let expected = "Projection: avg(t.number)\ - \n MergeScan [is_placeholder=false]"; - assert_eq!(expected, result.to_string()); - } - - #[test] - fn transform_distinct_order() { - let numbers_table = NumbersTable::table(0); - let table_source = Arc::new(DefaultTableSource::new(Arc::new( - DfTableProviderAdapter::new(numbers_table), - ))); - - let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) - .unwrap() - .distinct() - .unwrap() - .sort(vec![col("number").sort(true, false)]) - .unwrap() - .build() - .unwrap(); - - let config = ConfigOptions::default(); - let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); - let expected = ["Projection: t.number", " MergeScan [is_placeholder=false]"].join("\n"); - assert_eq!(expected, result.to_string()); - } - - #[test] - fn transform_single_limit() { - let numbers_table = NumbersTable::table(0); - let table_source = Arc::new(DefaultTableSource::new(Arc::new( - DfTableProviderAdapter::new(numbers_table), - ))); - - let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) - .unwrap() - .limit(0, Some(1)) - .unwrap() - .build() - .unwrap(); - - let config = ConfigOptions::default(); - let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); - let expected = "Projection: t.number\ - \n MergeScan [is_placeholder=false]"; - assert_eq!(expected, result.to_string()); - } - - #[test] - fn transform_unalighed_join_with_alias() { - let left = NumbersTable::table(0); - let right = NumbersTable::table(1); - let left_source = Arc::new(DefaultTableSource::new(Arc::new( - DfTableProviderAdapter::new(left), - ))); - let right_source = Arc::new(DefaultTableSource::new(Arc::new( - DfTableProviderAdapter::new(right), - ))); - - let right_plan = LogicalPlanBuilder::scan_with_filters("t", right_source, None, vec![]) - .unwrap() - .alias("right") - .unwrap() - .build() - .unwrap(); - - let plan = LogicalPlanBuilder::scan_with_filters("t", left_source, None, vec![]) - .unwrap() - .join_on( - right_plan, - JoinType::LeftSemi, - vec![col("t.number").eq(col("right.number"))], - ) - .unwrap() - .limit(0, Some(1)) - .unwrap() - .build() - .unwrap(); - - let config = ConfigOptions::default(); - let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); - let expected = [ - "Limit: skip=0, fetch=1", - " LeftSemi Join: Filter: t.number = right.number", - " Projection: t.number", - " MergeScan [is_placeholder=false]", - " SubqueryAlias: right", - " Projection: t.number", - " MergeScan [is_placeholder=false]", - ] - .join("\n"); - assert_eq!(expected, result.to_string()); - } -} diff --git a/src/query/src/dist_plan/analyzer/test.rs b/src/query/src/dist_plan/analyzer/test.rs new file mode 100644 index 0000000000..863f9fd3b4 --- /dev/null +++ b/src/query/src/dist_plan/analyzer/test.rs @@ -0,0 +1,1309 @@ +// 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::pin::Pin; +use std::sync::Arc; + +use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; +use common_error::ext::BoxedError; +use common_recordbatch::adapter::RecordBatchMetrics; +use common_recordbatch::error::Result as RecordBatchResult; +use common_recordbatch::{OrderOption, RecordBatch, RecordBatchStream, SendableRecordBatchStream}; +use common_telemetry::init_default_ut_logging; +use datafusion::datasource::DefaultTableSource; +use datafusion::functions_aggregate::expr_fn::avg; +use datafusion::functions_aggregate::min_max::{max, min}; +use datafusion_common::JoinType; +use datafusion_expr::{col, lit, Expr, LogicalPlanBuilder}; +use datafusion_sql::TableReference; +use datatypes::data_type::ConcreteDataType; +use datatypes::schema::{ColumnSchema, SchemaBuilder, SchemaRef}; +use futures::task::{Context, Poll}; +use futures::Stream; +use pretty_assertions::assert_eq; +use store_api::data_source::DataSource; +use store_api::storage::ScanRequest; +use table::metadata::{ + FilterPushDownType, TableId, TableInfoBuilder, TableInfoRef, TableMeta, TableType, +}; +use table::table::adapter::DfTableProviderAdapter; +use table::table::numbers::NumbersTable; +use table::{Table, TableRef}; + +use super::*; + +pub(crate) struct TestTable; + +impl TestTable { + pub fn table_with_name(table_id: TableId, name: String) -> TableRef { + let data_source = Arc::new(TestDataSource::new(Self::schema())); + let table = Table::new( + Self::table_info(table_id, name, "test_engine".to_string()), + FilterPushDownType::Unsupported, + data_source, + ); + Arc::new(table) + } + + pub fn schema() -> SchemaRef { + let column_schemas = vec![ + ColumnSchema::new("pk1", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("pk2", ConcreteDataType::string_datatype(), false), + ColumnSchema::new("pk3", ConcreteDataType::string_datatype(), false), + ColumnSchema::new( + "ts", + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ) + .with_time_index(true), + ColumnSchema::new("number", ConcreteDataType::uint32_datatype(), true), + ]; + let schema = SchemaBuilder::try_from_columns(column_schemas) + .unwrap() + .build() + .unwrap(); + Arc::new(schema) + } + + pub fn table_info(table_id: TableId, name: String, engine: String) -> TableInfoRef { + let table_meta = TableMeta { + schema: Self::schema(), + primary_key_indices: vec![0, 1, 2], + value_indices: vec![4], + engine, + region_numbers: vec![0, 1], + next_column_id: 5, + options: Default::default(), + created_on: Default::default(), + partition_key_indices: vec![0, 1], + }; + + let table_info = TableInfoBuilder::default() + .table_id(table_id) + .name(name) + .catalog_name(DEFAULT_CATALOG_NAME) + .schema_name(DEFAULT_SCHEMA_NAME) + .table_version(0) + .table_type(TableType::Base) + .meta(table_meta) + .build() + .unwrap(); + Arc::new(table_info) + } +} + +struct TestDataSource { + schema: SchemaRef, +} + +impl TestDataSource { + pub fn new(schema: SchemaRef) -> Self { + Self { schema } + } +} + +impl DataSource for TestDataSource { + fn get_stream(&self, request: ScanRequest) -> Result { + let projected_schema = match &request.projection { + Some(projection) => Arc::new(self.schema.try_project(projection).unwrap()), + None => self.schema.clone(), + }; + Ok(Box::pin(EmptyStream { + schema: projected_schema, + })) + } +} + +struct EmptyStream { + schema: SchemaRef, +} + +impl RecordBatchStream for EmptyStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn output_ordering(&self) -> Option<&[OrderOption]> { + None + } + + fn metrics(&self) -> Option { + None + } +} + +impl Stream for EmptyStream { + type Item = RecordBatchResult; + + fn poll_next(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll> { + Poll::Ready(None) + } +} + +#[test] +fn expand_proj_sort_proj() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![col("number"), col("pk1"), col("pk2"), col("pk3")]) + .unwrap() + .project(vec![ + col("number"), + col("pk1"), + col("pk3"), + col("pk1").eq(col("pk2")), + ]) + .unwrap() + .sort(vec![col("t.pk1 = t.pk2").sort(true, true)]) + .unwrap() + .project(vec![col("number")]) + .unwrap() + .project(vec![col("number")]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: t.number", + " MergeSort: t.pk1 = t.pk2 ASC NULLS FIRST", + " MergeScan [is_placeholder=false, remote_input=[", + "Projection: t.number, t.pk1 = t.pk2", + " Projection: t.number, t.pk1 = t.pk2", // notice both projections added `t.pk1 = t.pk2` column requirement + " Sort: t.pk1 = t.pk2 ASC NULLS FIRST", + " Projection: t.number, t.pk1, t.pk3, t.pk1 = t.pk2", + " Projection: t.number, t.pk1, t.pk2, t.pk3", // notice this projection doesn't add `t.pk1 = t.pk2` column requirement + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_sort_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: t.pk1, t.pk2, t.pk3, t.ts, t.number", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_sort_alias_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .project(vec![col("pk1").alias("something")]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: something", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.pk1 AS something, t.pk1", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// FIXME(discord9): alias to same name with col req makes it ambiguous +#[should_panic(expected = "AmbiguousReference")] +#[test] +fn expand_sort_alias_conflict_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .project(vec![col("pk2").alias("pk1")]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: something", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.pk2 AS pk1, t.pk1", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_sort_alias_conflict_but_not_really_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .project(vec![col("pk2").alias("t.pk1")]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: t.pk1", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.pk2 AS t.pk1, t.pk1", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// TODO(discord9): it is possible to expand `Sort` and `Limit` in the same step, +/// but it's too complicated to implement now, and probably not worth it since `Limit` already +/// greatly reduces the amount of data to sort. +#[test] +fn expand_limit_sort() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Sort: t.pk1 ASC NULLS LAST", + " Projection: t.pk1, t.pk2, t.pk3, t.ts, t.number", + " Limit: skip=0, fetch=10", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_sort_limit_sort() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Sort: t.pk1 ASC NULLS LAST", + " Projection: t.pk1, t.pk2, t.pk3, t.ts, t.number", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// test plan like: +/// ``` +/// Aggregate: min(t.number) +/// Projection: t.number +/// ``` +/// which means aggr introduce new column requirements that shouldn't be updated in lower projection +/// +/// this help test expand need actually add new column requirements +/// because ``Limit` doesn't introduce new column requirements +/// only `Sort/Aggregate` does, and for now since `aggregate` get expanded immediately, it's col requirements are not used anyway +#[test] +fn expand_proj_step_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![col("number")]) + .unwrap() + .aggregate(Vec::::new(), vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: min(t.number)", + " Projection: min(min(t.number)) AS min(t.number)", + " Aggregate: groupBy=[[]], aggr=[[min(min(t.number))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[]], aggr=[[min(t.number)]]", + " Projection: t.number", // This Projection shouldn't add new column requirements + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// Shouldn't push down the fake partition column aggregate(which is steppable) +/// as the `pk1` is a alias for `pk3` which is not partition column +#[test] +fn expand_proj_alias_fake_part_col_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + col("number"), + col("pk3").alias("pk1"), + col("pk2").alias("pk3"), + ]) + .unwrap() + .project(vec![ + col("number"), + col("pk1").alias("pk2"), + col("pk3").alias("pk1"), + ]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: pk1, pk2, min(t.number)", + " Projection: pk1, pk2, min(min(t.number)) AS min(t.number)", + " Aggregate: groupBy=[[pk1, pk2]], aggr=[[min(min(t.number))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[pk1, pk2]], aggr=[[min(t.number)]]", + " Projection: t.number, pk1 AS pk2, pk3 AS pk1", + " Projection: t.number, t.pk3 AS pk1, t.pk2 AS pk3", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_alias_aliased_part_col_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + col("number"), + col("pk1").alias("pk3"), + col("pk2").alias("pk4"), + ]) + .unwrap() + .project(vec![ + col("number"), + col("pk3").alias("pk42"), + col("pk4").alias("pk43"), + ]) + .unwrap() + .aggregate(vec![col("pk42"), col("pk43")], vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: pk42, pk43, min(t.number)", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[pk42, pk43]], aggr=[[min(t.number)]]", + " Projection: t.number, pk3 AS pk42, pk4 AS pk43", + " Projection: t.number, t.pk1 AS pk3, t.pk2 AS pk4", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// notice that step aggr then part col aggr seems impossible as the partition columns for part col aggr +/// can't pass through the step aggr without making step aggr also a part col aggr +/// so here only test part col aggr -> step aggr case +#[test] +fn expand_part_col_aggr_step_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![max(col("number"))]) + .unwrap() + .aggregate(Vec::::new(), vec![min(col("max(t.number)"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: min(max(t.number))", + " Projection: min(min(max(t.number))) AS min(max(t.number))", + " Aggregate: groupBy=[[]], aggr=[[min(min(max(t.number)))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[]], aggr=[[min(max(t.number))]]", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_step_aggr_step_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(Vec::::new(), vec![max(col("number"))]) + .unwrap() + .aggregate(Vec::::new(), vec![min(col("max(t.number)"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Aggregate: groupBy=[[]], aggr=[[min(max(t.number))]]", + " Projection: max(t.number)", + " Projection: max(max(t.number)) AS max(t.number)", + " Aggregate: groupBy=[[]], aggr=[[max(max(t.number))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[]], aggr=[[max(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_part_col_aggr_part_col_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![max(col("number"))]) + .unwrap() + .aggregate( + vec![col("pk1"), col("pk2")], + vec![min(col("max(t.number)"))], + ) + .unwrap() + .build() + .unwrap(); + + let expected_original = [ + "Aggregate: groupBy=[[t.pk1, t.pk2, max(t.number)]], aggr=[[min(max(t.number))]]", // notice here `max(t.number)` is added to groupBy due to aggr exprs depend on this column + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]", + " TableScan: t", + ] + .join("\n"); + assert_eq!(expected_original, plan.to_string()); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: t.pk1, t.pk2, max(t.number), min(max(t.number))", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[t.pk1, t.pk2, max(t.number)]], aggr=[[min(max(t.number))]]", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_step_aggr_proj() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![col("pk1")], vec![min(col("number"))]) + .unwrap() + .project(vec![col("min(t.number)")]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + "Projection: min(t.number)", + " Projection: t.pk1, min(t.number)", + " Projection: t.pk1, min(min(t.number)) AS min(t.number)", + " Aggregate: groupBy=[[t.pk1]], aggr=[[min(min(t.number))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[t.pk1]], aggr=[[min(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// should only expand `Sort`, notice `Sort` before `Aggregate` usually can and +/// will be optimized out, and dist planner shouldn't handle that case, but +/// for now, still handle that be expanding the `Sort` node +#[test] +fn expand_proj_sort_step_aggr_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .project(vec![Expr::Column(Column::new( + Some(TableReference::bare("t")), + "number", + ))]) + .unwrap() + .aggregate(Vec::::new(), vec![min(col("number"))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Limit: skip=0, fetch=10", + " Aggregate: groupBy=[[]], aggr=[[min(t.number)]]", + " Projection: t.number", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Projection: t.number, t.pk1", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_sort_limit_step_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk1").sort(true, false)]) + .unwrap() + .project(vec![Expr::Column(Column::new( + Some(TableReference::bare("t")), + "number", + ))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .aggregate(Vec::::new(), vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Aggregate: groupBy=[[]], aggr=[[min(t.number)]]", + " Projection: t.number", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk1 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.number, t.pk1", + " Sort: t.pk1 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_limit_step_aggr_sort() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![Expr::Column(Column::new( + Some(TableReference::bare("t")), + "number", + ))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .aggregate(Vec::::new(), vec![min(col("number"))]) + .unwrap() + .sort(vec![col("min(t.number)").sort(true, false)]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Sort: min(t.number) ASC NULLS LAST", + " Aggregate: groupBy=[[]], aggr=[[min(t.number)]]", + " Projection: t.number", + " Limit: skip=0, fetch=10", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.number", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_sort_part_col_aggr_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk3").sort(true, false)]) + .unwrap() + .project(vec![ + Expr::Column(Column::new(Some(TableReference::bare("t")), "number")), + col("pk1"), + col("pk2"), + ]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Limit: skip=0, fetch=10", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " Projection: t.number, t.pk1, t.pk2", + " MergeSort: t.pk3 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Projection: t.number, t.pk1, t.pk2, t.pk3", + " Sort: t.pk3 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_sort_limit_part_col_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .sort(vec![col("pk3").sort(true, false)]) + .unwrap() + .project(vec![ + Expr::Column(Column::new(Some(TableReference::bare("t")), "number")), + col("pk1"), + col("pk2"), + ]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " Projection: t.number, t.pk1, t.pk2", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk3 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.number, t.pk1, t.pk2, t.pk3", + " Sort: t.pk3 ASC NULLS LAST", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} +#[test] +fn expand_proj_part_col_aggr_limit_sort() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + Expr::Column(Column::new(Some(TableReference::bare("t")), "number")), + col("pk1"), + col("pk2"), + ]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .sort(vec![col("pk2").sort(true, false)]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Sort: t.pk2 ASC NULLS LAST", + " Projection: t.pk1, t.pk2, min(t.number)", + " Limit: skip=0, fetch=10", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " Projection: t.number, t.pk1, t.pk2", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_part_col_aggr_sort_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + Expr::Column(Column::new(Some(TableReference::bare("t")), "number")), + col("pk1"), + col("pk2"), + ]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .sort(vec![col("pk2").sort(true, false)]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Projection: t.pk1, t.pk2, min(t.number)", + " Limit: skip=0, fetch=10", + " MergeSort: t.pk2 ASC NULLS LAST", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Sort: t.pk2 ASC NULLS LAST", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " Projection: t.number, t.pk1, t.pk2", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_limit_part_col_aggr_sort() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + Expr::Column(Column::new(Some(TableReference::bare("t")), "number")), + col("pk1"), + col("pk2"), + ]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .sort(vec![col("pk2").sort(true, false)]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Sort: t.pk2 ASC NULLS LAST", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " Projection: t.number, t.pk1, t.pk2", + " Limit: skip=0, fetch=10", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.number, t.pk1, t.pk2", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn expand_proj_limit_sort_part_col_aggr() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + Expr::Column(Column::new(Some(TableReference::bare("t")), "number")), + col("pk1"), + col("pk2"), + ]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .sort(vec![col("pk2").sort(true, false)]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " Sort: t.pk2 ASC NULLS LAST", + " Projection: t.number, t.pk1, t.pk2", + " Limit: skip=0, fetch=10", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Projection: t.number, t.pk1, t.pk2", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// Notice how this limit can't be push down, or results will be wrong +#[test] +fn expand_step_aggr_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![col("pk1")], vec![min(col("number"))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Limit: skip=0, fetch=10", + " Projection: t.pk1, min(t.number)", + " Projection: t.pk1, min(min(t.number)) AS min(t.number)", + " Aggregate: groupBy=[[t.pk1]], aggr=[[min(min(t.number))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[t.pk1]], aggr=[[min(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// notice how `Limit` can still get expanded +#[test] +fn expand_part_col_aggr_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![col("pk1"), col("pk2")], vec![min(col("number"))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Projection: t.pk1, t.pk2, min(t.number)", + " Limit: skip=0, fetch=10", + " MergeScan [is_placeholder=false, remote_input=[", + "Limit: skip=0, fetch=10", + " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[ignore = "Projection is disabled for https://github.com/apache/arrow-datafusion/issues/6489"] +#[test] +fn transform_simple_projection_filter() { + let numbers_table = NumbersTable::table(0); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(numbers_table), + ))); + + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .filter(col("number").lt(lit(10))) + .unwrap() + .project(vec![col("number")]) + .unwrap() + .distinct() + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Distinct:", + " MergeScan [is_placeholder=false]", + " Distinct:", + " Projection: t.number", + " Filter: t.number < Int32(10)", + " TableScan: t", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn transform_aggregator() { + let numbers_table = NumbersTable::table(0); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(numbers_table), + ))); + + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(Vec::::new(), vec![avg(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = "Projection: avg(t.number)\ + \n MergeScan [is_placeholder=false, remote_input=[\ + \nAggregate: groupBy=[[]], aggr=[[avg(t.number)]]\ + \n TableScan: t\ + \n]]"; + assert_eq!(expected, result.to_string()); +} + +#[test] +fn transform_distinct_order() { + let numbers_table = NumbersTable::table(0); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(numbers_table), + ))); + + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .distinct() + .unwrap() + .sort(vec![col("number").sort(true, false)]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Projection: t.number", + " MergeScan [is_placeholder=false, remote_input=[ +Sort: t.number ASC NULLS LAST + Distinct: + TableScan: t +]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +#[test] +fn transform_single_limit() { + let numbers_table = NumbersTable::table(0); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(numbers_table), + ))); + + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .limit(0, Some(1)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = "Projection: t.number\ + \n MergeScan [is_placeholder=false, remote_input=[ +Limit: skip=0, fetch=1 + TableScan: t +]]"; + assert_eq!(expected, result.to_string()); +} + +#[test] +fn transform_unalighed_join_with_alias() { + let left = NumbersTable::table(0); + let right = NumbersTable::table(1); + let left_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(left), + ))); + let right_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(right), + ))); + + let right_plan = LogicalPlanBuilder::scan_with_filters("t", right_source, None, vec![]) + .unwrap() + .alias("right") + .unwrap() + .build() + .unwrap(); + + let plan = LogicalPlanBuilder::scan_with_filters("t", left_source, None, vec![]) + .unwrap() + .join_on( + right_plan, + JoinType::LeftSemi, + vec![col("t.number").eq(col("right.number"))], + ) + .unwrap() + .limit(0, Some(1)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Limit: skip=0, fetch=1", + " LeftSemi Join: Filter: t.number = right.number", + " Projection: t.number", + " MergeScan [is_placeholder=false, remote_input=[", + "TableScan: t", + "]]", + " SubqueryAlias: right", + " Projection: t.number", + " MergeScan [is_placeholder=false, remote_input=[", + "TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} diff --git a/src/query/src/dist_plan/analyzer/utils.rs b/src/query/src/dist_plan/analyzer/utils.rs new file mode 100644 index 0000000000..83b617f7a1 --- /dev/null +++ b/src/query/src/dist_plan/analyzer/utils.rs @@ -0,0 +1,318 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; + +use datafusion::datasource::DefaultTableSource; +use datafusion_common::Column; +use datafusion_expr::{Expr, LogicalPlan, TableScan}; +use table::metadata::TableType; +use table::table::adapter::DfTableProviderAdapter; + +/// Mapping of original column in table to all the alias at current node +pub type AliasMapping = HashMap>; + +/// tracking aliases for the source table columns in the plan +#[derive(Debug, Clone)] +pub struct AliasTracker { + /// mapping from the original table name to the alias used in the plan + /// notice how one column might have multiple aliases in the plan + /// + pub mapping: AliasMapping, +} + +impl AliasTracker { + pub fn new(table_scan: &TableScan) -> Option { + if let Some(source) = table_scan + .source + .as_any() + .downcast_ref::() + { + if let Some(provider) = source + .table_provider + .as_any() + .downcast_ref::() + { + if provider.table().table_type() == TableType::Base { + let info = provider.table().table_info(); + let schema = info.meta.schema.clone(); + let col_schema = schema.column_schemas(); + let mapping = col_schema + .iter() + .map(|col| { + ( + col.name.clone(), + HashSet::from_iter(std::iter::once(Column::new_unqualified( + col.name.clone(), + ))), + ) + }) + .collect(); + return Some(Self { mapping }); + } + } + } + + None + } + + /// update alias for original columns + /// + /// only handle `Alias` with column in `Projection` node + pub fn update_alias(&mut self, node: &LogicalPlan) { + if let LogicalPlan::Projection(projection) = node { + // first collect all the alias mapping, i.e. the col_a AS b AS c AS d become `a->d` + // notice one column might have multiple aliases + let mut alias_mapping: AliasMapping = HashMap::new(); + for expr in &projection.expr { + if let Expr::Alias(alias) = expr { + let outer_alias = alias.clone(); + let mut cur_alias = alias.clone(); + while let Expr::Alias(alias) = *cur_alias.expr { + cur_alias = alias; + } + if let Expr::Column(column) = *cur_alias.expr { + alias_mapping + .entry(column.name.clone()) + .or_default() + .insert(Column::new(outer_alias.relation, outer_alias.name)); + } + } else if let Expr::Column(column) = expr { + // identity mapping + alias_mapping + .entry(column.name.clone()) + .or_default() + .insert(column.clone()); + } + } + + // update mapping using `alias_mapping` + let mut new_mapping = HashMap::new(); + for (table_col_name, cur_columns) in std::mem::take(&mut self.mapping) { + let new_aliases = { + let mut new_aliases = HashSet::new(); + for cur_column in &cur_columns { + let new_alias_for_cur_column = alias_mapping + .get(cur_column.name()) + .cloned() + .unwrap_or_default(); + + for new_alias in new_alias_for_cur_column { + let is_table_ref_eq = match (&new_alias.relation, &cur_column.relation) + { + (Some(o), Some(c)) => o.resolved_eq(c), + _ => true, + }; + // is the same column if both name and table ref is eq + if is_table_ref_eq { + new_aliases.insert(new_alias.clone()); + } + } + } + new_aliases + }; + + new_mapping.insert(table_col_name, new_aliases); + } + + self.mapping = new_mapping; + common_telemetry::debug!( + "Updating alias tracker to {:?} using node: \n{node}", + self.mapping + ); + } + } + + pub fn get_all_alias_for_col(&self, col_name: &str) -> Option<&HashSet> { + self.mapping.get(col_name) + } + + #[allow(unused)] + pub fn is_alias_for(&self, original_col: &str, cur_col: &Column) -> bool { + self.mapping + .get(original_col) + .map(|cols| cols.contains(cur_col)) + .unwrap_or(false) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use common_telemetry::init_default_ut_logging; + use datafusion::error::Result as DfResult; + use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; + use datafusion_expr::{col, LogicalPlanBuilder}; + + use super::*; + use crate::dist_plan::analyzer::test::TestTable; + + #[derive(Debug)] + struct TrackerTester { + alias_tracker: Option, + mapping_at_each_level: Vec, + } + + impl TreeNodeVisitor<'_> for TrackerTester { + type Node = LogicalPlan; + + fn f_up(&mut self, node: &LogicalPlan) -> DfResult { + if let Some(alias_tracker) = &mut self.alias_tracker { + alias_tracker.update_alias(node); + self.mapping_at_each_level.push( + self.alias_tracker + .as_ref() + .map(|a| a.mapping.clone()) + .unwrap_or_default() + .clone(), + ); + } else if let LogicalPlan::TableScan(table_scan) = node { + self.alias_tracker = AliasTracker::new(table_scan); + self.mapping_at_each_level.push( + self.alias_tracker + .as_ref() + .map(|a| a.mapping.clone()) + .unwrap_or_default() + .clone(), + ); + } + Ok(TreeNodeRecursion::Continue) + } + } + + #[test] + fn proj_alias_tracker() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + col("number"), + col("pk3").alias("pk1"), + col("pk2").alias("pk3"), + ]) + .unwrap() + .project(vec![ + col("number"), + col("pk1").alias("pk2"), + col("pk3").alias("pk1"), + ]) + .unwrap() + .build() + .unwrap(); + + let mut tracker_tester = TrackerTester { + alias_tracker: None, + mapping_at_each_level: Vec::new(), + }; + plan.visit(&mut tracker_tester).unwrap(); + + assert_eq!( + tracker_tester.mapping_at_each_level, + vec![ + HashMap::from([ + ("number".to_string(), HashSet::from(["number".into()])), + ("pk1".to_string(), HashSet::from(["pk1".into()])), + ("pk2".to_string(), HashSet::from(["pk2".into()])), + ("pk3".to_string(), HashSet::from(["pk3".into()])), + ("ts".to_string(), HashSet::from(["ts".into()])) + ]), + HashMap::from([ + ("number".to_string(), HashSet::from(["t.number".into()])), + ("pk1".to_string(), HashSet::from([])), + ("pk2".to_string(), HashSet::from(["pk3".into()])), + ("pk3".to_string(), HashSet::from(["pk1".into()])), + ("ts".to_string(), HashSet::from([])) + ]), + HashMap::from([ + ("number".to_string(), HashSet::from(["t.number".into()])), + ("pk1".to_string(), HashSet::from([])), + ("pk2".to_string(), HashSet::from(["pk1".into()])), + ("pk3".to_string(), HashSet::from(["pk2".into()])), + ("ts".to_string(), HashSet::from([])) + ]) + ] + ); + } + + #[test] + fn proj_multi_alias_tracker() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .project(vec![ + col("number"), + col("pk3").alias("pk1"), + col("pk3").alias("pk2"), + ]) + .unwrap() + .project(vec![ + col("number"), + col("pk2").alias("pk4"), + col("pk1").alias("pk5"), + ]) + .unwrap() + .build() + .unwrap(); + + let mut tracker_tester = TrackerTester { + alias_tracker: None, + mapping_at_each_level: Vec::new(), + }; + plan.visit(&mut tracker_tester).unwrap(); + + assert_eq!( + tracker_tester.mapping_at_each_level, + vec![ + HashMap::from([ + ("number".to_string(), HashSet::from(["number".into()])), + ("pk1".to_string(), HashSet::from(["pk1".into()])), + ("pk2".to_string(), HashSet::from(["pk2".into()])), + ("pk3".to_string(), HashSet::from(["pk3".into()])), + ("ts".to_string(), HashSet::from(["ts".into()])) + ]), + HashMap::from([ + ("number".to_string(), HashSet::from(["t.number".into()])), + ("pk1".to_string(), HashSet::from([])), + ("pk2".to_string(), HashSet::from([])), + ( + "pk3".to_string(), + HashSet::from(["pk1".into(), "pk2".into()]) + ), + ("ts".to_string(), HashSet::from([])) + ]), + HashMap::from([ + ("number".to_string(), HashSet::from(["t.number".into()])), + ("pk1".to_string(), HashSet::from([])), + ("pk2".to_string(), HashSet::from([])), + ( + "pk3".to_string(), + HashSet::from(["pk4".into(), "pk5".into()]) + ), + ("ts".to_string(), HashSet::from([])) + ]) + ] + ); + } +} diff --git a/src/query/src/dist_plan/commutativity.rs b/src/query/src/dist_plan/commutativity.rs index 372db92f2d..e8ece184fc 100644 --- a/src/query/src/dist_plan/commutativity.rs +++ b/src/query/src/dist_plan/commutativity.rs @@ -27,6 +27,7 @@ use promql::extension_plan::{ EmptyMetric, InstantManipulate, RangeManipulate, SeriesDivide, SeriesNormalize, }; +use crate::dist_plan::analyzer::AliasMapping; use crate::dist_plan::merge_sort::{merge_sort_transformer, MergeSortLogicalPlan}; use crate::dist_plan::MergeScanLogicalPlan; @@ -139,9 +140,7 @@ pub fn step_aggr_to_upper_aggr( new_projection_exprs.push(aliased_output_aggr_expr); } let upper_aggr_plan = LogicalPlan::Aggregate(new_aggr); - debug!("Before recompute schema: {upper_aggr_plan:?}"); let upper_aggr_plan = upper_aggr_plan.recompute_schema()?; - debug!("After recompute schema: {upper_aggr_plan:?}"); // create a projection on top of the new aggregate plan let new_projection = Projection::try_new(new_projection_exprs, Arc::new(upper_aggr_plan.clone()))?; @@ -222,7 +221,7 @@ pub enum Commutativity { pub struct Categorizer {} impl Categorizer { - pub fn check_plan(plan: &LogicalPlan, partition_cols: Option>) -> Commutativity { + pub fn check_plan(plan: &LogicalPlan, partition_cols: Option) -> Commutativity { let partition_cols = partition_cols.unwrap_or_default(); match plan { @@ -247,7 +246,6 @@ impl Categorizer { transformer: Some(Arc::new(|plan: &LogicalPlan| { debug!("Before Step optimize: {plan}"); let ret = step_aggr_to_upper_aggr(plan); - debug!("After Step Optimize: {ret:?}"); ret.ok().map(|s| TransformerAction { extra_parent_plans: s.to_vec(), new_child_plan: None, @@ -264,7 +262,11 @@ impl Categorizer { return commutativity; } } - Commutativity::Commutative + // all group by expressions are partition columns can push down, unless + // another push down(including `Limit` or `Sort`) is already in progress(which will then prvent next cond commutative node from being push down). + // TODO(discord9): This is a temporary solution(that works), a better description of + // commutativity is needed under this situation. + Commutativity::ConditionalCommutative(None) } LogicalPlan::Sort(_) => { if partition_cols.is_empty() { @@ -322,17 +324,20 @@ impl Categorizer { pub fn check_extension_plan( plan: &dyn UserDefinedLogicalNode, - partition_cols: &[String], + partition_cols: &AliasMapping, ) -> Commutativity { match plan.name() { name if name == SeriesDivide::name() => { let series_divide = plan.as_any().downcast_ref::().unwrap(); let tags = series_divide.tags().iter().collect::>(); - for partition_col in partition_cols { - if !tags.contains(partition_col) { + + for all_alias in partition_cols.values() { + let all_alias = all_alias.iter().map(|c| &c.name).collect::>(); + if tags.intersection(&all_alias).count() == 0 { return Commutativity::NonCommutative; } } + Commutativity::Commutative } name if name == SeriesNormalize::name() @@ -396,7 +401,7 @@ impl Categorizer { /// Return true if the given expr and partition cols satisfied the rule. /// In this case the plan can be treated as fully commutative. - fn check_partition(exprs: &[Expr], partition_cols: &[String]) -> bool { + fn check_partition(exprs: &[Expr], partition_cols: &AliasMapping) -> bool { let mut ref_cols = HashSet::new(); for expr in exprs { expr.add_column_refs(&mut ref_cols); @@ -405,8 +410,14 @@ impl Categorizer { .into_iter() .map(|c| c.name.clone()) .collect::>(); - for col in partition_cols { - if !ref_cols.contains(col) { + for all_alias in partition_cols.values() { + let all_alias = all_alias + .iter() + .map(|c| c.name.clone()) + .collect::>(); + // check if ref columns intersect with all alias of partition columns + // is empty, if it's empty, not all partition columns show up in `exprs` + if ref_cols.intersection(&all_alias).count() == 0 { return false; } } @@ -424,7 +435,7 @@ pub type StageTransformer = Arc Option std::fmt::Result { - write!(f, "MergeScan [is_placeholder={}]", self.is_placeholder) + write!( + f, + "MergeScan [is_placeholder={}, remote_input=[\n{}\n]]", + self.is_placeholder, self.input + ) } fn with_exprs_and_inputs( @@ -155,6 +159,7 @@ impl std::fmt::Debug for MergeScanExec { .field("table", &self.table) .field("regions", &self.regions) .field("schema", &self.schema) + .field("plan", &self.plan) .finish() } } diff --git a/src/query/src/promql/planner.rs b/src/query/src/promql/planner.rs index 27c47d8307..09eaf1c879 100644 --- a/src/query/src/promql/planner.rs +++ b/src/query/src/promql/planner.rs @@ -191,18 +191,38 @@ impl PromPlanner { planner.prom_expr_to_plan(&stmt.expr, session_state).await } - #[async_recursion] pub async fn prom_expr_to_plan( &mut self, prom_expr: &PromExpr, session_state: &SessionState, + ) -> Result { + self.prom_expr_to_plan_inner(prom_expr, false, session_state) + .await + } + + /** + Converts a PromQL expression to a logical plan. + + NOTE: + The `timestamp_fn` indicates whether the PromQL `timestamp()` function is being evaluated in the current context. + If `true`, the planner generates a logical plan that projects the timestamp (time index) column + as the value column for each input row, implementing the PromQL `timestamp()` function semantics. + If `false`, the planner generates the standard logical plan for the given PromQL expression. + */ + #[async_recursion] + async fn prom_expr_to_plan_inner( + &mut self, + prom_expr: &PromExpr, + timestamp_fn: bool, + session_state: &SessionState, ) -> Result { let res = match prom_expr { PromExpr::Aggregate(expr) => self.prom_aggr_expr_to_plan(session_state, expr).await?, PromExpr::Unary(expr) => self.prom_unary_expr_to_plan(session_state, expr).await?, PromExpr::Binary(expr) => self.prom_binary_expr_to_plan(session_state, expr).await?, PromExpr::Paren(ParenExpr { expr }) => { - self.prom_expr_to_plan(expr, session_state).await? + self.prom_expr_to_plan_inner(expr, timestamp_fn, session_state) + .await? } PromExpr::Subquery(expr) => { self.prom_subquery_expr_to_plan(session_state, expr).await? @@ -210,7 +230,8 @@ impl PromPlanner { PromExpr::NumberLiteral(lit) => self.prom_number_lit_to_plan(lit)?, PromExpr::StringLiteral(lit) => self.prom_string_lit_to_plan(lit)?, PromExpr::VectorSelector(selector) => { - self.prom_vector_selector_to_plan(selector).await? + self.prom_vector_selector_to_plan(selector, timestamp_fn) + .await? } PromExpr::MatrixSelector(selector) => { self.prom_matrix_selector_to_plan(selector).await? @@ -673,6 +694,7 @@ impl PromPlanner { async fn prom_vector_selector_to_plan( &mut self, vector_selector: &VectorSelector, + timestamp_fn: bool, ) -> Result { let VectorSelector { name, @@ -687,6 +709,15 @@ impl PromPlanner { let normalize = self .selector_to_series_normalize_plan(offset, matchers, false) .await?; + + let normalize = if timestamp_fn { + // If evaluating the PromQL `timestamp()` function, project the time index column as the value column + // before wrapping with [`InstantManipulate`], so the output matches PromQL's `timestamp()` semantics. + self.create_timestamp_func_plan(normalize)? + } else { + normalize + }; + let manipulate = InstantManipulate::new( self.ctx.start, self.ctx.end, @@ -704,6 +735,43 @@ impl PromPlanner { })) } + /// Builds a projection plan for the PromQL `timestamp()` function. + /// Projects the time index column as the value column for each row. + /// + /// # Arguments + /// * `normalize` - Input [`LogicalPlan`] for the normalized series. + /// + /// # Returns + /// Returns a [`Result`] where the resulting logical plan projects the timestamp + /// column as the value column, along with the original tag and time index columns. + /// + /// # Timestamp vs. Time Function + /// + /// - **Timestamp Function (`timestamp()`)**: In PromQL, the `timestamp()` function returns the + /// timestamp (time index) of each sample as the value column. + /// + /// - **Time Function (`time()`)**: The `time()` function returns the evaluation time of the query + /// as a scalar value. + /// + /// # Side Effects + /// Updates the planner context's field columns to the timestamp column name. + /// + fn create_timestamp_func_plan(&mut self, normalize: LogicalPlan) -> Result { + let time_expr = build_special_time_expr(self.ctx.time_index_column.as_ref().unwrap()) + .alias(DEFAULT_FIELD_COLUMN); + self.ctx.field_columns = vec![time_expr.schema_name().to_string()]; + let mut project_exprs = Vec::with_capacity(self.ctx.tag_columns.len() + 2); + project_exprs.push(self.create_time_index_column_expr()?); + project_exprs.push(time_expr); + project_exprs.extend(self.create_tag_column_exprs()?); + + LogicalPlanBuilder::from(normalize) + .project(project_exprs) + .context(DataFusionPlanningSnafu)? + .build() + .context(DataFusionPlanningSnafu) + } + async fn prom_matrix_selector_to_plan( &mut self, matrix_selector: &MatrixSelector, @@ -768,7 +836,8 @@ impl PromPlanner { // transform function arguments let args = self.create_function_args(&args.args)?; let input = if let Some(prom_expr) = &args.input { - self.prom_expr_to_plan(prom_expr, session_state).await? + self.prom_expr_to_plan_inner(prom_expr, func.name == "timestamp", session_state) + .await? } else { self.ctx.time_index_column = Some(SPECIAL_TIME_FUNCTION.to_string()); self.ctx.reset_table_name_and_schema(); @@ -1654,7 +1723,7 @@ impl PromPlanner { ScalarFunc::GeneratedExpr } - "sort" | "sort_desc" | "sort_by_label" | "sort_by_label_desc" => { + "sort" | "sort_desc" | "sort_by_label" | "sort_by_label_desc" | "timestamp" => { // These functions are not expression but a part of plan, // they are processed by `prom_call_expr_to_plan`. for value in &self.ctx.field_columns { diff --git a/tests/cases/distributed/explain/join_10_tables.result b/tests/cases/distributed/explain/join_10_tables.result index 6a38a88032..ee0eaf36fd 100644 --- a/tests/cases/distributed/explain/join_10_tables.result +++ b/tests/cases/distributed/explain/join_10_tables.result @@ -84,17 +84,37 @@ limit 1; |_|_Inner Join: t_2.ts = t_3.ts, t_2.vin = t_3.vin_| |_|_Inner Join: t_1.ts = t_2.ts, t_1.vin = t_2.vin_| |_|_Filter: t_1.vin IS NOT NULL_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_1_| +|_| ]]_| |_|_Filter: t_2.vin IS NOT NULL_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_2_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_3_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_4_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_5_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_6_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_7_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_8_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_9_| +|_| ]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: t_10_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [ts@0 DESC], fetch=1_| |_|_SortExec: TopK(fetch=1), expr=[ts@0 DESC], preserve_partitioning=[true]_| |_|_CoalesceBatchesExec: target_batch_size=8192_| diff --git a/tests/cases/distributed/explain/multi_partitions.result b/tests/cases/distributed/explain/multi_partitions.result index 19f5c5d044..702eed8c69 100644 --- a/tests/cases/distributed/explain/multi_partitions.result +++ b/tests/cases/distributed/explain/multi_partitions.result @@ -26,7 +26,12 @@ explain SELECT * FROM demo WHERE ts > cast(1000000000 as timestamp) ORDER BY hos | plan_type_| plan_| +-+-+ | logical_plan_| MergeSort: demo.host ASC NULLS LAST_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Sort: demo.host ASC NULLS LAST_| +|_|_Projection: demo.host, demo.ts, demo.cpu, demo.memory, demo.disk_util_| +|_|_Filter: demo.ts > arrow_cast(Int64(1000000000), Utf8("Timestamp(Millisecond, None)"))_| +|_|_TableScan: demo_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [host@0 ASC NULLS LAST]_| |_|_MergeScanExec: REDACTED |_|_| diff --git a/tests/cases/distributed/explain/order_by.result b/tests/cases/distributed/explain/order_by.result index c0faf9681b..6984279823 100644 --- a/tests/cases/distributed/explain/order_by.result +++ b/tests/cases/distributed/explain/order_by.result @@ -12,7 +12,12 @@ EXPLAIN SELECT DISTINCT i%2 FROM integers ORDER BY 1; +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[ | +|_| Sort: integers.i % Int64(2) ASC NULLS LAST_| +|_|_Distinct:_| +|_|_Projection: integers.i % Int64(2)_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ @@ -35,7 +40,11 @@ EXPLAIN SELECT a, b FROM test ORDER BY a, b; +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| Sort: test.a ASC NULLS LAST, test.b ASC NULLS LAST | +|_|_Projection: test.a, test.b_| +|_|_TableScan: test_| +|_| ]]_| | physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ @@ -50,7 +59,12 @@ EXPLAIN SELECT DISTINCT a, b FROM test ORDER BY a, b; +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| Sort: test.a ASC NULLS LAST, test.b ASC NULLS LAST | +|_|_Distinct:_| +|_|_Projection: test.a, test.b_| +|_|_TableScan: test_| +|_| ]]_| | physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ diff --git a/tests/cases/distributed/explain/single_partition.result b/tests/cases/distributed/explain/single_partition.result index ca94fee301..ce0ae08c4c 100644 --- a/tests/cases/distributed/explain/single_partition.result +++ b/tests/cases/distributed/explain/single_partition.result @@ -12,7 +12,11 @@ EXPLAIN SELECT COUNT(*) FROM single_partition; +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| Projection: count(*)_| +|_|_Aggregate: groupBy=[[]], aggr=[[count(single_partition.j) AS count(*)]] | +|_|_TableScan: single_partition_| +|_| ]]_| | physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ @@ -27,7 +31,11 @@ EXPLAIN SELECT SUM(i) FROM single_partition; +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| Projection: sum(single_partition.i)_| +|_|_Aggregate: groupBy=[[]], aggr=[[sum(single_partition.i)]] | +|_|_TableScan: single_partition_| +|_| ]]_| | physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ @@ -42,7 +50,11 @@ EXPLAIN SELECT * FROM single_partition ORDER BY i DESC; +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| Sort: single_partition.i DESC NULLS FIRST_| +|_|_Projection: single_partition.i, single_partition.j, single_partition.k | +|_|_TableScan: single_partition_| +|_| ]]_| | physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ diff --git a/tests/cases/distributed/explain/step_aggr.result b/tests/cases/distributed/explain/step_aggr.result index 75fc3f3ecb..df9d1891a3 100644 --- a/tests/cases/distributed/explain/step_aggr.result +++ b/tests/cases/distributed/explain/step_aggr.result @@ -55,7 +55,10 @@ FROM +-+-+ | logical_plan_| Projection: sum(count(integers.i)) AS count(integers.i), sum(sum(integers.i)) AS sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))) AS uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))) AS hll_count(hll(integers.i))_| |_|_Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[count(integers.i), sum(integers.i), uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), hll(CAST(integers.i AS Utf8))]]_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i), sum(sum(integers.i))@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@3) as hll_count(hll(integers.i))] | |_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_| |_|_CoalescePartitionsExec_| @@ -156,7 +159,10 @@ ORDER BY | logical_plan_| Sort: integers.ts ASC NULLS LAST_| |_|_Projection: integers.ts, sum(count(integers.i)) AS count(integers.i), sum(sum(integers.i)) AS sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))) AS uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))) AS hll_count(hll(integers.i))_| |_|_Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i), sum(integers.i), uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), hll(CAST(integers.i AS Utf8))]]_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [ts@0 ASC NULLS LAST]_| |_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true]_| |_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i), sum(sum(integers.i))@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@4) as hll_count(hll(integers.i))] | diff --git a/tests/cases/distributed/explain/step_aggr_advance.result b/tests/cases/distributed/explain/step_aggr_advance.result new file mode 100644 index 0000000000..9bc3684eee --- /dev/null +++ b/tests/cases/distributed/explain/step_aggr_advance.result @@ -0,0 +1,974 @@ +CREATE TABLE IF NOT EXISTS aggr_optimize_not ( + a STRING NULL, + b STRING NULL, + c STRING NULL, + d STRING NULL, + greptime_timestamp TIMESTAMP(3) NOT NULL, + greptime_value DOUBLE NULL, + TIME INDEX (greptime_timestamp), + PRIMARY KEY (a, b, c, d) +) PARTITION ON COLUMNS (a, b, c) (a < 'b', a >= 'b',); + +Affected Rows: 0 + +-- Case 0: group by columns are the same as partition columns. +-- This query shouldn't push down aggregation even if group by columns are partitioned. +-- because sort is already pushed down. +-- If it does, it will cause a wrong result. +-- explain at 0s, 5s and 10s. No point at 0s. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') max by (a, b, c) (max_over_time(aggr_optimize_not [2m])); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.greptime_timestamp]], aggr=[[max(prom_max_over_time(greptime_timestamp_range,greptime_value))]] | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_max_over_time(greptime_timestamp_range,greptime_value) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range, greptime_value) AS prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST] | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp], aggr=[max(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1, 2]) | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], preserve_partitioning=[true] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[max(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1, 2]) | +| | ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), a@2 as a, b@3 as b, c@4 as c] | +| | SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') max by (a, b, c) (max_over_time(aggr_optimize_not [2m])); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp], aggr=[max(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1, 2]) REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[max(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1, 2]) REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), a@2 as a, b@3 as b, c@4 as c] REDACTED +|_|_|_SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 1: group by columns are prefix of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') sum by (a, b) (max_over_time(aggr_optimize_not [2m])); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.greptime_timestamp]], aggr=[[sum(prom_max_over_time(greptime_timestamp_range,greptime_value))]] | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_max_over_time(greptime_timestamp_range,greptime_value) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range, greptime_value) AS prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, greptime_timestamp@2 ASC NULLS LAST] | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, greptime_timestamp@2 ASC NULLS LAST], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, greptime_timestamp@2 as greptime_timestamp], aggr=[sum(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1]) | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1]) | +| | ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), a@2 as a, b@3 as b] | +| | SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') sum by (a, b) (max_over_time(aggr_optimize_not [2m])); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, greptime_timestamp@2 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, greptime_timestamp@2 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, greptime_timestamp@2 as greptime_timestamp], aggr=[sum(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1]) REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0, 1]) REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), a@2 as a, b@3 as b] REDACTED +|_|_|_SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 2: group by columns are prefix of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') avg by (a) (max_over_time(aggr_optimize_not [2m])); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.greptime_timestamp]], aggr=[[avg(prom_max_over_time(greptime_timestamp_range,greptime_value))]] | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_max_over_time(greptime_timestamp_range,greptime_value) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range, greptime_value) AS prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, greptime_timestamp@1 ASC NULLS LAST] | +| | SortExec: expr=[a@0 ASC NULLS LAST, greptime_timestamp@1 ASC NULLS LAST], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, greptime_timestamp@1 as greptime_timestamp], aggr=[avg(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0]) | +| | SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[a@2 as a, greptime_timestamp@0 as greptime_timestamp], aggr=[avg(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0]) | +| | ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), a@2 as a] | +| | SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') avg by (a) (max_over_time(aggr_optimize_not [2m])); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [a@0 ASC NULLS LAST, greptime_timestamp@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, greptime_timestamp@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, greptime_timestamp@1 as greptime_timestamp], aggr=[avg(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0]) REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@2 as a, greptime_timestamp@0 as greptime_timestamp], aggr=[avg(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=PartiallySorted([0]) REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), a@2 as a] REDACTED +|_|_|_SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 3: group by columns are superset of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') count by (a, b, c, d) (max_over_time(aggr_optimize_not [2m])); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.d ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d, aggr_optimize_not.greptime_timestamp]], aggr=[[count(prom_max_over_time(greptime_timestamp_range,greptime_value))]] | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_max_over_time(greptime_timestamp_range,greptime_value) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range, greptime_value) AS prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, d@3 ASC NULLS LAST, greptime_timestamp@4 ASC NULLS LAST] | +| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, d@3 as d, greptime_timestamp@4 as greptime_timestamp], aggr=[count(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=Sorted | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, d@3 ASC NULLS LAST, greptime_timestamp@4 ASC NULLS LAST], preserve_partitioning=[true] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, d@5 as d, greptime_timestamp@0 as greptime_timestamp], aggr=[count(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=Sorted | +| | SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') count by (a, b, c, d) (max_over_time(aggr_optimize_not [2m])); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, d@3 ASC NULLS LAST, greptime_timestamp@4 ASC NULLS LAST] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, d@3 as d, greptime_timestamp@4 as greptime_timestamp], aggr=[count(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=Sorted REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, d@3 ASC NULLS LAST, greptime_timestamp@4 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, d@5 as d, greptime_timestamp@0 as greptime_timestamp], aggr=[count(prom_max_over_time(greptime_timestamp_range,greptime_value))], ordering_mode=Sorted REDACTED +|_|_|_SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 4: group by columns are not prefix of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') min by (b, c, d) (max_over_time(aggr_optimize_not [2m])); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.d ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d, aggr_optimize_not.greptime_timestamp]], aggr=[[min(prom_max_over_time(greptime_timestamp_range,greptime_value))]] | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_max_over_time(greptime_timestamp_range,greptime_value) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_max_over_time(greptime_timestamp_range, greptime_value) AS prom_max_over_time(greptime_timestamp_range,greptime_value), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | SortPreservingMergeExec: [b@0 ASC NULLS LAST, c@1 ASC NULLS LAST, d@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST] | +| | SortExec: expr=[b@0 ASC NULLS LAST, c@1 ASC NULLS LAST, d@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, c@1 as c, d@2 as d, greptime_timestamp@3 as greptime_timestamp], aggr=[min(prom_max_over_time(greptime_timestamp_range,greptime_value))] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[b@2 as b, c@3 as c, d@4 as d, greptime_timestamp@0 as greptime_timestamp], aggr=[min(prom_max_over_time(greptime_timestamp_range,greptime_value))] | +| | ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), b@3 as b, c@4 as c, d@5 as d] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') min by (b, c, d) (max_over_time(aggr_optimize_not [2m])); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [b@0 ASC NULLS LAST, c@1 ASC NULLS LAST, d@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[b@0 ASC NULLS LAST, c@1 ASC NULLS LAST, d@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, c@1 as c, d@2 as d, greptime_timestamp@3 as greptime_timestamp], aggr=[min(prom_max_over_time(greptime_timestamp_range,greptime_value))] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[b@2 as b, c@3 as c, d@4 as d, greptime_timestamp@0 as greptime_timestamp], aggr=[min(prom_max_over_time(greptime_timestamp_range,greptime_value))] REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range,greptime_value)@1 as prom_max_over_time(greptime_timestamp_range,greptime_value), b@3 as b, c@4 as c, d@5 as d] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_max_over_time(greptime_timestamp_range,greptime_value)@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_max_over_time(greptime_timestamp_range@6, greptime_value@5) as prom_max_over_time(greptime_timestamp_range,greptime_value), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 5: a simple sum +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain sum(aggr_optimize_not); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.greptime_timestamp]], aggr=[[sum(aggr_optimize_not.greptime_value)]] | +| | Projection: aggr_optimize_not.greptime_timestamp, aggr_optimize_not.greptime_value | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | PromInstantManipulate: range=[0..0], lookback=[300000], interval=[300000], time index=[greptime_timestamp] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-300000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | SortPreservingMergeExec: [greptime_timestamp@0 ASC NULLS LAST] | +| | SortExec: expr=[greptime_timestamp@0 ASC NULLS LAST], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[greptime_timestamp@0 as greptime_timestamp], aggr=[sum(aggr_optimize_not.greptime_value)] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[greptime_timestamp@0 as greptime_timestamp], aggr=[sum(aggr_optimize_not.greptime_value)] | +| | ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, greptime_value@5 as greptime_value] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze sum(aggr_optimize_not); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [greptime_timestamp@0 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[greptime_timestamp@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[greptime_timestamp@0 as greptime_timestamp], aggr=[sum(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[greptime_timestamp@0 as greptime_timestamp], aggr=[sum(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, greptime_value@5 as greptime_value] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_PromInstantManipulateExec: range=[0..0], lookback=[300000], interval=[300000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_PromInstantManipulateExec: range=[0..0], lookback=[300000], interval=[300000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- TODO(discord9): more cases for aggr push down interacting with partitioning&tql +CREATE TABLE IF NOT EXISTS aggr_optimize_not_count ( + a STRING NULL, + b STRING NULL, + c STRING NULL, + d STRING NULL, + greptime_timestamp TIMESTAMP(3) NOT NULL, + greptime_value DOUBLE NULL, + TIME INDEX (greptime_timestamp), + PRIMARY KEY (a, b, c, d) +) PARTITION ON COLUMNS (a, b, c) (a < 'b', a >= 'b',); + +Affected Rows: 0 + +-- Case 6: Test average rate (sum/count like) +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') sum by (a, b, c) (rate(aggr_optimize_not [2m])) / sum by (a, b, c) (rate(aggr_optimize_not_count [2m])); + ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.greptime_timestamp, aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) AS aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) | +| | Inner Join: aggr_optimize_not.a = aggr_optimize_not_count.a, aggr_optimize_not.b = aggr_optimize_not_count.b, aggr_optimize_not.c = aggr_optimize_not_count.c, aggr_optimize_not.greptime_timestamp = aggr_optimize_not_count.greptime_timestamp | +| | SubqueryAlias: aggr_optimize_not | +| | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.greptime_timestamp]], aggr=[[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))]] | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c | +| | MergeSort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_rate(greptime_timestamp_range, greptime_value, aggr_optimize_not.greptime_timestamp, Int64(120000)) AS prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| | SubqueryAlias: aggr_optimize_not_count | +| | Sort: aggr_optimize_not_count.a ASC NULLS LAST, aggr_optimize_not_count.b ASC NULLS LAST, aggr_optimize_not_count.c ASC NULLS LAST, aggr_optimize_not_count.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.greptime_timestamp]], aggr=[[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))]] | +| | Projection: aggr_optimize_not_count.greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c | +| | MergeSort: aggr_optimize_not_count.a ASC NULLS FIRST, aggr_optimize_not_count.b ASC NULLS FIRST, aggr_optimize_not_count.c ASC NULLS FIRST, aggr_optimize_not_count.d ASC NULLS FIRST, aggr_optimize_not_count.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)) IS NOT NULL | +| | Projection: aggr_optimize_not_count.greptime_timestamp, prom_rate(greptime_timestamp_range, greptime_value, aggr_optimize_not_count.greptime_timestamp, Int64(120000)) AS prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.d | +| | PromRangeManipulate: req range=[0..0], interval=[300000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not_count.a ASC NULLS FIRST, aggr_optimize_not_count.b ASC NULLS FIRST, aggr_optimize_not_count.c ASC NULLS FIRST, aggr_optimize_not_count.d ASC NULLS FIRST, aggr_optimize_not_count.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not_count.greptime_timestamp >= TimestampMillisecond(-420000, None) AND aggr_optimize_not_count.greptime_timestamp <= TimestampMillisecond(300000, None) | +| | TableScan: aggr_optimize_not_count | +| | ]] | +| physical_plan | ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, greptime_timestamp@4 as greptime_timestamp, sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@0 / sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@5 as aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | REDACTED +| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], preserve_partitioning=[true] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) | +| | ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c] | +| | SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] | +| | MergeScanExec: REDACTED +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | CoalescePartitionsExec | +| | AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], preserve_partitioning=[true] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=REDACTED +| | AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) | +| | ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c] | +| | SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] | +| | MergeScanExec: REDACTED +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') sum by (a, b, c) (rate(aggr_optimize_not [2m])) / sum by (a, b, c) (rate(aggr_optimize_not_count [2m])); + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, greptime_timestamp@4 as greptime_timestamp, sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@0 / sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@5 as aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c] REDACTED +|_|_|_SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_CoalescePartitionsExec REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) REDACTED +|_|_|_SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))], ordering_mode=PartiallySorted([0, 1, 2]) REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c] REDACTED +|_|_|_SortExec: expr=[a@2 ASC, b@3 ASC, c@4 ASC, d@5 ASC, greptime_timestamp@0 ASC], preserve_partitioning=[true] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not.greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c, d@5 as d] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not.greptime_timestamp,Int64(120000))@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_rate(greptime_timestamp_range@6, greptime_value@5, greptime_timestamp@4, 120000) as prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not.greptime_timestamp,Int64(120000)), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not.greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c, d@5 as d] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not.greptime_timestamp,Int64(120000))@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_rate(greptime_timestamp_range@6, greptime_value@5, greptime_timestamp@4, 120000) as prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not.greptime_timestamp,Int64(120000)), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 0_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not_count.greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c, d@5 as d] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not_count.greptime_timestamp,Int64(120000))@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_rate(greptime_timestamp_range@6, greptime_value@5, greptime_timestamp@4, 120000) as prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not_count.greptime_timestamp,Int64(120000)), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +| 1_| 1_|_ProjectionExec: expr=[greptime_timestamp@0 as greptime_timestamp, prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not_count.greptime_timestamp,Int64(120000))@1 as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@2 as a, b@3 as b, c@4 as c, d@5 as d] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_FilterExec: prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not_count.greptime_timestamp,Int64(120000))@1 IS NOT NULL REDACTED +|_|_|_ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_rate(greptime_timestamp_range@6, greptime_value@5, greptime_timestamp@4, 120000) as prom_rate(greptime_timestamp_range,greptime_value,aggr_optimize_not_count.greptime_timestamp,Int64(120000)), a@0 as a, b@1 as b, c@2 as c, d@3 as d] REDACTED +|_|_|_PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] REDACTED +|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] REDACTED +|_|_|_PromSeriesDivideExec: tags=["a", "b", "c", "d"] REDACTED +|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 7: aggregate without sort should be pushed down. This one push down for include all partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c; + ++---------------+----------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+----------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: min(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c]], aggr=[[min(aggr_optimize_not.greptime_value)]] | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | MergeScanExec: REDACTED +| | | ++---------------+----------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@3 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +| 1_| 1_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@3 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 8: aggregate without sort should be pushed down. This one push down for include all partition columns then some +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c, + d; + ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: min(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d]], aggr=[[min(aggr_optimize_not.greptime_value)]] | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | MergeScanExec: REDACTED +| | | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c, + d; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@4 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, d@3 as d], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c, d@3 as d], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +| 1_| 1_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@4 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c, d@3 as d], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c, d@3 as d], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 9: aggregate without sort should be pushed down. This one push down for step aggr push down. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + ++---------------+------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: min(min(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(min(aggr_optimize_not.greptime_value))]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(aggr_optimize_not.greptime_value)]] | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 as min(aggr_optimize_not.greptime_value)] | +| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value))] | +| | MergeScanExec: REDACTED +| | | ++---------------+------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value))] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 10: aggregate without sort should be pushed down. This one push down for step aggr push down with complex aggr +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) + max(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + ++---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: min(min(aggr_optimize_not.greptime_value)) + max(max(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)]] | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 + max(max(aggr_optimize_not.greptime_value))@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] | +| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))] | +| | MergeScanExec: REDACTED +| | | ++---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) + max(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 + max(max(aggr_optimize_not.greptime_value))@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +-- Case 11: aggregate with subquery +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + a, + min(greptime_value) +FROM + ( + SELECT + a, + b, + greptime_value + FROM + aggr_optimize_not + ORDER BY + a, + b + ) +GROUP BY + a; + ++---------------+------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: aggr_optimize_not.a, min(min(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[min(min(aggr_optimize_not.greptime_value))]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[min(aggr_optimize_not.greptime_value)]] | +| | Projection: aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.greptime_value | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | ProjectionExec: expr=[a@0 as a, min(min(aggr_optimize_not.greptime_value))@1 as min(aggr_optimize_not.greptime_value)] | +| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(min(aggr_optimize_not.greptime_value))] | +| | MergeScanExec: REDACTED +| | | ++---------------+------------------------------------------------------------------------------------------------------------------------+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + a, + min(greptime_value) +FROM + ( + SELECT + a, + b, + greptime_value + FROM + aggr_optimize_not + ORDER BY + a, + b + ) +GROUP BY + a; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_ProjectionExec: expr=[a@0 as a, min(min(aggr_optimize_not.greptime_value))@1 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(min(aggr_optimize_not.greptime_value))] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED +|_|_|_RepartitionExec: partitioning=REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED +|_|_|_| +|_|_| Total rows: 0_| ++-+-+-+ + +drop table aggr_optimize_not_count; + +Affected Rows: 0 + +drop table aggr_optimize_not; + +Affected Rows: 0 + diff --git a/tests/cases/distributed/explain/step_aggr_advance.sql b/tests/cases/distributed/explain/step_aggr_advance.sql new file mode 100644 index 0000000000..aa8554bdb8 --- /dev/null +++ b/tests/cases/distributed/explain/step_aggr_advance.sql @@ -0,0 +1,307 @@ +CREATE TABLE IF NOT EXISTS aggr_optimize_not ( + a STRING NULL, + b STRING NULL, + c STRING NULL, + d STRING NULL, + greptime_timestamp TIMESTAMP(3) NOT NULL, + greptime_value DOUBLE NULL, + TIME INDEX (greptime_timestamp), + PRIMARY KEY (a, b, c, d) +) PARTITION ON COLUMNS (a, b, c) (a < 'b', a >= 'b',); + +-- Case 0: group by columns are the same as partition columns. +-- This query shouldn't push down aggregation even if group by columns are partitioned. +-- because sort is already pushed down. +-- If it does, it will cause a wrong result. +-- explain at 0s, 5s and 10s. No point at 0s. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') max by (a, b, c) (max_over_time(aggr_optimize_not [2m])); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') max by (a, b, c) (max_over_time(aggr_optimize_not [2m])); + +-- Case 1: group by columns are prefix of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') sum by (a, b) (max_over_time(aggr_optimize_not [2m])); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') sum by (a, b) (max_over_time(aggr_optimize_not [2m])); + +-- Case 2: group by columns are prefix of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') avg by (a) (max_over_time(aggr_optimize_not [2m])); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') avg by (a) (max_over_time(aggr_optimize_not [2m])); + +-- Case 3: group by columns are superset of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') count by (a, b, c, d) (max_over_time(aggr_optimize_not [2m])); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') count by (a, b, c, d) (max_over_time(aggr_optimize_not [2m])); + +-- Case 4: group by columns are not prefix of partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') min by (b, c, d) (max_over_time(aggr_optimize_not [2m])); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') min by (b, c, d) (max_over_time(aggr_optimize_not [2m])); + +-- Case 5: a simple sum +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain sum(aggr_optimize_not); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze sum(aggr_optimize_not); + +-- TODO(discord9): more cases for aggr push down interacting with partitioning&tql +CREATE TABLE IF NOT EXISTS aggr_optimize_not_count ( + a STRING NULL, + b STRING NULL, + c STRING NULL, + d STRING NULL, + greptime_timestamp TIMESTAMP(3) NOT NULL, + greptime_value DOUBLE NULL, + TIME INDEX (greptime_timestamp), + PRIMARY KEY (a, b, c, d) +) PARTITION ON COLUMNS (a, b, c) (a < 'b', a >= 'b',); + +-- Case 6: Test average rate (sum/count like) +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +tql explain (1752591864, 1752592164, '30s') sum by (a, b, c) (rate(aggr_optimize_not [2m])) / sum by (a, b, c) (rate(aggr_optimize_not_count [2m])); + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +tql analyze (1752591864, 1752592164, '30s') sum by (a, b, c) (rate(aggr_optimize_not [2m])) / sum by (a, b, c) (rate(aggr_optimize_not_count [2m])); + +-- Case 7: aggregate without sort should be pushed down. This one push down for include all partition columns. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c; + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c; + +-- Case 8: aggregate without sort should be pushed down. This one push down for include all partition columns then some +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c, + d; + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b, + c, + d; + +-- Case 9: aggregate without sort should be pushed down. This one push down for step aggr push down. +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + +-- Case 10: aggregate without sort should be pushed down. This one push down for step aggr push down with complex aggr +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + min(greptime_value) + max(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + min(greptime_value) + max(greptime_value) +FROM + aggr_optimize_not +GROUP BY + a, + b; + + +-- Case 11: aggregate with subquery +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN +SELECT + a, + min(greptime_value) +FROM + ( + SELECT + a, + b, + greptime_value + FROM + aggr_optimize_not + ORDER BY + a, + b + ) +GROUP BY + a; + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN ANALYZE +SELECT + a, + min(greptime_value) +FROM + ( + SELECT + a, + b, + greptime_value + FROM + aggr_optimize_not + ORDER BY + a, + b + ) +GROUP BY + a; + +drop table aggr_optimize_not_count; + +drop table aggr_optimize_not; diff --git a/tests/cases/distributed/explain/step_aggr_basic.result b/tests/cases/distributed/explain/step_aggr_basic.result index c522f249a7..973aa657fe 100644 --- a/tests/cases/distributed/explain/step_aggr_basic.result +++ b/tests/cases/distributed/explain/step_aggr_basic.result @@ -50,7 +50,10 @@ FROM +-+-+ | logical_plan_| Projection: sum(count(integers.i)) AS count(integers.i)_| |_|_Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[count(integers.i)]]_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i)]_| |_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i))]_| |_|_CoalescePartitionsExec_| @@ -144,7 +147,10 @@ ORDER BY | logical_plan_| Sort: integers.ts ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| |_|_Projection: integers.ts, sum(count(integers.i)) AS count(integers.i)_| |_|_Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i)]]_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_| |_|_SortExec: expr=[ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true]_| |_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i)]_| @@ -253,7 +259,10 @@ ORDER BY | logical_plan_| Sort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| |_|_Projection: date_bin(Utf8("1 hour"),integers.ts) AS time_window, sum(count(integers.i)) AS count(integers.i)_| |_|_Aggregate: groupBy=[[date_bin(Utf8("1 hour"),integers.ts)]], aggr=[[sum(count(integers.i))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]]_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_| |_|_SortExec: expr=[time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true]_| |_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, sum(count(integers.i))@1 as count(integers.i)]_| @@ -369,7 +378,10 @@ ORDER BY | logical_plan_| Sort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_| |_|_Projection: integers.ts + Int64(1), integers.i / Int64(2), sum(count(integers.i)) AS count(integers.i)_| |_|_Aggregate: groupBy=[[integers.ts + Int64(1), integers.i / Int64(2)]], aggr=[[sum(count(integers.i))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[CAST(integers.ts AS Int64) + Int64(1), integers.i / Int64(2)]], aggr=[[count(integers.i)]]_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST]_| |_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true]_| |_|_ProjectionExec: expr=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2), sum(count(integers.i))@2 as count(integers.i)]_| @@ -497,7 +509,10 @@ FROM +-+-+ | logical_plan_| Projection: uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))) AS udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))) AS hll_result_| |_|_Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), sink_table.udd_state), hll_merge(sink_table.hll_state)]]_| +|_|_TableScan: sink_table_| +|_| ]]_| | physical_plan | ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))@0) as udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))@1) as hll_result] | |_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]_| |_|_CoalescePartitionsExec_| diff --git a/tests/cases/distributed/explain/step_aggr_massive.result b/tests/cases/distributed/explain/step_aggr_massive.result index 5686962900..e0ec1e22a2 100644 --- a/tests/cases/distributed/explain/step_aggr_massive.result +++ b/tests/cases/distributed/explain/step_aggr_massive.result @@ -247,7 +247,11 @@ GROUP BY +-+-+ | logical_plan_| Projection: base_table.env, base_table.service_name, base_table.city, base_table.page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS min_fid, max(max(base_table.shard_key)) AS shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))_| |_|_Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(CAST(Utf8("60 seconds") AS Interval(MonthDayNano)), base_table.time), Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), max(base_table.shard_key)]]_| +|_|_Filter: (base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) OR base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) OR base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) OR base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) OR base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) OR base_table.fid > Int64(0) AND base_table.fid < Int64(3000000)) AND CAST(base_table.time AS Timestamp(Millisecond, Some("+00:00"))) >= CAST(now() AS Timestamp(Millisecond, Some("+00:00")))_| +|_|_TableScan: base_table_| +|_| ]]_| | physical_plan | ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@5 as lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@6 as max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@7 as min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@8 as fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@9 as max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@10 as min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@11 as fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@12 as max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@13 as min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@14 as fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@15 as max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@16 as min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@17 as tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@18 as max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@19 as min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@20 as fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@21 as max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@22 as min_fid, max(max(base_table.shard_key))@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] | |_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]_| |_|_CoalesceBatchesExec: target_batch_size=8192_| @@ -624,7 +628,11 @@ where +-+-+ | logical_plan_| Projection: count(*) AS count(*)_| |_|_Aggregate: groupBy=[[]], aggr=[[sum(count(*)) AS count(*)]]_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[count(base_table.time) AS count(*)]]_| +|_|_Filter: CAST(base_table.time AS Timestamp(Millisecond, Some("+00:00"))) >= CAST(now() AS Timestamp(Millisecond, Some("+00:00")))_| +|_|_TableScan: base_table_| +|_| ]]_| | physical_plan | AggregateExec: mode=Final, gby=[], aggr=[count(*)]_| |_|_CoalescePartitionsExec_| |_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)]_| diff --git a/tests/cases/distributed/explain/subqueries.result b/tests/cases/distributed/explain/subqueries.result index aff0ffc4b9..feb317c273 100644 --- a/tests/cases/distributed/explain/subqueries.result +++ b/tests/cases/distributed/explain/subqueries.result @@ -14,9 +14,14 @@ EXPLAIN SELECT * FROM integers WHERE i IN ((SELECT i FROM integers)) ORDER BY i; +-+-+ | logical_plan_| Sort: integers.i ASC NULLS LAST_| |_|_LeftSemi Join: integers.i = __correlated_sq_1.i_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: integers_| +|_| ]]_| |_|_SubqueryAlias: __correlated_sq_1_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Projection: integers.i_| +|_|_TableScan: integers_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [i@0 ASC NULLS LAST]_| |_|_SortExec: expr=[i@0 ASC NULLS LAST], preserve_partitioning=[true]_| |_|_CoalesceBatchesExec: target_batch_size=8192_| @@ -43,10 +48,14 @@ EXPLAIN SELECT * FROM integers i1 WHERE EXISTS(SELECT i FROM integers WHERE i=i1 | logical_plan_| Sort: i1.i ASC NULLS LAST_| |_|_LeftSemi Join: i1.i = __correlated_sq_1.i_| |_|_SubqueryAlias: i1_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: integers_| +|_| ]]_| |_|_SubqueryAlias: __correlated_sq_1_| |_|_Projection: integers.i_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: integers_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [i@0 ASC NULLS LAST]_| |_|_SortExec: expr=[i@0 ASC NULLS LAST], preserve_partitioning=[true]_| |_|_CoalesceBatchesExec: target_batch_size=8192_| @@ -85,9 +94,13 @@ order by t.i desc; |_|_Cross Join:_| |_|_Filter: integers.i IS NOT NULL_| |_|_Projection: integers.i_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: integers_| +|_| ]]_| |_|_Projection:_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: other_| +|_| ]]_| | physical_plan | SortPreservingMergeExec: [i@0 DESC]_| |_|_SortExec: expr=[i@0 DESC], preserve_partitioning=[true]_| |_|_CrossJoinExec_| @@ -116,9 +129,15 @@ EXPLAIN INSERT INTO other SELECT i, 2 FROM integers WHERE i=(SELECT MAX(i) FROM | | Projection: integers.i AS i, TimestampMillisecond(2, None) AS j | | | Inner Join: integers.i = __scalar_sq_1.max(integers.i) | | | Projection: integers.i | -| | MergeScan [is_placeholder=false] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | TableScan: integers | +| | ]] | | | SubqueryAlias: __scalar_sq_1 | -| | MergeScan [is_placeholder=false] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: max(integers.i) | +| | Aggregate: groupBy=[[]], aggr=[[max(integers.i)]] | +| | TableScan: integers | +| | ]] | | physical_plan_error | Error during planning: failed to resolve catalog: datafusion | +---------------------+-------------------------------------------------------------------+ diff --git a/tests/cases/distributed/optimizer/filter_push_down.result b/tests/cases/distributed/optimizer/filter_push_down.result index c9744b76be..bafff97a08 100644 --- a/tests/cases/distributed/optimizer/filter_push_down.result +++ b/tests/cases/distributed/optimizer/filter_push_down.result @@ -252,10 +252,14 @@ EXPLAIN SELECT * FROM (SELECT 0=1 AS cond FROM integers i1, integers i2) a1 WHER |_|_Cross Join:_| |_|_SubqueryAlias: i1_| |_|_Projection:_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[ | +|_| TableScan: integers_| +|_| ]]_| |_|_SubqueryAlias: i2_| |_|_Projection:_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[ | +|_| TableScan: integers_| +|_| ]]_| | physical_plan | CoalescePartitionsExec_| |_|_ProjectionExec: expr=[false as cond]_| |_|_CrossJoinExec_| diff --git a/tests/cases/distributed/optimizer/order_by.result b/tests/cases/distributed/optimizer/order_by.result index 45ee66a6c5..135e52509e 100644 --- a/tests/cases/distributed/optimizer/order_by.result +++ b/tests/cases/distributed/optimizer/order_by.result @@ -4,7 +4,10 @@ explain select * from numbers; +---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | +---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -15,7 +18,11 @@ explain select * from numbers order by number desc; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Sort: numbers.number DESC NULLS FIRST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: expr=[number@0 DESC], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | @@ -27,7 +34,11 @@ explain select * from numbers order by number asc; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Sort: numbers.number ASC NULLS LAST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | @@ -39,7 +50,12 @@ explain select * from numbers order by number desc limit 10; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Limit: skip=0, fetch=10 | +| | Sort: numbers.number DESC NULLS FIRST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: TopK(fetch=10), expr=[number@0 DESC], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | @@ -51,7 +67,12 @@ explain select * from numbers order by number asc limit 10; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Limit: skip=0, fetch=10 | +| | Sort: numbers.number ASC NULLS LAST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: TopK(fetch=10), expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | diff --git a/tests/cases/standalone/common/order/order_by_exceptions.result b/tests/cases/standalone/common/order/order_by_exceptions.result index 6385b9d721..be789c821e 100644 --- a/tests/cases/standalone/common/order/order_by_exceptions.result +++ b/tests/cases/standalone/common/order/order_by_exceptions.result @@ -70,8 +70,14 @@ EXPLAIN SELECT a % 2, b FROM test UNION SELECT a % 2 AS k, b FROM test ORDER BY | logical_plan | Sort: Int64(-1) ASC NULLS LAST | | | Aggregate: groupBy=[[test.a % Int64(2), test.b]], aggr=[[]] | | | Union | -| | MergeScan [is_placeholder=false] | -| | MergeScan [is_placeholder=false] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: CAST(test.a AS Int64) % Int64(2) AS test.a % Int64(2), test.b | +| | TableScan: test | +| | ]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: CAST(test.a AS Int64) % Int64(2) AS test.a % Int64(2), test.b | +| | TableScan: test | +| | ]] | | physical_plan | CoalescePartitionsExec | | | AggregateExec: mode=SinglePartitioned, gby=[test.a % Int64(2)@0 as test.a % Int64(2), b@1 as b], aggr=[] | | | InterleaveExec | diff --git a/tests/cases/standalone/common/promql/timestamp_fn.result b/tests/cases/standalone/common/promql/timestamp_fn.result new file mode 100644 index 0000000000..2fee237dcd --- /dev/null +++ b/tests/cases/standalone/common/promql/timestamp_fn.result @@ -0,0 +1,160 @@ +-- Test `timestamp()` function +-- timestamp() returns the timestamp of each sample as seconds since Unix epoch +create table timestamp_test (ts timestamp time index, val double); + +Affected Rows: 0 + +insert into timestamp_test values + (0, 1.0), + (1000, 2.0), + (60000, 3.0), + (3600000, 4.0), + -- 2021-01-01 00:00:00 + (1609459200000, 5.0), + -- 2021-01-01 00:01:00 + (1609459260000, 6.0); + +Affected Rows: 6 + +-- Test timestamp() with time series +tql eval (0, 3600, '30s') timestamp(timestamp_test); + ++---------------------+--------+ +| ts | value | ++---------------------+--------+ +| 1970-01-01T00:00:00 | 0.0 | +| 1970-01-01T00:00:30 | 1.0 | +| 1970-01-01T00:01:00 | 60.0 | +| 1970-01-01T00:01:30 | 60.0 | +| 1970-01-01T00:02:00 | 60.0 | +| 1970-01-01T00:02:30 | 60.0 | +| 1970-01-01T00:03:00 | 60.0 | +| 1970-01-01T00:03:30 | 60.0 | +| 1970-01-01T00:04:00 | 60.0 | +| 1970-01-01T00:04:30 | 60.0 | +| 1970-01-01T00:05:00 | 60.0 | +| 1970-01-01T00:05:30 | 60.0 | +| 1970-01-01T00:06:00 | 60.0 | +| 1970-01-01T01:00:00 | 3600.0 | ++---------------------+--------+ + +-- Test timestamp() with specific time range +tql eval (0, 60, '30s') timestamp(timestamp_test); + ++---------------------+-------+ +| ts | value | ++---------------------+-------+ +| 1970-01-01T00:00:00 | 0.0 | +| 1970-01-01T00:00:30 | 1.0 | +| 1970-01-01T00:01:00 | 60.0 | ++---------------------+-------+ + +tql eval (0, 60, '30s') -timestamp(timestamp_test); + ++---------------------+-----------+ +| ts | (- value) | ++---------------------+-----------+ +| 1970-01-01T00:00:00 | -0.0 | +| 1970-01-01T00:00:30 | -1.0 | +| 1970-01-01T00:01:00 | -60.0 | ++---------------------+-----------+ + +-- Test timestamp() with 2021 data +tql eval (1609459200, 1609459260, '30s') timestamp(timestamp_test); + ++---------------------+--------------+ +| ts | value | ++---------------------+--------------+ +| 2021-01-01T00:00:00 | 1609459200.0 | +| 2021-01-01T00:00:30 | 1609459200.0 | +| 2021-01-01T00:01:00 | 1609459260.0 | ++---------------------+--------------+ + +-- Test timestamp() with arithmetic operations +tql eval (0, 60, '30s') timestamp(timestamp_test) + 1; + ++---------------------+--------------------+ +| ts | value + Float64(1) | ++---------------------+--------------------+ +| 1970-01-01T00:00:00 | 1.0 | +| 1970-01-01T00:00:30 | 2.0 | +| 1970-01-01T00:01:00 | 61.0 | ++---------------------+--------------------+ + +-- Test timestamp() with boolean operations +tql eval (0, 60, '30s') timestamp(timestamp_test) > bool 30; + ++---------------------+---------------------+ +| ts | value > Float64(30) | ++---------------------+---------------------+ +| 1970-01-01T00:00:00 | 0.0 | +| 1970-01-01T00:00:30 | 0.0 | +| 1970-01-01T00:01:00 | 1.0 | ++---------------------+---------------------+ + +-- Test timestamp() with time functions +tql eval (0, 60, '30s') timestamp(timestamp_test) - time(); + ++---------------------+----------------------------+ +| ts | value - ts / Float64(1000) | ++---------------------+----------------------------+ +| 1970-01-01T00:00:00 | 0.0 | +| 1970-01-01T00:00:30 | -29.0 | +| 1970-01-01T00:01:00 | 0.0 | ++---------------------+----------------------------+ + +-- Test timestamp() with other functions +tql eval (0, 60, '30s') abs(timestamp(timestamp_test) - avg(timestamp(timestamp_test))) > 20; + +Error: 1004(InvalidArguments), Invalid function argument for unknown + +tql eval (0, 60, '30s') timestamp(timestamp_test) == 60; + ++---------------------+-------+ +| ts | value | ++---------------------+-------+ +| 1970-01-01T00:01:00 | 60.0 | ++---------------------+-------+ + +-- Test timestamp() with multiple metrics +create table timestamp_test2 (ts timestamp time index, val double); + +Affected Rows: 0 + +insert into timestamp_test2 values + (0, 10.0), + (1000, 20.0), + (60000, 30.0); + +Affected Rows: 3 + +-- SQLNESS SORT_RESULT 3 1 +tql eval (0, 60, '30s') timestamp(timestamp_test) + timestamp(timestamp_test2); + ++---------------------+----------------------------------------------+ +| ts | timestamp_test.value + timestamp_test2.value | ++---------------------+----------------------------------------------+ +| 1970-01-01T00:00:00 | 0.0 | +| 1970-01-01T00:00:30 | 2.0 | +| 1970-01-01T00:01:00 | 120.0 | ++---------------------+----------------------------------------------+ + +-- SQLNESS SORT_RESULT 3 1 +tql eval (0, 60, '30s') timestamp(timestamp_test) == timestamp(timestamp_test2); + ++---------------------+-------+---------------------+-------+ +| ts | value | ts | value | ++---------------------+-------+---------------------+-------+ +| 1970-01-01T00:00:00 | 0.0 | 1970-01-01T00:00:00 | 0.0 | +| 1970-01-01T00:00:30 | 1.0 | 1970-01-01T00:00:30 | 1.0 | +| 1970-01-01T00:01:00 | 60.0 | 1970-01-01T00:01:00 | 60.0 | ++---------------------+-------+---------------------+-------+ + +drop table timestamp_test; + +Affected Rows: 0 + +drop table timestamp_test2; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/promql/timestamp_fn.sql b/tests/cases/standalone/common/promql/timestamp_fn.sql new file mode 100644 index 0000000000..a2f6e9e317 --- /dev/null +++ b/tests/cases/standalone/common/promql/timestamp_fn.sql @@ -0,0 +1,57 @@ +-- Test `timestamp()` function +-- timestamp() returns the timestamp of each sample as seconds since Unix epoch + +create table timestamp_test (ts timestamp time index, val double); + +insert into timestamp_test values + (0, 1.0), + (1000, 2.0), + (60000, 3.0), + (3600000, 4.0), + -- 2021-01-01 00:00:00 + (1609459200000, 5.0), + -- 2021-01-01 00:01:00 + (1609459260000, 6.0); + +-- Test timestamp() with time series +tql eval (0, 3600, '30s') timestamp(timestamp_test); + +-- Test timestamp() with specific time range +tql eval (0, 60, '30s') timestamp(timestamp_test); + +tql eval (0, 60, '30s') -timestamp(timestamp_test); + +-- Test timestamp() with 2021 data +tql eval (1609459200, 1609459260, '30s') timestamp(timestamp_test); + +-- Test timestamp() with arithmetic operations +tql eval (0, 60, '30s') timestamp(timestamp_test) + 1; + +-- Test timestamp() with boolean operations +tql eval (0, 60, '30s') timestamp(timestamp_test) > bool 30; + +-- Test timestamp() with time functions +tql eval (0, 60, '30s') timestamp(timestamp_test) - time(); + +-- Test timestamp() with other functions +tql eval (0, 60, '30s') abs(timestamp(timestamp_test) - avg(timestamp(timestamp_test))) > 20; + +tql eval (0, 60, '30s') timestamp(timestamp_test) == 60; + +-- Test timestamp() with multiple metrics +create table timestamp_test2 (ts timestamp time index, val double); + +insert into timestamp_test2 values + (0, 10.0), + (1000, 20.0), + (60000, 30.0); + +-- SQLNESS SORT_RESULT 3 1 +tql eval (0, 60, '30s') timestamp(timestamp_test) + timestamp(timestamp_test2); + +-- SQLNESS SORT_RESULT 3 1 +tql eval (0, 60, '30s') timestamp(timestamp_test) == timestamp(timestamp_test2); + +drop table timestamp_test; + +drop table timestamp_test2; diff --git a/tests/cases/standalone/common/range/nest.result b/tests/cases/standalone/common/range/nest.result index 016b658bd3..89c0b9c8ff 100644 --- a/tests/cases/standalone/common/range/nest.result +++ b/tests/cases/standalone/common/range/nest.result @@ -57,7 +57,9 @@ EXPLAIN SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s'; +-+-+ | logical_plan_| RangeSelect: range_exprs=[min(host.val) RANGE 5s], align=5000ms, align_to=0ms, align_by=[host.host], time_index=ts | |_|_Projection: host.ts, host.host, host.val_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: host_| +|_| ]]_| | physical_plan | RangeSelectExec: range_expr=[min(host.val) RANGE 5s], align=5000ms, align_to=0ms, align_by=[host@1], time_index=ts | |_|_CoalescePartitionsExec_| |_|_MergeScanExec: REDACTED diff --git a/tests/cases/standalone/common/tql-explain-analyze/explain.result b/tests/cases/standalone/common/tql-explain-analyze/explain.result index bc3ebc0b35..4a3aa78b44 100644 --- a/tests/cases/standalone/common/tql-explain-analyze/explain.result +++ b/tests/cases/standalone/common/tql-explain-analyze/explain.result @@ -12,13 +12,19 @@ Affected Rows: 3 -- SQLNESS REPLACE (peers.*) REDACTED TQL EXPLAIN (0, 10, '5s') test; -+---------------+-------------------------------------------------+ -| plan_type | plan | -+---------------+-------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | ++---------------+--------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+--------------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | PromInstantManipulate: range=[0..0], lookback=[300000], interval=[300000], time index=[j] | +| | PromSeriesDivide: tags=["k"] | +| | Sort: test.k ASC NULLS FIRST, test.j ASC NULLS FIRST | +| | Filter: test.j >= TimestampMillisecond(-300000, None) AND test.j <= TimestampMillisecond(300000, None) | +| | TableScan: test | +| | ]] | | physical_plan | MergeScanExec: REDACTED -| | | -+---------------+-------------------------------------------------+ +| | | ++---------------+--------------------------------------------------------------------------------------------------------------+ -- 'lookback' parameter is not fully supported, the test has to be updated -- explain at 0s, 5s and 10s. No point at 0s. @@ -26,26 +32,38 @@ TQL EXPLAIN (0, 10, '5s') test; -- SQLNESS REPLACE (peers.*) REDACTED TQL EXPLAIN (0, 10, '1s', '2s') test; -+---------------+-------------------------------------------------+ -| plan_type | plan | -+---------------+-------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | ++---------------+----------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+----------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | PromInstantManipulate: range=[0..0], lookback=[2000], interval=[300000], time index=[j] | +| | PromSeriesDivide: tags=["k"] | +| | Sort: test.k ASC NULLS FIRST, test.j ASC NULLS FIRST | +| | Filter: test.j >= TimestampMillisecond(-2000, None) AND test.j <= TimestampMillisecond(2000, None) | +| | TableScan: test | +| | ]] | | physical_plan | MergeScanExec: REDACTED -| | | -+---------------+-------------------------------------------------+ +| | | ++---------------+----------------------------------------------------------------------------------------------------------+ -- explain at 0s, 5s and 10s. No point at 0s. -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED -- SQLNESS REPLACE (peers.*) REDACTED TQL EXPLAIN ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp + '10 seconds'::interval, '5s') test; -+---------------+-------------------------------------------------+ -| plan_type | plan | -+---------------+-------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | ++---------------+--------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+--------------------------------------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | PromInstantManipulate: range=[0..0], lookback=[300000], interval=[300000], time index=[j] | +| | PromSeriesDivide: tags=["k"] | +| | Sort: test.k ASC NULLS FIRST, test.j ASC NULLS FIRST | +| | Filter: test.j >= TimestampMillisecond(-300000, None) AND test.j <= TimestampMillisecond(300000, None) | +| | TableScan: test | +| | ]] | | physical_plan | MergeScanExec: REDACTED -| | | -+---------------+-------------------------------------------------+ +| | | ++---------------+--------------------------------------------------------------------------------------------------------------+ -- explain verbose at 0s, 5s and 10s. No point at 0s. -- SQLNESS REPLACE (-+) - @@ -71,7 +89,13 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test; | logical_plan after resolve_grouping_function_| SAME TEXT AS ABOVE_| | logical_plan after type_coercion_| SAME TEXT AS ABOVE_| | logical_plan after DistPlannerAnalyzer_| Projection: test.i, test.j, test.k_| -|_|_MergeScan [is_placeholder=false]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| PromInstantManipulate: range=[0..0], lookback=[300000], interval=[300000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test.k ASC NULLS FIRST, test.j ASC NULLS FIRST_| +|_|_Filter: test.j >= TimestampMillisecond(-300000, None) AND test.j <= TimestampMillisecond(300000, None)_| +|_|_TableScan: test_| +|_| ]]_| | analyzed_logical_plan_| SAME TEXT AS ABOVE_| | logical_plan after eliminate_nested_union_| SAME TEXT AS ABOVE_| | logical_plan after simplify_expressions_| SAME TEXT AS ABOVE_| @@ -97,7 +121,13 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test; | logical_plan after unwrap_cast_in_comparison_| SAME TEXT AS ABOVE_| | logical_plan after common_sub_expression_eliminate_| SAME TEXT AS ABOVE_| | logical_plan after eliminate_group_by_constant_| SAME TEXT AS ABOVE_| -| logical_plan after optimize_projections_| MergeScan [is_placeholder=false]_| +| logical_plan after optimize_projections_| MergeScan [is_placeholder=false, remote_input=[_| +|_| PromInstantManipulate: range=[0..0], lookback=[300000], interval=[300000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test.k ASC NULLS FIRST, test.j ASC NULLS FIRST_| +|_|_Filter: test.j >= TimestampMillisecond(-300000, None) AND test.j <= TimestampMillisecond(300000, None)_| +|_|_TableScan: test_| +|_| ]]_| | logical_plan after ScanHintRule_| SAME TEXT AS ABOVE_| | logical_plan after eliminate_nested_union_| SAME TEXT AS ABOVE_| | logical_plan after simplify_expressions_| SAME TEXT AS ABOVE_| @@ -125,7 +155,13 @@ TQL EXPLAIN VERBOSE (0, 10, '5s') test; | logical_plan after eliminate_group_by_constant_| SAME TEXT AS ABOVE_| | logical_plan after optimize_projections_| SAME TEXT AS ABOVE_| | logical_plan after ScanHintRule_| SAME TEXT AS ABOVE_| -| logical_plan_| MergeScan [is_placeholder=false]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| PromInstantManipulate: range=[0..0], lookback=[300000], interval=[300000], time index=[j]_| +|_|_PromSeriesDivide: tags=["k"]_| +|_|_Sort: test.k ASC NULLS FIRST, test.j ASC NULLS FIRST_| +|_|_Filter: test.j >= TimestampMillisecond(-300000, None) AND test.j <= TimestampMillisecond(300000, None)_| +|_|_TableScan: test_| +|_| ]]_| | initial_physical_plan_| MergeScanExec: REDACTED |_|_| | initial_physical_plan_with_stats_| MergeScanExec: REDACTED diff --git a/tests/cases/standalone/limit/limit.result b/tests/cases/standalone/limit/limit.result index 323b8ae78d..35c11ec358 100644 --- a/tests/cases/standalone/limit/limit.result +++ b/tests/cases/standalone/limit/limit.result @@ -5,23 +5,37 @@ SELECT * FROM (SELECT SUM(number) FROM numbers LIMIT 100000000000) LIMIT 0; EXPLAIN SELECT * FROM (SELECT SUM(number) FROM numbers LIMIT 100000000000) LIMIT 0; -+---------------+----------------------------------+ -| plan_type | plan | -+---------------+----------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | -| physical_plan | EmptyExec | -| | | -+---------------+----------------------------------+ ++---------------+-------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Limit: skip=0, fetch=0 | +| | Projection: sum(numbers.number) | +| | Limit: skip=0, fetch=100000000000 | +| | Projection: sum(numbers.number) | +| | Aggregate: groupBy=[[]], aggr=[[sum(CAST(numbers.number AS UInt64))]] | +| | TableScan: numbers | +| | ]] | +| physical_plan | EmptyExec | +| | | ++---------------+-------------------------------------------------------------------------------+ EXPLAIN SELECT * FROM (SELECT SUM(number) FROM numbers LIMIT 100000000000) WHERE 1=0; -+---------------+----------------------------------+ -| plan_type | plan | -+---------------+----------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | -| physical_plan | EmptyExec | -| | | -+---------------+----------------------------------+ ++---------------+-------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-------------------------------------------------------------------------------+ +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: sum(numbers.number) | +| | Filter: Int64(1) = Int64(0) | +| | Limit: skip=0, fetch=100000000000 | +| | Projection: sum(numbers.number) | +| | Aggregate: groupBy=[[]], aggr=[[sum(CAST(numbers.number AS UInt64))]] | +| | TableScan: numbers | +| | ]] | +| physical_plan | EmptyExec | +| | | ++---------------+-------------------------------------------------------------------------------+ CREATE TABLE test (a TIMESTAMP TIME INDEX, b INTEGER); diff --git a/tests/cases/standalone/optimizer/order_by.result b/tests/cases/standalone/optimizer/order_by.result index 322a7c5a59..78870479f8 100644 --- a/tests/cases/standalone/optimizer/order_by.result +++ b/tests/cases/standalone/optimizer/order_by.result @@ -3,7 +3,10 @@ explain select * from numbers; +---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | +---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ @@ -13,7 +16,11 @@ explain select * from numbers order by number desc; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Sort: numbers.number DESC NULLS FIRST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: expr=[number@0 DESC], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | @@ -24,7 +31,11 @@ explain select * from numbers order by number asc; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Sort: numbers.number ASC NULLS LAST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | @@ -35,7 +46,12 @@ explain select * from numbers order by number desc limit 10; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Limit: skip=0, fetch=10 | +| | Sort: numbers.number DESC NULLS FIRST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: TopK(fetch=10), expr=[number@0 DESC], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | | @@ -46,7 +62,12 @@ explain select * from numbers order by number asc limit 10; +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | plan_type | plan | +---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | MergeScan [is_placeholder=false] | +| logical_plan | MergeScan [is_placeholder=false, remote_input=[ | +| | Limit: skip=0, fetch=10 | +| | Sort: numbers.number ASC NULLS LAST | +| | Projection: numbers.number | +| | TableScan: numbers | +| | ]] | | physical_plan | SortExec: TopK(fetch=10), expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] | | | StreamScanAdapter: [], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] | | | |