mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2025-12-22 22:20:02 +00:00
* feat: add metrics for request wait time and adjust stall metrics (#6540) * feat: add metric greptime_mito_request_wait_time to observe wait time Signed-off-by: evenyag <realevenyag@gmail.com> * feat: add worker to wait time metric Signed-off-by: evenyag <realevenyag@gmail.com> * refactor: rename stall gauge to greptime_mito_write_stalling_count Signed-off-by: evenyag <realevenyag@gmail.com> * feat: change greptime_mito_write_stall_total to total stalled requests Signed-off-by: evenyag <realevenyag@gmail.com> * refactor: merge lazy static blocks Signed-off-by: evenyag <realevenyag@gmail.com> --------- Signed-off-by: evenyag <realevenyag@gmail.com> * fix: estimate mem size for bulk ingester (#6550) Signed-off-by: evenyag <realevenyag@gmail.com> * fix: flow mirror cache (#6551) * fix: invalid cache when flownode change address Signed-off-by: discord9 <discord9@163.com> * update comments Signed-off-by: discord9 <discord9@163.com> * fix Signed-off-by: discord9 <discord9@163.com> * refactor: add log&rename Signed-off-by: discord9 <discord9@163.com> * stuff Signed-off-by: discord9 <discord9@163.com> --------- Signed-off-by: discord9 <discord9@163.com> Signed-off-by: evenyag <realevenyag@gmail.com> * feat: impl timestamp function for promql (#6556) * feat: impl timestamp function for promql Signed-off-by: Dennis Zhuang <killme2008@gmail.com> * chore: style and typo Signed-off-by: Dennis Zhuang <killme2008@gmail.com> * fix: test Signed-off-by: Dennis Zhuang <killme2008@gmail.com> * docs: update comments Signed-off-by: Dennis Zhuang <killme2008@gmail.com> * chore: comment Signed-off-by: Dennis Zhuang <killme2008@gmail.com> --------- Signed-off-by: Dennis Zhuang <killme2008@gmail.com> Signed-off-by: evenyag <realevenyag@gmail.com> * feat: MergeScan print input (#6563) * feat: MergeScan print input Signed-off-by: discord9 <discord9@163.com> * test: fix ut Signed-off-by: discord9 <discord9@163.com> --------- Signed-off-by: discord9 <discord9@163.com> Signed-off-by: evenyag <realevenyag@gmail.com> * fix: aggr group by all partition cols use partial commutative (#6534) * fix: aggr group by all partition cols use partial commutative Signed-off-by: discord9 <discord9@163.com> * test: bugged case Signed-off-by: discord9 <discord9@163.com> * test: sqlness fix Signed-off-by: discord9 <discord9@163.com> * test: more redacted Signed-off-by: discord9 <discord9@163.com> * more cases Signed-off-by: discord9 <discord9@163.com> * even more test cases Signed-off-by: discord9 <discord9@163.com> * join testcase Signed-off-by: discord9 <discord9@163.com> * fix: column requirement added in correct location Signed-off-by: discord9 <discord9@163.com> * fix test Signed-off-by: discord9 <discord9@163.com> * chore: clippy Signed-off-by: discord9 <discord9@163.com> * track col reqs per stack Signed-off-by: discord9 <discord9@163.com> * fix: continue Signed-off-by: discord9 <discord9@163.com> * chore: clippy Signed-off-by: discord9 <discord9@163.com> * refactor: test mod Signed-off-by: discord9 <discord9@163.com> * test utils Signed-off-by: discord9 <discord9@163.com> * test: better test Signed-off-by: discord9 <discord9@163.com> * more testcases Signed-off-by: discord9 <discord9@163.com> * test limit push down Signed-off-by: discord9 <discord9@163.com> * more testcases Signed-off-by: discord9 <discord9@163.com> * more testcase Signed-off-by: discord9 <discord9@163.com> * more test Signed-off-by: discord9 <discord9@163.com> * chore: update sqlness Signed-off-by: discord9 <discord9@163.com> * chore: update commnets Signed-off-by: discord9 <discord9@163.com> * fix: check col reqs from bottom to upper Signed-off-by: discord9 <discord9@163.com> * chore: more comment Signed-off-by: discord9 <discord9@163.com> * docs: more todo Signed-off-by: discord9 <discord9@163.com> * chore: comments Signed-off-by: discord9 <discord9@163.com> * test: a new failing test that should be fixed Signed-off-by: discord9 <discord9@163.com> * fix: part col alias tracking Signed-off-by: discord9 <discord9@163.com> * chore: unused Signed-off-by: discord9 <discord9@163.com> * chore: clippy Signed-off-by: discord9 <discord9@163.com> * docs: comment Signed-off-by: discord9 <discord9@163.com> * mroe testcase Signed-off-by: discord9 <discord9@163.com> * more testcase for step/part aggr combine Signed-off-by: discord9 <discord9@163.com> * FIXME: a new bug Signed-off-by: discord9 <discord9@163.com> * literally unfixable Signed-off-by: discord9 <discord9@163.com> * chore: remove some debug print Signed-off-by: discord9 <discord9@163.com> --------- Signed-off-by: discord9 <discord9@163.com> Signed-off-by: evenyag <realevenyag@gmail.com> --------- Signed-off-by: evenyag <realevenyag@gmail.com> Signed-off-by: discord9 <discord9@163.com> Signed-off-by: Dennis Zhuang <killme2008@gmail.com> Co-authored-by: fys <40801205+fengys1996@users.noreply.github.com> Co-authored-by: discord9 <55937128+discord9@users.noreply.github.com> Co-authored-by: dennis zhuang <killme2008@gmail.com>
This commit is contained in:
19
src/common/meta/src/cache/flow/table_flownode.rs
vendored
19
src/common/meta/src/cache/flow/table_flownode.rs
vendored
@@ -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<TableId,
|
||||
// we have a corresponding cache invalidation mechanism to invalidate `(Key, EmptyHashSet)`.
|
||||
.map(Arc::new)
|
||||
.map(Some)
|
||||
.inspect(|set| {
|
||||
info!(
|
||||
"Initialized table_flownode cache for table_id: {}, set: {:?}",
|
||||
table_id, set
|
||||
);
|
||||
})
|
||||
})
|
||||
})
|
||||
}
|
||||
@@ -167,6 +174,13 @@ fn invalidator<'a>(
|
||||
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)]
|
||||
|
||||
@@ -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::<Self>()
|
||||
}
|
||||
}
|
||||
|
||||
pub type CacheInvalidatorRef = Arc<dyn CacheInvalidator>;
|
||||
@@ -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(())
|
||||
|
||||
@@ -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),
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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<WorkerRequest>,
|
||||
pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
|
||||
/// Waiters of the compaction request.
|
||||
pub(crate) waiters: Vec<OutputTx>,
|
||||
/// Start time of compaction task.
|
||||
@@ -101,7 +101,7 @@ pub(crate) struct CompactionScheduler {
|
||||
/// Compacting regions.
|
||||
region_status: HashMap<RegionId, CompactionStatus>,
|
||||
/// Request sender of the worker that this scheduler belongs to.
|
||||
request_sender: Sender<WorkerRequest>,
|
||||
request_sender: Sender<WorkerRequestWithTime>,
|
||||
cache_manager: CacheManagerRef,
|
||||
engine_config: Arc<MitoConfig>,
|
||||
listener: WorkerListener,
|
||||
@@ -112,7 +112,7 @@ pub(crate) struct CompactionScheduler {
|
||||
impl CompactionScheduler {
|
||||
pub(crate) fn new(
|
||||
scheduler: SchedulerRef,
|
||||
request_sender: Sender<WorkerRequest>,
|
||||
request_sender: Sender<WorkerRequestWithTime>,
|
||||
cache_manager: CacheManagerRef,
|
||||
engine_config: Arc<MitoConfig>,
|
||||
listener: WorkerListener,
|
||||
@@ -559,7 +559,7 @@ impl CompactionStatus {
|
||||
#[allow(clippy::too_many_arguments)]
|
||||
fn new_compaction_request(
|
||||
&mut self,
|
||||
request_sender: Sender<WorkerRequest>,
|
||||
request_sender: Sender<WorkerRequestWithTime>,
|
||||
mut waiter: OptionOutputTx,
|
||||
engine_config: Arc<MitoConfig>,
|
||||
cache_manager: CacheManagerRef,
|
||||
|
||||
@@ -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<WorkerRequest>,
|
||||
pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
|
||||
/// Senders that are used to notify waiters waiting for pending compaction tasks.
|
||||
pub waiters: Vec<OutputTx>,
|
||||
/// 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
|
||||
|
||||
@@ -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<OutputTx>,
|
||||
/// Request sender to notify the worker.
|
||||
pub(crate) request_sender: mpsc::Sender<WorkerRequest>,
|
||||
pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
|
||||
|
||||
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
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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<dyn RemoteJobScheduler>;
|
||||
@@ -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<WorkerRequest>,
|
||||
pub(crate) request_sender: Sender<WorkerRequestWithTime>,
|
||||
}
|
||||
|
||||
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!(
|
||||
|
||||
@@ -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<WorkerRequest>,
|
||||
request_sender: Sender<WorkerRequestWithTime>,
|
||||
) -> CompactionScheduler {
|
||||
let scheduler = self.get_scheduler();
|
||||
|
||||
|
||||
@@ -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<S: LogStore> WorkerStarter<S> {
|
||||
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<WorkerRequest>,
|
||||
sender: Sender<WorkerRequestWithTime>,
|
||||
/// Handle to the worker thread.
|
||||
handle: Mutex<Option<JoinHandle<()>>>,
|
||||
/// 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<S> {
|
||||
/// Regions that are opening.
|
||||
opening_regions: OpeningRegionsRef,
|
||||
/// Request sender.
|
||||
sender: Sender<WorkerRequest>,
|
||||
sender: Sender<WorkerRequestWithTime>,
|
||||
/// Request receiver.
|
||||
receiver: Receiver<WorkerRequest>,
|
||||
receiver: Receiver<WorkerRequestWithTime>,
|
||||
/// WAL of the engine.
|
||||
wal: Wal<S>,
|
||||
/// Manages object stores for manifest and SSTs.
|
||||
@@ -706,10 +713,12 @@ struct RegionWorkerLoop<S> {
|
||||
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<S: LogStore> RegionWorkerLoop<S> {
|
||||
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<S: LogStore> RegionWorkerLoop<S> {
|
||||
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,
|
||||
}
|
||||
|
||||
@@ -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<S> RegionWorkerLoop<S> {
|
||||
}),
|
||||
};
|
||||
// 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<S> RegionWorkerLoop<S> {
|
||||
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<S> RegionWorkerLoop<S> {
|
||||
.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
|
||||
|
||||
@@ -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<S: LogStore> RegionWorkerLoop<S> {
|
||||
}
|
||||
|
||||
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<S: LogStore> RegionWorkerLoop<S> {
|
||||
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<S: LogStore> RegionWorkerLoop<S> {
|
||||
/// 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<S: LogStore> RegionWorkerLoop<S> {
|
||||
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<S: LogStore> RegionWorkerLoop<S> {
|
||||
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<S> RegionWorkerLoop<S> {
|
||||
"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<S> RegionWorkerLoop<S> {
|
||||
"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;
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
})?;
|
||||
|
||||
@@ -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<Vec<String>>,
|
||||
column_requirements: HashSet<Column>,
|
||||
alias_tracker: Option<AliasTracker>,
|
||||
/// 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<Column>, 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::<Vec<String>>()
|
||||
.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::<Vec<_>>()
|
||||
.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<AliasMapping> {
|
||||
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<Column>,
|
||||
/// 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<Column>, 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<Column>, 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<Transformed<Self::Node>> {
|
||||
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<Transformed<Self::Node>> {
|
||||
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::<Expr>::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());
|
||||
}
|
||||
}
|
||||
|
||||
1309
src/query/src/dist_plan/analyzer/test.rs
Normal file
1309
src/query/src/dist_plan/analyzer/test.rs
Normal file
File diff suppressed because it is too large
Load Diff
318
src/query/src/dist_plan/analyzer/utils.rs
Normal file
318
src/query/src/dist_plan/analyzer/utils.rs
Normal file
@@ -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<String, HashSet<Column>>;
|
||||
|
||||
/// 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<Self> {
|
||||
if let Some(source) = table_scan
|
||||
.source
|
||||
.as_any()
|
||||
.downcast_ref::<DefaultTableSource>()
|
||||
{
|
||||
if let Some(provider) = source
|
||||
.table_provider
|
||||
.as_any()
|
||||
.downcast_ref::<DfTableProviderAdapter>()
|
||||
{
|
||||
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<Column>> {
|
||||
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<AliasTracker>,
|
||||
mapping_at_each_level: Vec<AliasMapping>,
|
||||
}
|
||||
|
||||
impl TreeNodeVisitor<'_> for TrackerTester {
|
||||
type Node = LogicalPlan;
|
||||
|
||||
fn f_up(&mut self, node: &LogicalPlan) -> DfResult<TreeNodeRecursion> {
|
||||
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([]))
|
||||
])
|
||||
]
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -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<Vec<String>>) -> Commutativity {
|
||||
pub fn check_plan(plan: &LogicalPlan, partition_cols: Option<AliasMapping>) -> 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::<SeriesDivide>().unwrap();
|
||||
let tags = series_divide.tags().iter().collect::<HashSet<_>>();
|
||||
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::<HashSet<_>>();
|
||||
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::<HashSet<_>>();
|
||||
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::<HashSet<_>>();
|
||||
// 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<dyn Fn(&LogicalPlan) -> Option<TransformerAction
|
||||
pub struct TransformerAction {
|
||||
/// list of plans that need to be applied to parent plans, in the order of parent to child.
|
||||
/// i.e. if this returns `[Projection, Aggregate]`, then the parent plan should be transformed to
|
||||
/// ```
|
||||
/// ```ignore
|
||||
/// Original Parent Plan:
|
||||
/// Projection:
|
||||
/// Aggregate:
|
||||
@@ -453,7 +464,7 @@ mod test {
|
||||
fetch: None,
|
||||
});
|
||||
assert!(matches!(
|
||||
Categorizer::check_plan(&plan, Some(vec![])),
|
||||
Categorizer::check_plan(&plan, Some(Default::default())),
|
||||
Commutativity::Commutative
|
||||
));
|
||||
}
|
||||
|
||||
@@ -88,7 +88,11 @@ impl UserDefinedLogicalNodeCore for MergeScanLogicalPlan {
|
||||
}
|
||||
|
||||
fn fmt_for_explain(&self, f: &mut std::fmt::Formatter) -> 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()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<LogicalPlan> {
|
||||
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<LogicalPlan> {
|
||||
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<LogicalPlan> {
|
||||
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<LogicalPlan>`] 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<LogicalPlan> {
|
||||
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 {
|
||||
|
||||
@@ -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_|
|
||||
|
||||
@@ -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
|
||||
|_|_|
|
||||
|
||||
@@ -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
|
||||
|_|_|
|
||||
+-+-+
|
||||
|
||||
@@ -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
|
||||
|_|_|
|
||||
+-+-+
|
||||
|
||||
@@ -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))] |
|
||||
|
||||
974
tests/cases/distributed/explain/step_aggr_advance.result
Normal file
974
tests/cases/distributed/explain/step_aggr_advance.result
Normal file
@@ -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
|
||||
|
||||
307
tests/cases/distributed/explain/step_aggr_advance.sql
Normal file
307
tests/cases/distributed/explain/step_aggr_advance.sql
Normal file
@@ -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;
|
||||
@@ -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_|
|
||||
|
||||
@@ -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(*)]_|
|
||||
|
||||
@@ -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 |
|
||||
+---------------------+-------------------------------------------------------------------+
|
||||
|
||||
|
||||
@@ -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_|
|
||||
|
||||
@@ -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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] |
|
||||
| | |
|
||||
|
||||
@@ -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 |
|
||||
|
||||
160
tests/cases/standalone/common/promql/timestamp_fn.result
Normal file
160
tests/cases/standalone/common/promql/timestamp_fn.result
Normal file
@@ -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
|
||||
|
||||
57
tests/cases/standalone/common/promql/timestamp_fn.sql
Normal file
57
tests/cases/standalone/common/promql/timestamp_fn.sql
Normal file
@@ -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;
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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);
|
||||
|
||||
|
||||
@@ -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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], 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: [<SendableRecordBatchStream>], schema: [Schema { fields: [Field { name: "number", data_type: UInt32, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }], metadata: {"greptime:version": "0"} }] |
|
||||
| | |
|
||||
|
||||
Reference in New Issue
Block a user