feat: support explain analyze verbose (#5763)

* Add explain_verbose to QueryContext

* feat: fmt plan by display type

* feat: update proto to use ExplainOptions

* feat: display more info in verbose mode

* chore: fix clippy

* test: add sqlness test

* test: update sqlness result

* chore: update proto version

* chore: Simplify QueryContextBuilder::explain_options using get_or_insert_default
This commit is contained in:
Yingwen
2025-03-25 11:48:36 +08:00
committed by GitHub
parent 3b547d9d13
commit d88482b996
15 changed files with 261 additions and 28 deletions

2
Cargo.lock generated
View File

@@ -4705,7 +4705,7 @@ dependencies = [
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=a7274ddce299f33d23dbe8af5bbe6219f07c559a#a7274ddce299f33d23dbe8af5bbe6219f07c559a"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=97e298d119fdb9499bc6ba9e03f375cfa7cdf130#97e298d119fdb9499bc6ba9e03f375cfa7cdf130"
dependencies = [
"prost 0.13.3",
"serde",

View File

@@ -129,7 +129,7 @@ etcd-client = "0.14"
fst = "0.4.7"
futures = "0.3"
futures-util = "0.3"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "a7274ddce299f33d23dbe8af5bbe6219f07c559a" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "97e298d119fdb9499bc6ba9e03f375cfa7cdf130" }
hex = "0.4"
http = "1"
humantime = "2.1"

View File

@@ -1240,6 +1240,7 @@ impl From<QueryContext> for PbQueryContext {
extensions,
channel: channel as u32,
snapshot_seqs: None,
explain: None,
}
}
}

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt::Display;
use std::fmt::{self, Display};
use std::future::Future;
use std::marker::PhantomData;
use std::pin::Pin;
@@ -28,7 +28,7 @@ use datafusion::logical_expr::Expr;
use datafusion::physical_expr::create_physical_expr;
use datafusion::physical_plan::metrics::{BaselineMetrics, MetricValue};
use datafusion::physical_plan::{
accept, displayable, ExecutionPlan, ExecutionPlanVisitor, PhysicalExpr,
accept, DisplayFormatType, ExecutionPlan, ExecutionPlanVisitor, PhysicalExpr,
RecordBatchStream as DfRecordBatchStream,
};
use datafusion_common::arrow::error::ArrowError;
@@ -206,13 +206,16 @@ impl Stream for DfRecordBatchStreamAdapter {
}
/// DataFusion [SendableRecordBatchStream](DfSendableRecordBatchStream) -> Greptime [RecordBatchStream].
/// The reverse one is [DfRecordBatchStreamAdapter]
/// The reverse one is [DfRecordBatchStreamAdapter].
/// It can collect metrics from DataFusion execution plan.
pub struct RecordBatchStreamAdapter {
schema: SchemaRef,
stream: DfSendableRecordBatchStream,
metrics: Option<BaselineMetrics>,
/// Aggregated plan-level metrics. Resolved after an [ExecutionPlan] is finished.
metrics_2: Metrics,
/// Display plan and metrics in verbose mode.
explain_verbose: bool,
}
/// Json encoded metrics. Contains metric from a whole plan tree.
@@ -231,6 +234,7 @@ impl RecordBatchStreamAdapter {
stream,
metrics: None,
metrics_2: Metrics::Unavailable,
explain_verbose: false,
})
}
@@ -246,12 +250,18 @@ impl RecordBatchStreamAdapter {
stream,
metrics: Some(metrics),
metrics_2: Metrics::Unresolved(df_plan),
explain_verbose: false,
})
}
pub fn set_metrics2(&mut self, plan: Arc<dyn ExecutionPlan>) {
self.metrics_2 = Metrics::Unresolved(plan)
}
/// Set the verbose mode for displaying plan and metrics.
pub fn set_explain_verbose(&mut self, verbose: bool) {
self.explain_verbose = verbose;
}
}
impl RecordBatchStream for RecordBatchStreamAdapter {
@@ -296,7 +306,7 @@ impl Stream for RecordBatchStreamAdapter {
}
Poll::Ready(None) => {
if let Metrics::Unresolved(df_plan) = &self.metrics_2 {
let mut metric_collector = MetricCollector::default();
let mut metric_collector = MetricCollector::new(self.explain_verbose);
accept(df_plan.as_ref(), &mut metric_collector).unwrap();
self.metrics_2 = Metrics::Resolved(metric_collector.record_batch_metrics);
}
@@ -312,10 +322,20 @@ impl Stream for RecordBatchStreamAdapter {
}
/// An [ExecutionPlanVisitor] to collect metrics from a [ExecutionPlan].
#[derive(Default)]
pub struct MetricCollector {
current_level: usize,
pub record_batch_metrics: RecordBatchMetrics,
verbose: bool,
}
impl MetricCollector {
pub fn new(verbose: bool) -> Self {
Self {
current_level: 0,
record_batch_metrics: RecordBatchMetrics::default(),
verbose,
}
}
}
impl ExecutionPlanVisitor for MetricCollector {
@@ -339,7 +359,7 @@ impl ExecutionPlanVisitor for MetricCollector {
.sorted_for_display()
.timestamps_removed();
let mut plan_metric = PlanMetrics {
plan: displayable(plan).one_line().to_string(),
plan: one_line(plan, self.verbose).to_string(),
level: self.current_level,
metrics: Vec::with_capacity(metric.iter().size_hint().0),
};
@@ -371,6 +391,29 @@ impl ExecutionPlanVisitor for MetricCollector {
}
}
/// Returns a single-line summary of the root of the plan.
/// If the `verbose` flag is set, it will display detailed information about the plan.
fn one_line(plan: &dyn ExecutionPlan, verbose: bool) -> impl fmt::Display + '_ {
struct Wrapper<'a> {
plan: &'a dyn ExecutionPlan,
format_type: DisplayFormatType,
}
impl fmt::Display for Wrapper<'_> {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
self.plan.fmt_as(self.format_type, f)?;
writeln!(f)
}
}
let format_type = if verbose {
DisplayFormatType::Verbose
} else {
DisplayFormatType::Default
};
Wrapper { plan, format_type }
}
/// [`RecordBatchMetrics`] carrys metrics value
/// from datanode to frontend through gRPC
#[derive(serde::Serialize, serde::Deserialize, Default, Debug, Clone)]

