mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-04 04:12:55 +00:00
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:
2
Cargo.lock
generated
2
Cargo.lock
generated
@@ -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",
|
||||
|
||||
@@ -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"
|
||||
|
||||
@@ -1240,6 +1240,7 @@ impl From<QueryContext> for PbQueryContext {
|
||||
extensions,
|
||||
channel: channel as u32,
|
||||
snapshot_seqs: None,
|
||||
explain: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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)]
|
||||
|
||||
@@ -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.
|
||||
|
||||
@@ -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),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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();
|
||||
});
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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_|
|
||||
+-+-+-+
|
||||
|
||||
@@ -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_|
|
||||
+-+-+-+
|
||||
|
||||
Reference in New Issue
Block a user