View File

@@ -917,7 +917,7 @@ impl StreamContext {
}
/// Format the context for explain.
pub(crate) fn format_for_explain(&self, f: &mut fmt::Formatter) -> fmt::Result {
pub(crate) fn format_for_explain(&self, verbose: bool, f: &mut fmt::Formatter) -> fmt::Result {
let (mut num_mem_ranges, mut num_file_ranges) = (0, 0);
for range_meta in &self.ranges {
for idx in &range_meta.row_group_indices {
@@ -939,8 +939,77 @@ impl StreamContext {
if let Some(selector) = &self.input.series_row_selector {
write!(f, ", selector={}", selector)?;
}
if let Some(distribution) = &self.input.distribution {
write!(f, ", distribution={}", distribution)?;
}
if verbose {
self.format_verbose_content(f)?;
}
Ok(())
}
fn format_verbose_content(&self, f: &mut fmt::Formatter) -> fmt::Result {
struct FileWrapper<'a> {
file: &'a FileHandle,
}
impl fmt::Debug for FileWrapper<'_> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(
f,
"[file={}, time_range=({}::{}, {}::{}), rows={}, size={}, index_size={}]",
self.file.file_id(),
self.file.time_range().0.value(),
self.file.time_range().0.unit(),
self.file.time_range().1.value(),
self.file.time_range().1.unit(),
self.file.num_rows(),
self.file.size(),
self.file.index_size()
)
}
}
struct InputWrapper<'a> {
input: &'a ScanInput,
}
impl fmt::Debug for InputWrapper<'_> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let output_schema = self.input.mapper.output_schema();
if !output_schema.is_empty() {
write!(f, ", projection=")?;
f.debug_list()
.entries(output_schema.column_schemas().iter().map(|col| &col.name))
.finish()?;
}
if let Some(predicate) = &self.input.predicate.predicate() {
if !predicate.exprs().is_empty() {
write!(f, ", filters=[")?;
for (i, expr) in predicate.exprs().iter().enumerate() {
if i == predicate.exprs().len() - 1 {
write!(f, "{}]", expr)?;
} else {
write!(f, "{}, ", expr)?;
}
}
}
}
if !self.input.files.is_empty() {
write!(f, ", files=")?;
f.debug_list()
.entries(self.input.files.iter().map(|file| FileWrapper { file }))
.finish()?;
}
Ok(())
}
}
write!(f, "{:?}", InputWrapper { input: &self.input })
}
}
/// Predicates to evaluate.

View File

@@ -458,13 +458,16 @@ impl RegionScanner for SeqScan {
}
impl DisplayAs for SeqScan {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
write!(
f,
"SeqScan: region={}, ",
self.stream_ctx.input.mapper.metadata().region_id
)?;
self.stream_ctx.format_for_explain(f)
match t {
DisplayFormatType::Default => self.stream_ctx.format_for_explain(false, f),
DisplayFormatType::Verbose => self.stream_ctx.format_for_explain(true, f),
}
}
}

View File

@@ -254,13 +254,16 @@ impl RegionScanner for UnorderedScan {
}
impl DisplayAs for UnorderedScan {
fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
write!(
f,
"UnorderedScan: region={}, ",
self.stream_ctx.input.mapper.metadata().region_id
)?;
self.stream_ctx.format_for_explain(f)
match t {
DisplayFormatType::Default => self.stream_ctx.format_for_explain(false, f),
DisplayFormatType::Verbose => self.stream_ctx.format_for_explain(true, f),
}
}
}

View File

@@ -248,6 +248,10 @@ impl FileHandle {
self.inner.meta.file_size
}
pub fn index_size(&self) -> u64 {
self.inner.meta.index_file_size
}
pub fn num_rows(&self) -> usize {
self.inner.meta.num_rows as usize
}

View File

@@ -50,12 +50,13 @@ pub struct DistAnalyzeExec {
input: Arc<dyn ExecutionPlan>,
schema: SchemaRef,
properties: PlanProperties,
verbose: bool,
format: AnalyzeFormat,
}
impl DistAnalyzeExec {
/// Create a new DistAnalyzeExec
pub fn new(input: Arc<dyn ExecutionPlan>, format: AnalyzeFormat) -> Self {
pub fn new(input: Arc<dyn ExecutionPlan>, verbose: bool, format: AnalyzeFormat) -> Self {
let schema = SchemaRef::new(Schema::new(vec![
Field::new(STAGE, DataType::UInt32, true),
Field::new(NODE, DataType::UInt32, true),
@@ -66,6 +67,7 @@ impl DistAnalyzeExec {
input,
schema,
properties,
verbose,
format,
}
}
@@ -116,7 +118,11 @@ impl ExecutionPlan for DistAnalyzeExec {
self: Arc<Self>,
mut children: Vec<Arc<dyn ExecutionPlan>>,
) -> DfResult<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(Self::new(children.pop().unwrap(), self.format)))
Ok(Arc::new(Self::new(
children.pop().unwrap(),
self.verbose,
self.format,
)))
}
fn execute(
@@ -138,6 +144,7 @@ impl ExecutionPlan for DistAnalyzeExec {
// Finish the input stream and create the output
let format = self.format;
let verbose = self.verbose;
let mut input_stream = coalesce_partition_plan.execute(0, context)?;
let output = async move {
let mut total_rows = 0;
@@ -145,7 +152,7 @@ impl ExecutionPlan for DistAnalyzeExec {
total_rows += batch.num_rows();
}
create_output_batch(total_rows, captured_input, captured_schema, format)
create_output_batch(total_rows, captured_input, captured_schema, format, verbose)
};
Ok(Box::pin(RecordBatchStreamAdapter::new(
@@ -205,11 +212,12 @@ fn create_output_batch(
input: Arc<dyn ExecutionPlan>,
schema: SchemaRef,
format: AnalyzeFormat,
verbose: bool,
) -> DfResult<DfRecordBatch> {
let mut builder = AnalyzeOutputBuilder::new(schema);
// Treat the current stage as stage 0. Fetch its metrics
let mut collector = MetricCollector::default();
let mut collector = MetricCollector::new(verbose);
// Safety: metric collector won't return error
accept(input.as_ref(), &mut collector).unwrap();
let stage_0_metrics = collector.record_batch_metrics;

View File

@@ -367,8 +367,15 @@ impl DatafusionQueryEngine {
} else {
AnalyzeFormat::TEXT
};
// Sets the verbose flag of the query context.
// The MergeScanExec plan uses the verbose flag to determine whether to print the plan in verbose mode.
ctx.query_ctx().set_explain_verbose(analyze_plan.verbose());
Arc::new(DistAnalyzeExec::new(analyze_plan.input().clone(), format))
Arc::new(DistAnalyzeExec::new(
analyze_plan.input().clone(),
analyze_plan.verbose(),
format,
))
// let mut new_plan = analyze_plan.input().clone();
// for optimizer in state.physical_optimizers() {
// new_plan = optimizer
@@ -511,6 +518,7 @@ impl QueryExecutor for DatafusionQueryEngine {
.map_err(BoxedError::new)
.context(QueryExecutionSnafu)?;
stream.set_metrics2(plan.clone());
stream.set_explain_verbose(ctx.query_ctx().explain_verbose());
let stream = OnDone::new(Box::pin(stream), move || {
exec_timer.observe_duration();
});
@@ -537,6 +545,7 @@ impl QueryExecutor for DatafusionQueryEngine {
.map_err(BoxedError::new)
.context(QueryExecutionSnafu)?;
stream.set_metrics2(plan.clone());
stream.set_explain_verbose(ctx.query_ctx().explain_verbose());
let stream = OnDone::new(Box::pin(stream), move || {
exec_timer.observe_duration();
});

View File

@@ -19,6 +19,7 @@ use std::sync::{Arc, RwLock};
use std::time::Duration;
use api::v1::region::RegionRequestHeader;
use api::v1::ExplainOptions;
use arc_swap::ArcSwap;
use auth::UserInfoRef;
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
@@ -69,6 +70,8 @@ pub struct QueryContextMutableFields {
warning: Option<String>,
// TODO: remove this when format is supported in datafusion
explain_format: Option<String>,
/// Explain options to control the verbose analyze output.
explain_options: Option<ExplainOptions>,
}
impl Display for QueryContext {
@@ -111,6 +114,15 @@ impl QueryContextBuilder {
.timezone = timezone;
self
}
pub fn explain_options(mut self, explain_options: Option<ExplainOptions>) -> Self {
self.mutable_query_context_data
.get_or_insert_default()
.write()
.unwrap()
.explain_options = explain_options;
self
}
}
impl From<&RegionRequestHeader> for QueryContext {
@@ -125,7 +137,8 @@ impl From<&RegionRequestHeader> for QueryContext {
.channel(ctx.channel.into())
.snapshot_seqs(Arc::new(RwLock::new(
ctx.snapshot_seqs.clone().unwrap_or_default().snapshot_seqs,
)));
)))
.explain_options(ctx.explain);
}
builder.build()
}
@@ -142,6 +155,7 @@ impl From<api::v1::QueryContext> for QueryContext {
.snapshot_seqs(Arc::new(RwLock::new(
ctx.snapshot_seqs.clone().unwrap_or_default().snapshot_seqs,
)))
.explain_options(ctx.explain)
.build()
}
}
@@ -154,9 +168,11 @@ impl From<QueryContext> for api::v1::QueryContext {
extensions,
channel,
snapshot_seqs,
mutable_query_context_data,
..
}: QueryContext,
) -> Self {
let explain = mutable_query_context_data.read().unwrap().explain_options;
let mutable_inner = mutable_inner.read().unwrap();
api::v1::QueryContext {
current_catalog,
@@ -167,6 +183,7 @@ impl From<QueryContext> for api::v1::QueryContext {
snapshot_seqs: Some(api::v1::SnapshotSequences {
snapshot_seqs: snapshot_seqs.read().unwrap().clone(),
}),
explain,
}
}
}
@@ -319,6 +336,24 @@ impl QueryContext {
.explain_format = Some(format);
}
pub fn explain_verbose(&self) -> bool {
self.mutable_query_context_data
.read()
.unwrap()
.explain_options
.map(|opts| opts.verbose)
.unwrap_or(false)
}
pub fn set_explain_verbose(&self, verbose: bool) {
self.mutable_query_context_data
.write()
.unwrap()
.explain_options
.get_or_insert_default()
.verbose = verbose;
}
pub fn query_timeout(&self) -> Option<Duration> {
self.mutable_session_data.read().unwrap().query_timeout
}

View File

@@ -284,6 +284,27 @@ EXPLAIN ANALYZE SELECT * FROM test_pk ORDER BY t LIMIT 5;
|_|_| Total rows: 5_|
+-+-+-+
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
-- SQLNESS REPLACE (files.*) REDACTED
EXPLAIN ANALYZE VERBOSE SELECT * FROM test_pk ORDER BY t LIMIT 5;
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortPreservingMergeExec: [t@2 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 ASC NULLS LAST num_ranges=4 fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 ASC NULLS LAST num_ranges=4 limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges), projection=["pk", "i", "t"], REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
SELECT * FROM test_pk ORDER BY t DESC LIMIT 5;
+----+---+-------------------------+
@@ -349,6 +370,27 @@ EXPLAIN ANALYZE SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
|_|_| Total rows: 5_|
+-+-+-+
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
-- SQLNESS REPLACE (files.*) REDACTED
EXPLAIN ANALYZE VERBOSE SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortPreservingMergeExec: [t@2 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 ASC NULLS LAST num_ranges=4 fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 ASC NULLS LAST num_ranges=4 limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges), projection=["pk", "i", "t"], filters=[pk > Int32(7)], REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
DROP TABLE test_pk;
Affected Rows: 0

View File

@@ -91,6 +91,14 @@ SELECT * FROM test_pk ORDER BY t LIMIT 5;
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
EXPLAIN ANALYZE SELECT * FROM test_pk ORDER BY t LIMIT 5;
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
-- SQLNESS REPLACE (files.*) REDACTED
EXPLAIN ANALYZE VERBOSE SELECT * FROM test_pk ORDER BY t LIMIT 5;
SELECT * FROM test_pk ORDER BY t DESC LIMIT 5;
-- SQLNESS REPLACE (-+) -
@@ -110,4 +118,12 @@ SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
EXPLAIN ANALYZE SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
-- SQLNESS REPLACE (files.*) REDACTED
EXPLAIN ANALYZE VERBOSE SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
DROP TABLE test_pk;

View File

@@ -25,7 +25,7 @@ tql analyze (1, 3, '1s') t1{ a = "a" };
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[a@0 DESC NULLS LAST, b@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 3_|
+-+-+-+
@@ -49,7 +49,7 @@ tql analyze (1, 3, '1s') t1{ a =~ ".*" };
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[a@0 DESC NULLS LAST, b@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 6_|
+-+-+-+
@@ -73,7 +73,7 @@ tql analyze (1, 3, '1s') t1{ a =~ "a.*" };
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[a@0 DESC NULLS LAST, b@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 3_|
+-+-+-+

View File

@@ -27,7 +27,7 @@ TQL ANALYZE (0, 10, '5s') test;
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[k@2 DESC NULLS LAST, j@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -53,7 +53,7 @@ TQL ANALYZE (0, 10, '1s', '2s') test;
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[k@2 DESC NULLS LAST, j@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -78,7 +78,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[k@2 DESC NULLS LAST, j@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -105,7 +105,7 @@ TQL ANALYZE VERBOSE (0, 10, '5s') test;
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[k@2 DESC NULLS LAST, j@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries, projection=["i", "j", "k"], filters=[j >= TimestampMillisecond(-300000, None), j <= TimestampMillisecond(310000, None)] REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -138,10 +138,10 @@ TQL ANALYZE (0, 10, '5s') test;
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortExec: expr=[k@2 DESC NULLS LAST, l@3 DESC NULLS LAST, j@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
| 1_| 1_|_SortExec: expr=[k@2 DESC NULLS LAST, l@3 DESC NULLS LAST, j@1 DESC NULLS LAST], preserve_partitioning=[false] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+