Compare commits

...

15 Commits

Author SHA1 Message Date
Ruihang Xia
51454c93d9 feat: organize EXPLAIN ANALYZE VERBOSE's output in JSON format (#6308)
Signed-off-by: Ruihang Xia <waynestxia@gmail.com>
2025-06-13 15:59:51 +08:00
discord9
5dd6f92b60 feat: retry with smaller time when failed query
Signed-off-by: discord9 <discord9@163.com>
2025-06-12 21:24:31 +08:00
discord9
11e4a8abb3 feat: flow query parallel=1&query faster when
windows too many&min one time window

Signed-off-by: discord9 <discord9@163.com>
2025-06-12 21:24:31 +08:00
discord9
a9cbcbb0c8 feat: parallelism hint in grpc (#6306)
* feat: parallelism hint in grpc

Signed-off-by: discord9 <discord9@163.com>

* chore: per review

Signed-off-by: discord9 <discord9@163.com>

* chore: comment

Signed-off-by: discord9 <discord9@163.com>

* chore:docs

Signed-off-by: discord9 <discord9@163.com>

---------

Signed-off-by: discord9 <discord9@163.com>

fix: check for zero parallelism

Signed-off-by: discord9 <discord9@163.com>
2025-06-12 19:20:03 +08:00
discord9
ebd815d288 fix: case when can push down 2025-06-12 15:17:24 +08:00
discord9
403958f5ba feat: steppable aggr fn dist push down
Signed-off-by: discord9 <discord9@163.com>

poc: step aggr query

Signed-off-by: discord9 <discord9@163.com>

feat: mvp poc stuff

Signed-off-by: discord9 <discord9@163.com>

test: sqlness

Signed-off-by: discord9 <discord9@163.com>

chore: import missing

Signed-off-by: discord9 <discord9@163.com>

feat: support first/last_value

Signed-off-by: discord9 <discord9@163.com>

fix: check also include first/last value

Signed-off-by: discord9 <discord9@163.com>

chore: clean up after rebase

feat: optimize yes!

fix: alias qualifled

test: more testcases

chore: qualified column

chore: per review
2025-06-12 13:26:58 +08:00
discord9
ffe84af343 chore: added stalled time window range
Signed-off-by: discord9 <discord9@163.com>
2025-06-11 20:26:46 +08:00
discord9
41c40688c4 feat: adjust flow 2025-06-11 15:40:59 +08:00
discord9
59a82a3f29 feat: truly limit time range by split window 2025-06-11 15:40:59 +08:00
discord9
0d7012a1ea feat: flownode to frontend load balance with guess 2025-06-11 15:40:59 +08:00
discord9
88da98c829 metrics: more useful metrics batching mode 2025-06-11 15:40:59 +08:00
discord9
73ce5914f2 feat: metrics 2025-06-11 15:40:59 +08:00
discord9
f5eac3528c feat: mark dirty time window 2025-06-11 15:40:59 +08:00
Lei, HUANG
9cd61d221d feat/bulk-support-flow-batch:
### Add Dirty Window Handling in Flow Module

 - **Updated `greptime-proto` Dependency**: Updated the `greptime-proto` dependency to a new revision in `Cargo.lock` and `Cargo.toml`.
 - **Flow Module Enhancements**:
   - Added `DirtyWindowRequest` handling in `flow.rs`, `node_manager.rs`, `test_util.rs`, `flownode_impl.rs`, and `server.rs`.
   - Implemented `handle_mark_window_dirty` function to manage dirty time windows.
 - **Bulk Insert Enhancements**:
   - Modified `bulk_insert.rs` to notify flownodes about dirty time windows using `update_flow_dirty_window`.
 - **Removed Unused Imports**: Cleaned up unused imports in `greptime_handler.rs`, `grpc.rs`, and `mod.rs`.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
2025-06-11 15:40:59 +08:00
Lei, HUANG
7527ff976e feat/bulk-support-flow-batch:
### Refactor and Enhance Timestamp Handling in gRPC and Bulk Insert

 - **Refactor Table Handling**:
   - Updated `put_record_batch` method to use `TableRef` instead of `TableId` in `grpc.rs`, `greptime_handler.rs`, and `grpc.rs`.
   - Modified `handle_bulk_insert` to accept `TableRef` and extract `TableId` internally in `bulk_insert.rs`.

 - **Enhance Timestamp Processing**:
   - Added `compute_timestamp_range` function to calculate timestamp range in `bulk_insert.rs`.
   - Introduced error handling for invalid time index types in `error.rs`.

 - **Test Adjustments**:
   - Updated `DummyInstance` implementation in `tests/mod.rs` to align with new method signatures.

Signed-off-by: Lei, HUANG <mrsatangel@gmail.com>
2025-06-11 15:40:59 +08:00
61 changed files with 2890 additions and 329 deletions

2
Cargo.lock generated
View File

@@ -5133,7 +5133,7 @@ dependencies = [
[[package]]
name = "greptime-proto"
version = "0.1.0"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=454c52634c3bac27de10bf0d85d5533eed1cf03f#454c52634c3bac27de10bf0d85d5533eed1cf03f"
source = "git+https://github.com/GreptimeTeam/greptime-proto.git?rev=52083925a15d741c259800a9a54eba3467939180#52083925a15d741c259800a9a54eba3467939180"
dependencies = [
"prost 0.13.5",
"serde",

View File

@@ -133,7 +133,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 = "454c52634c3bac27de10bf0d85d5533eed1cf03f" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "52083925a15d741c259800a9a54eba3467939180" }
hex = "0.4"
http = "1"
humantime = "2.1"

View File

@@ -196,12 +196,22 @@ impl Database {
/// Retry if connection fails, max_retries is the max number of retries, so the total wait time
/// is `max_retries * GRPC_CONN_TIMEOUT`
pub async fn handle_with_retry(&self, request: Request, max_retries: u32) -> Result<u32> {
pub async fn handle_with_retry(
&self,
request: Request,
max_retries: u32,
hints: &[(&str, &str)],
) -> Result<u32> {
let mut client = make_database_client(&self.client)?.inner;
let mut retries = 0;
let request = self.to_rpc_request(request);
loop {
let raw_response = client.handle(request.clone()).await;
let mut tonic_request = tonic::Request::new(request.clone());
let metadata = tonic_request.metadata_mut();
Self::put_hints(metadata, hints)?;
let raw_response = client.handle(tonic_request).await;
match (raw_response, retries < max_retries) {
(Ok(resp), _) => return from_grpc_response(resp.into_inner()),
(Err(err), true) => {

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use api::v1::flow::{FlowRequest, FlowResponse};
use api::v1::flow::{DirtyWindowRequest, DirtyWindowRequests, FlowRequest, FlowResponse};
use api::v1::region::InsertRequests;
use common_error::ext::BoxedError;
use common_meta::node_manager::Flownode;
@@ -44,6 +44,16 @@ impl Flownode for FlowRequester {
.map_err(BoxedError::new)
.context(common_meta::error::ExternalSnafu)
}
async fn handle_mark_window_dirty(
&self,
req: DirtyWindowRequest,
) -> common_meta::error::Result<FlowResponse> {
self.handle_mark_window_dirty(req)
.await
.map_err(BoxedError::new)
.context(common_meta::error::ExternalSnafu)
}
}
impl FlowRequester {
@@ -91,4 +101,20 @@ impl FlowRequester {
.into_inner();
Ok(response)
}
async fn handle_mark_window_dirty(&self, req: DirtyWindowRequest) -> Result<FlowResponse> {
let (addr, mut client) = self.client.raw_flow_client()?;
let response = client
.handle_mark_dirty_time_window(DirtyWindowRequests {
requests: vec![req],
})
.await
.or_else(|e| {
let code = e.code();
let err: crate::error::Error = e.into();
Err(BoxedError::new(err)).context(FlowServerSnafu { addr, code })
})?
.into_inner();
Ok(response)
}
}

View File

@@ -0,0 +1,90 @@
// 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 common_macro::admin_fn;
use common_query::error::{
InvalidFuncArgsSnafu, MissingFlowServiceHandlerSnafu, Result, UnsupportedInputDataTypeSnafu,
};
use common_query::prelude::Signature;
use datafusion::logical_expr::Volatility;
use datatypes::value::{Value, ValueRef};
use session::context::QueryContextRef;
use snafu::ensure;
use store_api::storage::ConcreteDataType;
use crate::handlers::FlowServiceHandlerRef;
use crate::helper::parse_catalog_flow;
fn adjust_signature() -> Signature {
Signature::exact(
vec![
ConcreteDataType::string_datatype(), // flow name
ConcreteDataType::uint64_datatype(), // min_run_interval in seconds
ConcreteDataType::uint64_datatype(), // max filter number per query
],
Volatility::Immutable,
)
}
#[admin_fn(
name = AdjustFlowFunction,
display_name = adjust_flow,
sig_fn = adjust_signature,
ret = uint64
)]
pub(crate) async fn adjust_flow(
flow_service_handler: &FlowServiceHandlerRef,
query_ctx: &QueryContextRef,
params: &[ValueRef<'_>],
) -> Result<Value> {
ensure!(
params.len() == 3,
InvalidFuncArgsSnafu {
err_msg: format!(
"The length of the args is not correct, expect 3, have: {}",
params.len()
),
}
);
let (flow_name, min_run_interval, max_filter_num) = match (params[0], params[1], params[2]) {
(
ValueRef::String(flow_name),
ValueRef::UInt64(min_run_interval),
ValueRef::UInt64(max_filter_num),
) => (flow_name, min_run_interval, max_filter_num),
_ => {
return UnsupportedInputDataTypeSnafu {
function: "adjust_flow",
datatypes: params.iter().map(|v| v.data_type()).collect::<Vec<_>>(),
}
.fail();
}
};
let (catalog_name, flow_name) = parse_catalog_flow(flow_name, query_ctx)?;
let res = flow_service_handler
.adjust(
&catalog_name,
&flow_name,
min_run_interval,
max_filter_num as usize,
query_ctx.clone(),
)
.await?;
let affected_rows = res.affected_rows;
Ok(Value::from(affected_rows))
}

View File

@@ -26,6 +26,7 @@ use flush_compact_table::{CompactTableFunction, FlushTableFunction};
use migrate_region::MigrateRegionFunction;
use remove_region_follower::RemoveRegionFollowerFunction;
use crate::adjust_flow::AdjustFlowFunction;
use crate::flush_flow::FlushFlowFunction;
use crate::function_registry::FunctionRegistry;
@@ -43,5 +44,6 @@ impl AdminFunction {
registry.register_async(Arc::new(FlushTableFunction));
registry.register_async(Arc::new(CompactTableFunction));
registry.register_async(Arc::new(FlushFlowFunction));
registry.register_async(Arc::new(AdjustFlowFunction));
}
}

View File

@@ -14,8 +14,8 @@
use crate::function_registry::FunctionRegistry;
pub(crate) mod hll;
mod uddsketch;
pub mod hll;
pub mod uddsketch;
pub(crate) struct ApproximateFunction;

View File

@@ -12,21 +12,19 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use common_error::ext::BoxedError;
use common_macro::admin_fn;
use common_query::error::{
ExecuteSnafu, InvalidFuncArgsSnafu, MissingFlowServiceHandlerSnafu, Result,
UnsupportedInputDataTypeSnafu,
InvalidFuncArgsSnafu, MissingFlowServiceHandlerSnafu, Result, UnsupportedInputDataTypeSnafu,
};
use common_query::prelude::Signature;
use datafusion::logical_expr::Volatility;
use datatypes::value::{Value, ValueRef};
use session::context::QueryContextRef;
use snafu::{ensure, ResultExt};
use sql::parser::ParserContext;
use snafu::ensure;
use store_api::storage::ConcreteDataType;
use crate::handlers::FlowServiceHandlerRef;
use crate::helper::parse_catalog_flow;
fn flush_signature() -> Signature {
Signature::uniform(
@@ -47,20 +45,6 @@ pub(crate) async fn flush_flow(
query_ctx: &QueryContextRef,
params: &[ValueRef<'_>],
) -> Result<Value> {
let (catalog_name, flow_name) = parse_flush_flow(params, query_ctx)?;
let res = flow_service_handler
.flush(&catalog_name, &flow_name, query_ctx.clone())
.await?;
let affected_rows = res.affected_rows;
Ok(Value::from(affected_rows))
}
fn parse_flush_flow(
params: &[ValueRef<'_>],
query_ctx: &QueryContextRef,
) -> Result<(String, String)> {
ensure!(
params.len() == 1,
InvalidFuncArgsSnafu {
@@ -70,7 +54,6 @@ fn parse_flush_flow(
),
}
);
let ValueRef::String(flow_name) = params[0] else {
return UnsupportedInputDataTypeSnafu {
function: "flush_flow",
@@ -78,27 +61,14 @@ fn parse_flush_flow(
}
.fail();
};
let obj_name = ParserContext::parse_table_name(flow_name, query_ctx.sql_dialect())
.map_err(BoxedError::new)
.context(ExecuteSnafu)?;
let (catalog_name, flow_name) = parse_catalog_flow(flow_name, query_ctx)?;
let (catalog_name, flow_name) = match &obj_name.0[..] {
[flow_name] => (
query_ctx.current_catalog().to_string(),
flow_name.value.clone(),
),
[catalog, flow_name] => (catalog.value.clone(), flow_name.value.clone()),
_ => {
return InvalidFuncArgsSnafu {
err_msg: format!(
"expect flow name to be <catalog>.<flow-name> or <flow-name>, actual: {}",
obj_name
),
}
.fail()
}
};
Ok((catalog_name, flow_name))
let res = flow_service_handler
.flush(&catalog_name, &flow_name, query_ctx.clone())
.await?;
let affected_rows = res.affected_rows;
Ok(Value::from(affected_rows))
}
#[cfg(test)]
@@ -154,10 +124,7 @@ mod test {
("catalog.flow_name", ("catalog", "flow_name")),
];
for (input, expected) in testcases.iter() {
let args = vec![*input];
let args = args.into_iter().map(ValueRef::String).collect::<Vec<_>>();
let result = parse_flush_flow(&args, &QueryContext::arc()).unwrap();
let result = parse_catalog_flow(input, &QueryContext::arc()).unwrap();
assert_eq!(*expected, (result.0.as_str(), result.1.as_str()));
}
}

View File

@@ -87,6 +87,15 @@ pub trait FlowServiceHandler: Send + Sync {
flow: &str,
ctx: QueryContextRef,
) -> Result<api::v1::flow::FlowResponse>;
async fn adjust(
&self,
catalog: &str,
flow: &str,
min_run_interval_secs: u64,
max_filter_num_per_query: usize,
ctx: QueryContextRef,
) -> Result<api::v1::flow::FlowResponse>;
}
pub type TableMutationHandlerRef = Arc<dyn TableMutationHandler>;

View File

@@ -12,12 +12,15 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use common_query::error::{InvalidInputTypeSnafu, Result};
use common_error::ext::BoxedError;
use common_query::error::{ExecuteSnafu, InvalidFuncArgsSnafu, InvalidInputTypeSnafu, Result};
use common_query::prelude::{Signature, TypeSignature, Volatility};
use datatypes::prelude::ConcreteDataType;
use datatypes::types::cast::cast;
use datatypes::value::ValueRef;
use session::context::QueryContextRef;
use snafu::ResultExt;
use sql::parser::ParserContext;
/// Create a function signature with oneof signatures of interleaving two arguments.
pub fn one_of_sigs2(args1: Vec<ConcreteDataType>, args2: Vec<ConcreteDataType>) -> Signature {
@@ -43,3 +46,30 @@ pub fn cast_u64(value: &ValueRef) -> Result<Option<u64>> {
})
.map(|v| v.as_u64())
}
pub fn parse_catalog_flow(
flow_name: &str,
query_ctx: &QueryContextRef,
) -> Result<(String, String)> {
let obj_name = ParserContext::parse_table_name(flow_name, query_ctx.sql_dialect())
.map_err(BoxedError::new)
.context(ExecuteSnafu)?;
let (catalog_name, flow_name) = match &obj_name.0[..] {
[flow_name] => (
query_ctx.current_catalog().to_string(),
flow_name.value.clone(),
),
[catalog, flow_name] => (catalog.value.clone(), flow_name.value.clone()),
_ => {
return InvalidFuncArgsSnafu {
err_msg: format!(
"expect flow name to be <catalog>.<flow-name> or <flow-name>, actual: {}",
obj_name
),
}
.fail()
}
};
Ok((catalog_name, flow_name))
}

View File

@@ -15,6 +15,7 @@
#![feature(let_chains)]
#![feature(try_blocks)]
mod adjust_flow;
mod admin;
mod flush_flow;
mod macros;

View File

@@ -148,6 +148,17 @@ impl FunctionState {
) -> Result<api::v1::flow::FlowResponse> {
todo!()
}
async fn adjust(
&self,
_catalog: &str,
_flow: &str,
_min_run_interval_secs: u64,
_max_filter_num_per_query: usize,
_ctx: QueryContextRef,
) -> Result<api::v1::flow::FlowResponse> {
todo!()
}
}
Self {

View File

@@ -15,7 +15,7 @@
use std::sync::Arc;
use api::region::RegionResponse;
use api::v1::flow::{FlowRequest, FlowResponse};
use api::v1::flow::{DirtyWindowRequest, FlowRequest, FlowResponse};
use api::v1::region::{InsertRequests, RegionRequest};
pub use common_base::AffectedRows;
use common_query::request::QueryRequest;
@@ -42,6 +42,9 @@ pub trait Flownode: Send + Sync {
async fn handle(&self, request: FlowRequest) -> Result<FlowResponse>;
async fn handle_inserts(&self, request: InsertRequests) -> Result<FlowResponse>;
/// Handles requests to mark time window as dirty.
async fn handle_mark_window_dirty(&self, req: DirtyWindowRequest) -> Result<FlowResponse>;
}
pub type FlownodeRef = Arc<dyn Flownode>;

View File

@@ -15,7 +15,7 @@
use std::sync::Arc;
use api::region::RegionResponse;
use api::v1::flow::{FlowRequest, FlowResponse};
use api::v1::flow::{DirtyWindowRequest, FlowRequest, FlowResponse};
use api::v1::region::{InsertRequests, RegionRequest};
pub use common_base::AffectedRows;
use common_query::request::QueryRequest;
@@ -67,6 +67,14 @@ pub trait MockFlownodeHandler: Sync + Send + Clone {
) -> Result<FlowResponse> {
unimplemented!()
}
async fn handle_mark_window_dirty(
&self,
_peer: &Peer,
_req: DirtyWindowRequest,
) -> Result<FlowResponse> {
unimplemented!()
}
}
/// A mock struct implements [NodeManager] only implement the `datanode` method.
@@ -134,6 +142,10 @@ impl<T: MockFlownodeHandler> Flownode for MockNode<T> {
async fn handle_inserts(&self, requests: InsertRequests) -> Result<FlowResponse> {
self.handler.handle_inserts(&self.peer, requests).await
}
async fn handle_mark_window_dirty(&self, req: DirtyWindowRequest) -> Result<FlowResponse> {
self.handler.handle_mark_window_dirty(&self.peer, req).await
}
}
#[async_trait::async_trait]

View File

@@ -61,6 +61,7 @@ prost.workspace = true
query.workspace = true
rand.workspace = true
serde.workspace = true
serde_json.workspace = true
servers.workspace = true
session.workspace = true
smallvec.workspace = true

View File

@@ -316,7 +316,7 @@ impl StreamingEngine {
);
METRIC_FLOW_ROWS
.with_label_values(&["out"])
.with_label_values(&["out-streaming"])
.inc_by(total_rows as u64);
let now = self.tick_manager.tick();

View File

@@ -18,7 +18,7 @@ use std::sync::atomic::AtomicBool;
use std::sync::Arc;
use api::v1::flow::{
flow_request, CreateRequest, DropRequest, FlowRequest, FlowResponse, FlushFlow,
flow_request, AdjustFlow, CreateRequest, DropRequest, FlowRequest, FlowResponse, FlushFlow,
};
use api::v1::region::InsertRequests;
use catalog::CatalogManager;
@@ -31,7 +31,9 @@ use common_runtime::JoinHandle;
use common_telemetry::{error, info, trace, warn};
use datatypes::value::Value;
use futures::TryStreamExt;
use greptime_proto::v1::flow::DirtyWindowRequest;
use itertools::Itertools;
use serde::{Deserialize, Serialize};
use session::context::QueryContextBuilder;
use snafu::{ensure, IntoError, OptionExt, ResultExt};
use store_api::storage::{RegionId, TableId};
@@ -46,7 +48,7 @@ use crate::error::{
IllegalCheckTaskStateSnafu, InsertIntoFlowSnafu, InternalSnafu, JoinTaskSnafu, ListFlowsSnafu,
NoAvailableFrontendSnafu, SyncCheckTaskSnafu, UnexpectedSnafu,
};
use crate::metrics::METRIC_FLOW_TASK_COUNT;
use crate::metrics::{METRIC_FLOW_ROWS, METRIC_FLOW_TASK_COUNT};
use crate::repr::{self, DiffRow};
use crate::{Error, FlowId};
@@ -689,6 +691,9 @@ impl FlowEngine for FlowDualEngine {
let mut to_stream_engine = Vec::with_capacity(request.requests.len());
let mut to_batch_engine = request.requests;
let mut batching_row_cnt = 0;
let mut streaming_row_cnt = 0;
{
// not locking this, or recover flows will be starved when also handling flow inserts
let src_table2flow = self.src_table2flow.read().await;
@@ -698,9 +703,11 @@ impl FlowEngine for FlowDualEngine {
let is_in_stream = src_table2flow.in_stream(table_id);
let is_in_batch = src_table2flow.in_batch(table_id);
if is_in_stream {
streaming_row_cnt += req.rows.as_ref().map(|rs| rs.rows.len()).unwrap_or(0);
to_stream_engine.push(req.clone());
}
if is_in_batch {
batching_row_cnt += req.rows.as_ref().map(|rs| rs.rows.len()).unwrap_or(0);
return true;
}
if !is_in_batch && !is_in_stream {
@@ -713,6 +720,14 @@ impl FlowEngine for FlowDualEngine {
// can't use drop due to https://github.com/rust-lang/rust/pull/128846
}
METRIC_FLOW_ROWS
.with_label_values(&["in-streaming"])
.inc_by(streaming_row_cnt as u64);
METRIC_FLOW_ROWS
.with_label_values(&["in-batching"])
.inc_by(batching_row_cnt as u64);
let streaming_engine = self.streaming_engine.clone();
let stream_handler: JoinHandle<Result<(), Error>> =
common_runtime::spawn_global(async move {
@@ -809,6 +824,25 @@ impl common_meta::node_manager::Flownode for FlowDualEngine {
..Default::default()
})
}
Some(flow_request::Body::Adjust(AdjustFlow { flow_id, options })) => {
#[derive(Debug, Serialize, Deserialize)]
struct Options {
min_run_interval_secs: u64,
max_filter_num_per_query: usize,
}
let options: Options = serde_json::from_str(&options).with_context(|_| {
common_meta::error::DeserializeFromJsonSnafu { input: options }
})?;
self.batching_engine
.adjust_flow(
flow_id.unwrap().id as u64,
options.min_run_interval_secs,
options.max_filter_num_per_query,
)
.await
.map_err(to_meta_err(snafu::location!()))?;
Ok(Default::default())
}
other => common_meta::error::InvalidFlowRequestBodySnafu { body: other }.fail(),
}
}
@@ -819,6 +853,11 @@ impl common_meta::node_manager::Flownode for FlowDualEngine {
.map(|_| Default::default())
.map_err(to_meta_err(snafu::location!()))
}
async fn handle_mark_window_dirty(&self, _req: DirtyWindowRequest) -> MetaResult<FlowResponse> {
// todo: implement
unimplemented!()
}
}
/// return a function to convert `crate::error::Error` to `common_meta::error::Error`
@@ -926,6 +965,11 @@ impl common_meta::node_manager::Flownode for StreamingEngine {
.map(|_| Default::default())
.map_err(to_meta_err(snafu::location!()))
}
async fn handle_mark_window_dirty(&self, _req: DirtyWindowRequest) -> MetaResult<FlowResponse> {
// todo: implement
unimplemented!()
}
}
impl FlowEngine for StreamingEngine {

View File

@@ -17,6 +17,7 @@
use std::collections::{BTreeMap, HashMap};
use std::sync::Arc;
use api::v1::flow::{DirtyWindowRequests, FlowResponse};
use catalog::CatalogManagerRef;
use common_error::ext::BoxedError;
use common_meta::ddl::create_flow::FlowType;
@@ -29,8 +30,7 @@ use common_telemetry::{debug, info};
use common_time::TimeToLive;
use query::QueryEngineRef;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::storage::RegionId;
use table::metadata::TableId;
use store_api::storage::{RegionId, TableId};
use tokio::sync::{oneshot, RwLock};
use crate::batching_mode::frontend_client::FrontendClient;
@@ -42,6 +42,7 @@ use crate::error::{
ExternalSnafu, FlowAlreadyExistSnafu, FlowNotFoundSnafu, TableNotFoundMetaSnafu,
UnexpectedSnafu, UnsupportedSnafu,
};
use crate::metrics::METRIC_FLOW_BATCHING_ENGINE_BULK_MARK_TIME_WINDOW_RANGE;
use crate::{CreateFlowArgs, Error, FlowId, TableName};
/// Batching mode Engine, responsible for driving all the batching mode tasks
@@ -77,6 +78,122 @@ impl BatchingEngine {
}
}
pub async fn handle_mark_dirty_time_window(
&self,
reqs: DirtyWindowRequests,
) -> Result<FlowResponse, Error> {
let table_info_mgr = self.table_meta.table_info_manager();
let mut group_by_table_id: HashMap<u32, Vec<_>> = HashMap::new();
for r in reqs.requests {
let tid = TableId::from(r.table_id);
let entry = group_by_table_id.entry(tid).or_default();
entry.extend(r.dirty_time_ranges);
}
let tids = group_by_table_id.keys().cloned().collect::<Vec<TableId>>();
let table_infos =
table_info_mgr
.batch_get(&tids)
.await
.with_context(|_| TableNotFoundMetaSnafu {
msg: format!("Failed to get table info for table ids: {:?}", tids),
})?;
let group_by_table_name = group_by_table_id
.into_iter()
.filter_map(|(id, rows)| {
let table_name = table_infos.get(&id).map(|info| info.table_name());
let Some(table_name) = table_name else {
warn!("Failed to get table infos for table id: {:?}", id);
return None;
};
let table_name = [
table_name.catalog_name,
table_name.schema_name,
table_name.table_name,
];
let schema = &table_infos.get(&id).unwrap().table_info.meta.schema;
let time_index_unit = schema.column_schemas[schema.timestamp_index.unwrap()]
.data_type
.as_timestamp()
.unwrap()
.unit();
Some((table_name, (rows, time_index_unit)))
})
.collect::<HashMap<_, _>>();
let group_by_table_name = Arc::new(group_by_table_name);
let mut handles = Vec::new();
let tasks = self.tasks.read().await;
for (_flow_id, task) in tasks.iter() {
let src_table_names = &task.config.source_table_names;
if src_table_names
.iter()
.all(|name| !group_by_table_name.contains_key(name))
{
continue;
}
let group_by_table_name = group_by_table_name.clone();
let task = task.clone();
let handle: JoinHandle<Result<(), Error>> = tokio::spawn(async move {
let src_table_names = &task.config.source_table_names;
let mut all_dirty_windows = vec![];
for src_table_name in src_table_names {
if let Some((window_ranges, unit)) = group_by_table_name.get(src_table_name) {
let Some(expr) = &task.config.time_window_expr else {
continue;
};
for window in window_ranges {
let align_start = expr
.eval(common_time::Timestamp::new(window.start_value, *unit))?
.0
.context(UnexpectedSnafu {
reason: "Failed to eval start value",
})?;
let align_end = expr
.eval(common_time::Timestamp::new(window.end_value, *unit))?
.1
.context(UnexpectedSnafu {
reason: "Failed to eval end value",
})?;
all_dirty_windows.push((align_start, align_end));
}
}
}
let mut state = task.state.write().unwrap();
let flow_id_label = task.config.flow_id.to_string();
for (s, e) in all_dirty_windows {
METRIC_FLOW_BATCHING_ENGINE_BULK_MARK_TIME_WINDOW_RANGE
.with_label_values(&[&flow_id_label])
.observe(e.sub(&s).unwrap_or_default().num_seconds() as f64);
state.dirty_time_windows.add_window(s, Some(e));
}
Ok(())
});
handles.push(handle);
}
drop(tasks);
for handle in handles {
match handle.await {
Err(e) => {
warn!("Failed to handle inserts: {e}");
}
Ok(Ok(())) => (),
Ok(Err(e)) => {
warn!("Failed to handle inserts: {e}");
}
}
}
Ok(Default::default())
}
pub async fn handle_inserts_inner(
&self,
request: api::v1::region::InsertRequests,
@@ -388,6 +505,20 @@ impl BatchingEngine {
pub async fn flow_exist_inner(&self, flow_id: FlowId) -> bool {
self.tasks.read().await.contains_key(&flow_id)
}
pub async fn adjust_flow(
&self,
flow_id: FlowId,
min_run_interval_secs: u64,
max_filter_num_per_query: usize,
) -> Result<(), Error> {
let task = self.tasks.read().await.get(&flow_id).cloned();
let task = task.with_context(|| FlowNotFoundSnafu { id: flow_id })?;
debug!("Adjusting flow {flow_id} with min_run_interval_secs={} and max_filter_num_per_query={}", min_run_interval_secs, max_filter_num_per_query);
task.adjust(min_run_interval_secs, max_filter_num_per_query);
Ok(())
}
}
impl FlowEngine for BatchingEngine {

View File

@@ -14,8 +14,9 @@
//! Frontend client to run flow as batching task which is time-window-aware normal query triggered every tick set by user
use std::sync::{Arc, Weak};
use std::time::SystemTime;
use std::collections::HashMap;
use std::sync::{Arc, Mutex, Weak};
use std::time::{Duration, Instant, SystemTime};
use api::v1::greptime_request::Request;
use api::v1::CreateTableExpr;
@@ -26,20 +27,21 @@ use common_meta::cluster::{NodeInfo, NodeInfoKey, Role};
use common_meta::peer::Peer;
use common_meta::rpc::store::RangeRequest;
use common_query::Output;
use common_telemetry::warn;
use common_telemetry::{debug, warn};
use itertools::Itertools;
use meta_client::client::MetaClient;
use rand::rng;
use rand::seq::SliceRandom;
use servers::query_handler::grpc::GrpcQueryHandler;
use session::context::{QueryContextBuilder, QueryContextRef};
use snafu::{OptionExt, ResultExt};
use crate::batching_mode::task::BatchingTask;
use crate::batching_mode::{
DEFAULT_BATCHING_ENGINE_QUERY_TIMEOUT, FRONTEND_ACTIVITY_TIMEOUT, GRPC_CONN_TIMEOUT,
GRPC_MAX_RETRIES,
};
use crate::error::{ExternalSnafu, InvalidRequestSnafu, NoAvailableFrontendSnafu, UnexpectedSnafu};
use crate::{Error, FlowAuthHeader};
use crate::metrics::METRIC_FLOW_BATCHING_ENGINE_GUESS_FE_LOAD;
use crate::{Error, FlowAuthHeader, FlowId};
/// Just like [`GrpcQueryHandler`] but use BoxedError
///
@@ -74,6 +76,105 @@ impl<
type HandlerMutable = Arc<std::sync::Mutex<Option<Weak<dyn GrpcQueryHandlerWithBoxedError>>>>;
/// Statistics about running query on this frontend from flownode
#[derive(Debug, Default, Clone)]
struct FrontendStat {
/// The query for flow id has been running since this timestamp
since: HashMap<FlowId, Instant>,
/// The average query time for each flow id
/// This is used to calculate the average query time for each flow id
past_query_avg: HashMap<FlowId, (usize, Duration)>,
}
#[derive(Debug, Default, Clone)]
pub struct FrontendStats {
/// The statistics for each flow id
stats: Arc<Mutex<HashMap<String, FrontendStat>>>,
}
impl FrontendStats {
pub fn observe(&self, frontend_addr: &str, flow_id: FlowId) -> FrontendStatsGuard {
let mut stats = self.stats.lock().expect("Failed to lock frontend stats");
let stat = stats.entry(frontend_addr.to_string()).or_default();
stat.since.insert(flow_id, Instant::now());
FrontendStatsGuard {
stats: self.stats.clone(),
frontend_addr: frontend_addr.to_string(),
cur: flow_id,
}
}
/// return frontend addrs sorted by load, from lightest to heaviest
/// The load is calculated as the total average query time for each flow id plus running query's total running time elapsed
pub fn sort_by_load(&self) -> Vec<String> {
let stats = self.stats.lock().expect("Failed to lock frontend stats");
let fe_load_factor = stats
.iter()
.map(|(node_addr, stat)| {
// total expected avg running time for all currently running queries
let total_expect_avg_run_time = stat
.since
.keys()
.map(|f| {
let (count, total_duration) =
stat.past_query_avg.get(f).unwrap_or(&(0, Duration::ZERO));
if *count == 0 {
0.0
} else {
total_duration.as_secs_f64() / *count as f64
}
})
.sum::<f64>();
let total_cur_running_time = stat
.since
.values()
.map(|since| since.elapsed().as_secs_f64())
.sum::<f64>();
(
node_addr.to_string(),
total_expect_avg_run_time + total_cur_running_time,
)
})
.sorted_by(|(_, load_a), (_, load_b)| {
load_a
.partial_cmp(load_b)
.unwrap_or(std::cmp::Ordering::Equal)
})
.collect::<Vec<_>>();
debug!("Frontend load factor: {:?}", fe_load_factor);
for (node_addr, load) in &fe_load_factor {
METRIC_FLOW_BATCHING_ENGINE_GUESS_FE_LOAD
.with_label_values(&[&node_addr.to_string()])
.observe(*load);
}
fe_load_factor
.into_iter()
.map(|(addr, _)| addr)
.collect::<Vec<_>>()
}
}
pub struct FrontendStatsGuard {
stats: Arc<Mutex<HashMap<String, FrontendStat>>>,
frontend_addr: String,
cur: FlowId,
}
impl Drop for FrontendStatsGuard {
fn drop(&mut self) {
let mut stats = self.stats.lock().expect("Failed to lock frontend stats");
if let Some(stat) = stats.get_mut(&self.frontend_addr) {
if let Some(since) = stat.since.remove(&self.cur) {
let elapsed = since.elapsed();
let (count, total_duration) = stat.past_query_avg.entry(self.cur).or_default();
*count += 1;
*total_duration += elapsed;
}
}
}
}
/// A simple frontend client able to execute sql using grpc protocol
///
/// This is for computation-heavy query which need to offload computation to frontend, lifting the load from flownode
@@ -83,6 +184,7 @@ pub enum FrontendClient {
meta_client: Arc<MetaClient>,
chnl_mgr: ChannelManager,
auth: Option<FlowAuthHeader>,
fe_stats: FrontendStats,
},
Standalone {
/// for the sake of simplicity still use grpc even in standalone mode
@@ -114,6 +216,7 @@ impl FrontendClient {
ChannelManager::with_config(cfg)
},
auth,
fe_stats: Default::default(),
}
}
@@ -192,6 +295,7 @@ impl FrontendClient {
meta_client: _,
chnl_mgr,
auth,
fe_stats,
} = self
else {
return UnexpectedSnafu {
@@ -208,8 +312,21 @@ impl FrontendClient {
.duration_since(SystemTime::UNIX_EPOCH)
.unwrap()
.as_millis() as i64;
// shuffle the frontends to avoid always pick the same one
frontends.shuffle(&mut rng());
let node_addrs_by_load = fe_stats.sort_by_load();
// index+1 to load order asc, so that the lightest node has load 1 and non-existent node has load 0
let addr2load = node_addrs_by_load
.iter()
.enumerate()
.map(|(i, id)| (id.clone(), i + 1))
.collect::<HashMap<_, _>>();
// sort frontends by load, from lightest to heaviest
frontends.sort_by(|(_, a), (_, b)| {
// if not even in stats, treat as 0 load since never been queried
let load_a = addr2load.get(&a.peer.addr).unwrap_or(&0);
let load_b = addr2load.get(&b.peer.addr).unwrap_or(&0);
load_a.cmp(load_b)
});
debug!("Frontend nodes sorted by load: {:?}", frontends);
// found node with maximum last_activity_ts
for (_, node_info) in frontends
@@ -257,6 +374,7 @@ impl FrontendClient {
create: CreateTableExpr,
catalog: &str,
schema: &str,
task: Option<&BatchingTask>,
) -> Result<u32, Error> {
self.handle(
Request::Ddl(api::v1::DdlRequest {
@@ -265,6 +383,7 @@ impl FrontendClient {
catalog,
schema,
&mut None,
task,
)
.await
}
@@ -276,17 +395,21 @@ impl FrontendClient {
catalog: &str,
schema: &str,
peer_desc: &mut Option<PeerDesc>,
task: Option<&BatchingTask>,
) -> Result<u32, Error> {
match self {
FrontendClient::Distributed { .. } => {
FrontendClient::Distributed { fe_stats, .. } => {
let db = self.get_random_active_frontend(catalog, schema).await?;
*peer_desc = Some(PeerDesc::Dist {
peer: db.peer.clone(),
});
let flow_id = task.map(|t| t.config.flow_id).unwrap_or_default();
let _guard = fe_stats.observe(&db.peer.addr, flow_id);
db.database
.handle_with_retry(req.clone(), GRPC_MAX_RETRIES)
.handle_with_retry(req.clone(), GRPC_MAX_RETRIES, &[("query_parallelism", "1")])
.await
.with_context(|_| InvalidRequestSnafu {
context: format!("Failed to handle request at {:?}: {:?}", db.peer, req),

View File

@@ -31,7 +31,9 @@ use crate::batching_mode::time_window::TimeWindowExpr;
use crate::batching_mode::MIN_REFRESH_DURATION;
use crate::error::{DatatypesSnafu, InternalSnafu, TimeSnafu, UnexpectedSnafu};
use crate::metrics::{
METRIC_FLOW_BATCHING_ENGINE_QUERY_TIME_RANGE, METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_CNT,
METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_CNT, METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_SIZE,
METRIC_FLOW_BATCHING_ENGINE_STALLED_QUERY_WINDOW_CNT,
METRIC_FLOW_BATCHING_ENGINE_STALLED_WINDOW_SIZE,
};
use crate::{Error, FlowId};
@@ -52,6 +54,16 @@ pub struct TaskState {
pub(crate) shutdown_rx: oneshot::Receiver<()>,
/// Task handle
pub(crate) task_handle: Option<tokio::task::JoinHandle<()>>,
/// min run interval in seconds
pub(crate) min_run_interval: Option<u64>,
/// max filter number per query
pub(crate) max_filter_num: Option<usize>,
/// Current filter count, will grow when query succeeds(capped by `max_filter_num`),
/// and reset to 1 when query fails.
///
/// This is useful for controlling resource usage
pub(crate) cur_filter_cnt: usize,
}
impl TaskState {
pub fn new(query_ctx: QueryContextRef, shutdown_rx: oneshot::Receiver<()>) -> Self {
@@ -63,6 +75,9 @@ impl TaskState {
exec_state: ExecState::Idle,
shutdown_rx,
task_handle: None,
min_run_interval: None,
max_filter_num: None,
cur_filter_cnt: 1,
}
}
@@ -90,24 +105,29 @@ impl TaskState {
time_window_size: &Option<Duration>,
max_timeout: Option<Duration>,
) -> Instant {
let last_duration = max_timeout
.unwrap_or(self.last_query_duration)
.min(self.last_query_duration)
.max(MIN_REFRESH_DURATION);
// = last query duration, capped by [max(min_run_interval, time_window_size), max_timeout], note at most `max_timeout`
let lower = self.min_run_interval
.map(Duration::from_secs)
.unwrap_or(MIN_REFRESH_DURATION).max(time_window_size.unwrap_or_default());
let next_duration = self.last_query_duration.max(lower);
let next_duration = if let Some(max_timeout) = max_timeout {
next_duration.min(max_timeout)
} else {
next_duration
};
let next_duration = time_window_size
.map(|t| {
let half = t / 2;
half.max(last_duration)
})
.unwrap_or(last_duration);
// if have dirty time window, execute immediately to clean dirty time window
if self.dirty_time_windows.windows.is_empty() {
let cur_dirty_window_size = self.dirty_time_windows.window_size();
let max_query_update_range = time_window_size.clone().unwrap_or_default().mul_f64(
self.max_filter_num
.unwrap_or(DirtyTimeWindows::MAX_FILTER_NUM) as f64,
);
if cur_dirty_window_size < max_query_update_range {
self.last_update_time + next_duration
} else {
// if dirty time windows can't be clean up in one query, execute immediately to faster
// clean up dirty time windows
debug!(
"Flow id = {}, still have {} dirty time window({:?}), execute immediately",
"Flow id = {}, still have too many{} dirty time window({:?}), execute immediately",
flow_id,
self.dirty_time_windows.windows.len(),
self.dirty_time_windows.windows
@@ -126,6 +146,17 @@ pub struct DirtyTimeWindows {
windows: BTreeMap<Timestamp, Option<Timestamp>>,
}
/// Time windows that are being worked on, which are not dirty but are currently being processed
#[derive(Debug, Clone, Default)]
pub struct WorkingTimeWindows {
/// windows's `start -> end` and non-overlapping
/// `end` is exclusive(and optional)
pub windows: BTreeMap<Timestamp, Option<Timestamp>>,
/// Filter expression for the time windows
/// This is used to filter the data in the time windows.
pub filter: Option<datafusion_expr::Expr>,
}
impl DirtyTimeWindows {
/// Time window merge distance
///
@@ -147,10 +178,28 @@ impl DirtyTimeWindows {
}
}
pub fn window_size(&self) -> Duration {
let mut ret = Duration::from_secs(0);
for (start, end) in &self.windows {
if let Some(end) = end {
if let Some(duration) = end.sub(start) {
ret += duration.to_std().unwrap_or_default();
}
}
}
ret
}
pub fn add_window(&mut self, start: Timestamp, end: Option<Timestamp>) {
self.windows.insert(start, end);
}
pub fn add_windows(&mut self, windows: BTreeMap<Timestamp, Option<Timestamp>>) {
for (start, end) in windows {
self.windows.insert(start, end);
}
}
/// Clean all dirty time windows, useful when can't found time window expr
pub fn clean(&mut self) {
self.windows.clear();
@@ -169,7 +218,7 @@ impl DirtyTimeWindows {
window_cnt: usize,
flow_id: FlowId,
task_ctx: Option<&BatchingTask>,
) -> Result<Option<datafusion_expr::Expr>, Error> {
) -> Result<WorkingTimeWindows, Error> {
debug!(
"expire_lower_bound: {:?}, window_size: {:?}",
expire_lower_bound.map(|t| t.to_iso8601_string()),
@@ -206,47 +255,63 @@ impl DirtyTimeWindows {
// get the first `window_cnt` time windows
let max_time_range = window_size * window_cnt as i32;
let nth = {
let mut cur_time_range = chrono::Duration::zero();
let mut nth_key = None;
for (idx, (start, end)) in self.windows.iter().enumerate() {
// if time range is too long, stop
if cur_time_range > max_time_range {
nth_key = Some(*start);
break;
}
// if we have enough time windows, stop
if idx >= window_cnt {
nth_key = Some(*start);
break;
}
let mut to_be_query = BTreeMap::new();
let mut new_windows = self.windows.clone();
let mut cur_time_range = chrono::Duration::zero();
for (idx, (start, end)) in self.windows.iter().enumerate() {
let first_end = start
.add_duration(window_size.to_std().unwrap())
.context(TimeSnafu)?;
let end = end.unwrap_or(first_end);
if let Some(end) = end {
if let Some(x) = end.sub(start) {
cur_time_range += x;
}
}
// if time range is too long, stop
if cur_time_range >= max_time_range {
break;
}
nth_key
};
let first_nth = {
if let Some(nth) = nth {
let mut after = self.windows.split_off(&nth);
std::mem::swap(&mut self.windows, &mut after);
after
} else {
std::mem::take(&mut self.windows)
// if we have enough time windows, stop
if idx >= window_cnt {
break;
}
};
if let Some(x) = end.sub(start) {
if cur_time_range + x <= max_time_range {
to_be_query.insert(*start, Some(end));
new_windows.remove(start);
cur_time_range += x;
} else {
// too large a window, split it
// split at window_size * times
let surplus = max_time_range - cur_time_range;
let times = surplus.num_seconds() / window_size.num_seconds();
let split_offset = window_size * times as i32;
let split_at = start
.add_duration(split_offset.to_std().unwrap())
.context(TimeSnafu)?;
to_be_query.insert(*start, Some(split_at));
// remove the original window
new_windows.remove(start);
new_windows.insert(split_at, Some(end));
cur_time_range += split_offset;
break;
}
}
}
self.windows = new_windows;
METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_CNT
.with_label_values(&[flow_id.to_string().as_str()])
.observe(first_nth.len() as f64);
.observe(to_be_query.len() as f64);
let full_time_range = first_nth
METRIC_FLOW_BATCHING_ENGINE_STALLED_QUERY_WINDOW_CNT
.with_label_values(&[flow_id.to_string().as_str()])
.observe(self.windows.len() as f64);
let full_time_range = to_be_query
.iter()
.fold(chrono::Duration::zero(), |acc, (start, end)| {
if let Some(end) = end {
@@ -256,12 +321,27 @@ impl DirtyTimeWindows {
}
})
.num_seconds() as f64;
METRIC_FLOW_BATCHING_ENGINE_QUERY_TIME_RANGE
METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_SIZE
.with_label_values(&[flow_id.to_string().as_str()])
.observe(full_time_range);
let stalled_time_range =
self.windows
.iter()
.fold(chrono::Duration::zero(), |acc, (start, end)| {
if let Some(end) = end {
acc + end.sub(start).unwrap_or(chrono::Duration::zero())
} else {
acc
}
});
METRIC_FLOW_BATCHING_ENGINE_STALLED_WINDOW_SIZE
.with_label_values(&[flow_id.to_string().as_str()])
.observe(stalled_time_range.num_seconds() as f64);
let mut expr_lst = vec![];
for (start, end) in first_nth.into_iter() {
for (start, end) in to_be_query.clone().into_iter() {
// align using time window exprs
let (start, end) = if let Some(ctx) = task_ctx {
let Some(time_window_expr) = &ctx.config.time_window_expr else {
@@ -293,7 +373,12 @@ impl DirtyTimeWindows {
expr_lst.push(expr);
}
let expr = expr_lst.into_iter().reduce(|a, b| a.or(b));
Ok(expr)
let working = WorkingTimeWindows {
windows: to_be_query,
filter: expr,
};
Ok(working)
}
fn align_time_window(
@@ -495,6 +580,64 @@ mod test {
"((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts < CAST('1970-01-01 00:00:21' AS TIMESTAMP)))",
)
),
// split range
(
Vec::from_iter((0..20).map(|i|Timestamp::new_second(i*3)).chain(std::iter::once(
Timestamp::new_second(60 + 3 * (DirtyTimeWindows::MERGE_DIST as i64 + 1)),
))),
(chrono::Duration::seconds(3), None),
BTreeMap::from([
(
Timestamp::new_second(0),
Some(Timestamp::new_second(
60
)),
),
(
Timestamp::new_second(60 + 3 * (DirtyTimeWindows::MERGE_DIST as i64 + 1)),
Some(Timestamp::new_second(
60 + 3 * (DirtyTimeWindows::MERGE_DIST as i64 + 1) + 3
)),
)]),
Some(
"((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts < CAST('1970-01-01 00:01:00' AS TIMESTAMP)))",
)
),
// split 2 min into 1 min
(
Vec::from_iter((0..40).map(|i|Timestamp::new_second(i*3))),
(chrono::Duration::seconds(3), None),
BTreeMap::from([
(
Timestamp::new_second(0),
Some(Timestamp::new_second(
40 * 3
)),
)]),
Some(
"((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts < CAST('1970-01-01 00:01:00' AS TIMESTAMP)))",
)
),
// split 3s + 1min into 3s + 57s
(
Vec::from_iter(std::iter::once(Timestamp::new_second(0)).chain((0..40).map(|i|Timestamp::new_second(20+i*3)))),
(chrono::Duration::seconds(3), None),
BTreeMap::from([
(
Timestamp::new_second(0),
Some(Timestamp::new_second(
3
)),
),(
Timestamp::new_second(20),
Some(Timestamp::new_second(
140
)),
)]),
Some(
"(((ts >= CAST('1970-01-01 00:00:00' AS TIMESTAMP)) AND (ts < CAST('1970-01-01 00:00:03' AS TIMESTAMP))) OR ((ts >= CAST('1970-01-01 00:00:20' AS TIMESTAMP)) AND (ts < CAST('1970-01-01 00:01:17' AS TIMESTAMP))))",
)
),
// expired
(
vec![
@@ -511,6 +654,8 @@ mod test {
None
),
];
// let len = testcases.len();
// let testcases = testcases[(len - 2)..(len - 1)].to_vec();
for (lower_bounds, (window_size, expire_lower_bound), expected, expected_filter_expr) in
testcases
{
@@ -529,7 +674,8 @@ mod test {
0,
None,
)
.unwrap();
.unwrap()
.filter;
let unparser = datafusion::sql::unparser::Unparser::default();
let to_sql = filter_expr

View File

@@ -46,7 +46,7 @@ use tokio::time::Instant;
use crate::adapter::{AUTO_CREATED_PLACEHOLDER_TS_COL, AUTO_CREATED_UPDATE_AT_TS_COL};
use crate::batching_mode::frontend_client::FrontendClient;
use crate::batching_mode::state::{DirtyTimeWindows, TaskState};
use crate::batching_mode::state::{DirtyTimeWindows, TaskState, WorkingTimeWindows};
use crate::batching_mode::time_window::TimeWindowExpr;
use crate::batching_mode::utils::{
get_table_info_df_schema, sql_to_df_plan, AddAutoColumnRewriter, AddFilterRewriter,
@@ -61,7 +61,9 @@ use crate::error::{
SubstraitEncodeLogicalPlanSnafu, UnexpectedSnafu,
};
use crate::metrics::{
METRIC_FLOW_BATCHING_ENGINE_CUR_WND_CNT, METRIC_FLOW_BATCHING_ENGINE_ERROR_CNT,
METRIC_FLOW_BATCHING_ENGINE_QUERY_TIME, METRIC_FLOW_BATCHING_ENGINE_SLOW_QUERY,
METRIC_FLOW_BATCHING_ENGINE_START_QUERY_CNT, METRIC_FLOW_ROWS,
};
use crate::{Error, FlowId};
@@ -108,6 +110,13 @@ enum QueryType {
Sql,
}
/// A plan with working time windows, used to track the time windows that are currently being processed
#[derive(Debug, Clone)]
pub struct PlanWithWindow {
pub plan: Option<LogicalPlan>,
pub working_windows: WorkingTimeWindows,
}
#[derive(Clone)]
pub struct BatchingTask {
pub config: Arc<TaskConfig>,
@@ -144,6 +153,12 @@ impl BatchingTask {
})
}
pub fn adjust(&self, min_run_interval_secs: u64, max_filter_num_per_query: usize) {
let mut state = self.state.write().unwrap();
state.min_run_interval = Some(min_run_interval_secs);
state.max_filter_num = Some(max_filter_num_per_query);
}
/// mark time window range (now - expire_after, now) as dirty (or (0, now) if expire_after not set)
///
/// useful for flush_flow to flush dirty time windows range
@@ -210,30 +225,29 @@ impl BatchingTask {
engine: &QueryEngineRef,
frontend_client: &Arc<FrontendClient>,
) -> Result<Option<(u32, Duration)>, Error> {
if let Some(new_query) = self.gen_insert_plan(engine).await? {
if let Some(new_query) = self.gen_insert_plan(engine).await?.plan {
debug!("Generate new query: {}", new_query);
self.execute_logical_plan(frontend_client, &new_query).await
self.execute_logical_plan(frontend_client, &new_query)
.await
.map(Some)
} else {
debug!("Generate no query");
Ok(None)
}
}
pub async fn gen_insert_plan(
&self,
engine: &QueryEngineRef,
) -> Result<Option<LogicalPlan>, Error> {
pub async fn gen_insert_plan(&self, engine: &QueryEngineRef) -> Result<PlanWithWindow, Error> {
let (table, df_schema) = get_table_info_df_schema(
self.config.catalog_manager.clone(),
self.config.sink_table_name.clone(),
)
.await?;
let new_query = self
let new_query_info = self
.gen_query_with_time_window(engine.clone(), &table.meta.schema)
.await?;
let insert_into = if let Some((new_query, _column_cnt)) = new_query {
let insert_into = if let Some(new_query) = new_query_info.plan {
// first check if all columns in input query exists in sink table
// since insert into ref to names in record batch generate by given query
let table_columns = df_schema
@@ -264,12 +278,15 @@ impl BatchingTask {
Arc::new(new_query),
))
} else {
return Ok(None);
return Ok(new_query_info);
};
let insert_into = insert_into.recompute_schema().context(DatafusionSnafu {
context: "Failed to recompute schema",
})?;
Ok(Some(insert_into))
Ok(PlanWithWindow {
plan: Some(insert_into),
working_windows: new_query_info.working_windows,
})
}
pub async fn create_table(
@@ -280,7 +297,7 @@ impl BatchingTask {
let catalog = &self.config.sink_table_name[0];
let schema = &self.config.sink_table_name[1];
frontend_client
.create(expr.clone(), catalog, schema)
.create(expr.clone(), catalog, schema, Some(self))
.await?;
Ok(())
}
@@ -289,7 +306,7 @@ impl BatchingTask {
&self,
frontend_client: &Arc<FrontendClient>,
plan: &LogicalPlan,
) -> Result<Option<(u32, Duration)>, Error> {
) -> Result<(u32, Duration), Error> {
let instant = Instant::now();
let flow_id = self.config.flow_id;
@@ -361,7 +378,7 @@ impl BatchingTask {
};
frontend_client
.handle(req, catalog, schema, &mut peer_desc)
.handle(req, catalog, schema, &mut peer_desc, Some(self))
.await
};
@@ -371,6 +388,9 @@ impl BatchingTask {
"Flow {flow_id} executed, affected_rows: {affected_rows:?}, elapsed: {:?}",
elapsed
);
METRIC_FLOW_ROWS
.with_label_values(&[format!("{}-out-batching", flow_id).as_str()])
.inc_by(*affected_rows as _);
} else if let Err(err) = &res {
warn!(
"Failed to execute Flow {flow_id} on frontend {:?}, result: {err:?}, elapsed: {:?} with query: {}",
@@ -399,7 +419,7 @@ impl BatchingTask {
let res = res?;
Ok(Some((res, elapsed)))
Ok((res, elapsed))
}
/// start executing query in a loop, break when receive shutdown signal
@@ -410,6 +430,7 @@ impl BatchingTask {
engine: QueryEngineRef,
frontend_client: Arc<FrontendClient>,
) {
let flow_id_str = self.config.flow_id.to_string();
loop {
// first check if shutdown signal is received
// if so, break the loop
@@ -427,8 +448,11 @@ impl BatchingTask {
Err(TryRecvError::Empty) => (),
}
}
METRIC_FLOW_BATCHING_ENGINE_START_QUERY_CNT
.with_label_values(&[&flow_id_str])
.inc();
let new_query = match self.gen_insert_plan(&engine).await {
let new_query_info = match self.gen_insert_plan(&engine).await {
Ok(new_query) => new_query,
Err(err) => {
common_telemetry::error!(err; "Failed to generate query for flow={}", self.config.flow_id);
@@ -438,8 +462,10 @@ impl BatchingTask {
}
};
let res = if let Some(new_query) = &new_query {
self.execute_logical_plan(&frontend_client, new_query).await
let res = if let Some(new_query) = &new_query_info.plan {
self.execute_logical_plan(&frontend_client, new_query)
.await
.map(Some)
} else {
Ok(None)
};
@@ -448,7 +474,17 @@ impl BatchingTask {
// normal execute, sleep for some time before doing next query
Ok(Some(_)) => {
let sleep_until = {
let state = self.state.write().unwrap();
let mut state = self.state.write().unwrap();
// double cur_filter_cnt
state.cur_filter_cnt = state.cur_filter_cnt.saturating_mul(2).min(
state
.max_filter_num
.unwrap_or(DirtyTimeWindows::MAX_FILTER_NUM),
);
METRIC_FLOW_BATCHING_ENGINE_CUR_WND_CNT
.with_label_values(&[&flow_id_str])
.set(state.cur_filter_cnt as i64);
state.get_next_start_query_time(
self.config.flow_id,
@@ -473,7 +509,10 @@ impl BatchingTask {
}
// TODO(discord9): this error should have better place to go, but for now just print error, also more context is needed
Err(err) => {
match new_query {
METRIC_FLOW_BATCHING_ENGINE_ERROR_CNT
.with_label_values(&[&flow_id_str])
.inc();
match new_query_info.plan {
Some(query) => {
common_telemetry::error!(err; "Failed to execute query for flow={} with query: {query}", self.config.flow_id)
}
@@ -481,6 +520,17 @@ impl BatchingTask {
common_telemetry::error!(err; "Failed to generate query for flow={}", self.config.flow_id)
}
}
{
// return working windows to dirty windows, and reset current filter cnt so next time we generate query only generate a small query
let mut state = self.state.write().unwrap();
state
.dirty_time_windows
.add_windows(new_query_info.working_windows.windows);
state.cur_filter_cnt = 1;
METRIC_FLOW_BATCHING_ENGINE_CUR_WND_CNT
.with_label_values(&[&flow_id_str])
.set(state.cur_filter_cnt as i64);
}
// also sleep for a little while before try again to prevent flooding logs
tokio::time::sleep(MIN_REFRESH_DURATION).await;
}
@@ -509,7 +559,7 @@ impl BatchingTask {
&self,
engine: QueryEngineRef,
sink_table_schema: &Arc<Schema>,
) -> Result<Option<(LogicalPlan, usize)>, Error> {
) -> Result<PlanWithWindow, Error> {
let query_ctx = self.state.read().unwrap().query_ctx.clone();
let start = SystemTime::now();
let since_the_epoch = start
@@ -522,7 +572,6 @@ impl BatchingTask {
.unwrap_or(u64::MIN);
let low_bound = Timestamp::new_second(low_bound as i64);
let schema_len = self.config.output_schema.fields().len();
let expire_time_window_bound = self
.config
@@ -555,10 +604,12 @@ impl BatchingTask {
context: format!("Failed to rewrite plan:\n {}\n", plan),
})?
.data;
let schema_len = plan.schema().fields().len();
// since no time window lower/upper bound is found, just return the original query(with auto columns)
return Ok(Some((plan, schema_len)));
return Ok(PlanWithWindow {
plan: Some(plan),
working_windows: WorkingTimeWindows::default(),
});
};
debug!(
@@ -580,24 +631,25 @@ impl BatchingTask {
),
})?;
let expr = self
.state
.write()
.unwrap()
.dirty_time_windows
.gen_filter_exprs(
let working_windows = {
let mut state = self.state.write().unwrap();
let cur_wnd_cnt = state.cur_filter_cnt;
state.dirty_time_windows.gen_filter_exprs(
&col_name,
Some(l),
window_size,
DirtyTimeWindows::MAX_FILTER_NUM,
cur_wnd_cnt,
self.config.flow_id,
Some(self),
)?;
)?
};
debug!(
"Flow id={:?}, Generated filter expr: {:?}",
self.config.flow_id,
expr.as_ref()
working_windows
.filter
.as_ref()
.map(|expr| expr_to_sql(expr).with_context(|_| DatafusionSnafu {
context: format!("Failed to generate filter expr from {expr:?}"),
}))
@@ -605,13 +657,16 @@ impl BatchingTask {
.map(|s| s.to_string())
);
let Some(expr) = expr else {
let Some(expr) = &working_windows.filter else {
// no new data, hence no need to update
debug!("Flow id={:?}, no new data, not update", self.config.flow_id);
return Ok(None);
return Ok(PlanWithWindow {
plan: None,
working_windows,
});
};
let mut add_filter = AddFilterRewriter::new(expr);
let mut add_filter = AddFilterRewriter::new(expr.clone());
let mut add_auto_column = AddAutoColumnRewriter::new(sink_table_schema.clone());
let plan =
@@ -627,7 +682,10 @@ impl BatchingTask {
// only apply optimize after complex rewrite is done
let new_plan = apply_df_optimizer(rewrite).await?;
Ok(Some((new_plan, schema_len)))
Ok(PlanWithWindow {
plan: Some(new_plan),
working_windows,
})
}
}

View File

@@ -42,6 +42,14 @@ lazy_static! {
vec![60., 2. * 60., 3. * 60., 5. * 60., 10. * 60.]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_STALLED_QUERY_WINDOW_CNT: HistogramVec =
register_histogram_vec!(
"greptime_flow_batching_engine_stalled_query_window_cnt",
"flow batching engine stalled query time window count",
&["flow_id"],
vec![0.0, 5., 10., 20., 40.]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_CNT: HistogramVec =
register_histogram_vec!(
"greptime_flow_batching_engine_query_window_cnt",
@@ -50,19 +58,63 @@ lazy_static! {
vec![0.0, 5., 10., 20., 40.]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_QUERY_TIME_RANGE: HistogramVec =
pub static ref METRIC_FLOW_BATCHING_ENGINE_QUERY_WINDOW_SIZE: HistogramVec =
register_histogram_vec!(
"greptime_flow_batching_engine_query_time_range_secs",
"flow batching engine query time range(seconds)",
"greptime_flow_batching_engine_query_window_size_secs",
"flow batching engine query window size(seconds)",
&["flow_id"],
vec![60., 4. * 60., 16. * 60., 64. * 60., 256. * 60.]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_STALLED_WINDOW_SIZE: HistogramVec =
register_histogram_vec!(
"greptime_flow_batching_engine_stalled_window_size_secs",
"flow batching engine stalled window size(seconds)",
&["flow_id"],
vec![60., 4. * 60., 16. * 60., 64. * 60., 256. * 60.]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_BULK_MARK_TIME_WINDOW_RANGE: HistogramVec =
register_histogram_vec!(
"greptime_flow_batching_engine_bulk_mark_time_window_range_secs",
"flow batching engine query time window range marked by bulk memtable in seconds",
&["flow_id"],
vec![0.0, 60., 4. * 60., 16. * 60., 64. * 60., 256. * 60.]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_START_QUERY_CNT: IntCounterVec =
register_int_counter_vec!(
"greptime_flow_batching_start_query_count",
"flow batching engine started query count",
&["flow_id"],
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_ERROR_CNT: IntCounterVec =
register_int_counter_vec!(
"greptime_flow_batching_error_count",
"flow batching engine error count per flow id",
&["flow_id"]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_CUR_WND_CNT: IntGaugeVec = register_int_gauge_vec!(
"greptime_flow_batching_current_window_count",
"flow batching engine current query window count per flow id",
&["flow_id"]
)
.unwrap();
pub static ref METRIC_FLOW_BATCHING_ENGINE_GUESS_FE_LOAD: HistogramVec =
register_histogram_vec!(
"greptime_flow_batching_engine_guess_fe_load",
"flow batching engine guessed frontend load",
&["fe_addr"],
vec![60., 4. * 60., 16. * 60., 64. * 60., 256. * 60.]
)
.unwrap();
pub static ref METRIC_FLOW_RUN_INTERVAL_MS: IntGauge =
register_int_gauge!("greptime_flow_run_interval_ms", "flow run interval in ms").unwrap();
pub static ref METRIC_FLOW_ROWS: IntCounterVec = register_int_counter_vec!(
"greptime_flow_processed_rows",
"Count of rows flowing through the system",
"Count of rows flowing through the system.",
&["direction"]
)
.unwrap();

View File

@@ -17,6 +17,7 @@
use std::net::SocketAddr;
use std::sync::Arc;
use api::v1::flow::DirtyWindowRequests;
use api::v1::{RowDeleteRequests, RowInsertRequests};
use cache::{TABLE_FLOWNODE_SET_CACHE_NAME, TABLE_ROUTE_CACHE_NAME};
use catalog::CatalogManagerRef;
@@ -136,6 +137,18 @@ impl flow_server::Flow for FlowService {
.map(Response::new)
.map_err(to_status_with_last_err)
}
async fn handle_mark_dirty_time_window(
&self,
reqs: Request<DirtyWindowRequests>,
) -> Result<Response<FlowResponse>, Status> {
self.dual_engine
.batching_engine()
.handle_mark_dirty_time_window(reqs.into_inner())
.await
.map(Response::new)
.map_err(to_status_with_last_err)
}
}
#[derive(Clone)]

View File

@@ -35,8 +35,8 @@ use servers::query_handler::grpc::GrpcQueryHandler;
use servers::query_handler::sql::SqlQueryHandler;
use session::context::QueryContextRef;
use snafu::{ensure, OptionExt, ResultExt};
use table::metadata::TableId;
use table::table_name::TableName;
use table::TableRef;
use crate::error::{
CatalogSnafu, DataFusionSnafu, Error, InFlightWriteBytesExceededSnafu,
@@ -235,34 +235,33 @@ impl GrpcQueryHandler for Instance {
async fn put_record_batch(
&self,
table: &TableName,
table_id: &mut Option<TableId>,
table_name: &TableName,
table_ref: &mut Option<TableRef>,
decoder: &mut FlightDecoder,
data: FlightData,
) -> Result<AffectedRows> {
let table_id = if let Some(table_id) = table_id {
*table_id
let table = if let Some(table) = table_ref {
table.clone()
} else {
let table = self
.catalog_manager()
.table(
&table.catalog_name,
&table.schema_name,
&table.table_name,
&table_name.catalog_name,
&table_name.schema_name,
&table_name.table_name,
None,
)
.await
.context(CatalogSnafu)?
.with_context(|| TableNotFoundSnafu {
table_name: table.to_string(),
table_name: table_name.to_string(),
})?;
let id = table.table_info().table_id();
*table_id = Some(id);
id
*table_ref = Some(table.clone());
table
};
self.inserter
.handle_bulk_insert(table_id, decoder, data)
.handle_bulk_insert(table, decoder, data)
.await
.context(TableOperationSnafu)
}

View File

@@ -962,19 +962,22 @@ impl StreamContext {
}
}
}
if verbose {
write!(f, "{{")?;
}
write!(
f,
"partition_count={} ({} memtable ranges, {} file {} ranges)",
"\"partition_count\":{{\"count\":{}, \"mem_ranges\":{}, \"files\":{}, \"file_ranges\":{}}}",
self.ranges.len(),
num_mem_ranges,
self.input.num_files(),
num_file_ranges,
)?;
if let Some(selector) = &self.input.series_row_selector {
write!(f, ", selector={}", selector)?;
write!(f, ", \"selector\":\"{}\"", selector)?;
}
if let Some(distribution) = &self.input.distribution {
write!(f, ", distribution={}", distribution)?;
write!(f, ", \"distribution\":\"{}\"", distribution)?;
}
if verbose {
@@ -991,14 +994,15 @@ impl StreamContext {
impl fmt::Debug for FileWrapper<'_> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
let (start, end) = self.file.time_range();
write!(
f,
"[file={}, time_range=({}::{}, {}::{}), rows={}, size={}, index_size={}]",
r#"{{"file_id":"{}","time_range_start":"{}::{}","time_range_end":"{}::{}","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(),
start.value(),
start.unit(),
end.value(),
end.unit(),
self.file.num_rows(),
self.file.size(),
self.file.index_size()
@@ -1014,25 +1018,22 @@ impl StreamContext {
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()?;
let names: Vec<_> = output_schema
.column_schemas()
.iter()
.map(|col| &col.name)
.collect();
write!(f, ", \"projection\": {:?}", names)?;
}
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)?;
}
}
let exprs: Vec<_> =
predicate.exprs().iter().map(|e| e.to_string()).collect();
write!(f, ", \"filters\": {:?}", exprs)?;
}
}
if !self.input.files.is_empty() {
write!(f, ", files=")?;
write!(f, ", \"files\": ")?;
f.debug_list()
.entries(self.input.files.iter().map(|file| FileWrapper { file }))
.finish()?;

View File

@@ -142,36 +142,36 @@ impl fmt::Debug for ScanMetricsSet {
write!(
f,
"{{prepare_scan_cost={prepare_scan_cost:?}, \
build_reader_cost={build_reader_cost:?}, \
scan_cost={scan_cost:?}, \
convert_cost={convert_cost:?}, \
yield_cost={yield_cost:?}, \
total_cost={total_cost:?}, \
num_rows={num_rows}, \
num_batches={num_batches}, \
num_mem_ranges={num_mem_ranges}, \
num_file_ranges={num_file_ranges}, \
build_parts_cost={build_parts_cost:?}, \
rg_total={rg_total}, \
rg_fulltext_filtered={rg_fulltext_filtered}, \
rg_inverted_filtered={rg_inverted_filtered}, \
rg_minmax_filtered={rg_minmax_filtered}, \
rg_bloom_filtered={rg_bloom_filtered}, \
rows_before_filter={rows_before_filter}, \
rows_fulltext_filtered={rows_fulltext_filtered}, \
rows_inverted_filtered={rows_inverted_filtered}, \
rows_bloom_filtered={rows_bloom_filtered}, \
rows_precise_filtered={rows_precise_filtered}, \
num_sst_record_batches={num_sst_record_batches}, \
num_sst_batches={num_sst_batches}, \
num_sst_rows={num_sst_rows}, \
first_poll={first_poll:?}, \
num_series_send_timeout={num_series_send_timeout}, \
num_distributor_rows={num_distributor_rows}, \
num_distributor_batches={num_distributor_batches}, \
distributor_scan_cost={distributor_scan_cost:?}, \
distributor_yield_cost={distributor_yield_cost:?}}},"
"{{\"prepare_scan_cost\":\"{prepare_scan_cost:?}\", \
\"build_reader_cost\":\"{build_reader_cost:?}\", \
\"scan_cost\":\"{scan_cost:?}\", \
\"convert_cost\":\"{convert_cost:?}\", \
\"yield_cost\":\"{yield_cost:?}\", \
\"total_cost\":\"{total_cost:?}\", \
\"num_rows\":{num_rows}, \
\"num_batches\":{num_batches}, \
\"num_mem_ranges\":{num_mem_ranges}, \
\"num_file_ranges\":{num_file_ranges}, \
\"build_parts_cost\":\"{build_parts_cost:?}\", \
\"rg_total\":{rg_total}, \
\"rg_fulltext_filtered\":{rg_fulltext_filtered}, \
\"rg_inverted_filtered\":{rg_inverted_filtered}, \
\"rg_minmax_filtered\":{rg_minmax_filtered}, \
\"rg_bloom_filtered\":{rg_bloom_filtered}, \
\"rows_before_filter\":{rows_before_filter}, \
\"rows_fulltext_filtered\":{rows_fulltext_filtered}, \
\"rows_inverted_filtered\":{rows_inverted_filtered}, \
\"rows_bloom_filtered\":{rows_bloom_filtered}, \
\"rows_precise_filtered\":{rows_precise_filtered}, \
\"num_sst_record_batches\":{num_sst_record_batches}, \
\"num_sst_batches\":{num_sst_batches}, \
\"num_sst_rows\":{num_sst_rows}, \
\"first_poll\":\"{first_poll:?}\", \
\"num_series_send_timeout\":{num_series_send_timeout}, \
\"num_distributor_rows\":{num_distributor_rows}, \
\"num_distributor_batches\":{num_distributor_batches}, \
\"distributor_scan_cost\":\"{distributor_scan_cost:?}\", \
\"distributor_yield_cost\":\"{distributor_yield_cost:?}\"}}"
)
}
}
@@ -390,10 +390,11 @@ impl PartitionMetricsList {
/// Format verbose metrics for each partition for explain.
pub(crate) fn format_verbose_metrics(&self, f: &mut fmt::Formatter) -> fmt::Result {
let list = self.0.lock().unwrap();
write!(f, ", metrics_per_partition: ")?;
write!(f, ", \"metrics_per_partition\": ")?;
f.debug_list()
.entries(list.iter().filter_map(|p| p.as_ref()))
.finish()
.finish()?;
write!(f, "}}")
}
}
@@ -488,7 +489,11 @@ impl PartitionMetrics {
impl fmt::Debug for PartitionMetrics {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
let metrics = self.0.metrics.lock().unwrap();
write!(f, "[partition={}, {:?}]", self.0.partition, metrics)
write!(
f,
r#"{{"partition":{}, "metrics":{:?}}}"#,
self.0.partition, metrics
)
}
}

View File

@@ -12,18 +12,28 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::collections::HashSet;
use ahash::{HashMap, HashMapExt};
use api::v1::flow::{DirtyWindowRequest, WindowRange};
use api::v1::region::{
bulk_insert_request, region_request, BulkInsertRequest, RegionRequest, RegionRequestHeader,
};
use api::v1::ArrowIpc;
use arrow::array::{
Array, TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
TimestampSecondArray,
};
use arrow::datatypes::{DataType, Int64Type, TimeUnit};
use arrow::record_batch::RecordBatch;
use common_base::AffectedRows;
use common_grpc::flight::{FlightDecoder, FlightEncoder, FlightMessage};
use common_grpc::FlightData;
use common_telemetry::error;
use common_telemetry::tracing_context::TracingContext;
use snafu::ResultExt;
use store_api::storage::RegionId;
use table::metadata::TableId;
use snafu::{OptionExt, ResultExt};
use store_api::storage::{RegionId, TableId};
use table::TableRef;
use crate::insert::Inserter;
use crate::{error, metrics};
@@ -32,10 +42,11 @@ impl Inserter {
/// Handle bulk insert request.
pub async fn handle_bulk_insert(
&self,
table_id: TableId,
table: TableRef,
decoder: &mut FlightDecoder,
data: FlightData,
) -> error::Result<AffectedRows> {
let table_id = table.table_info().table_id();
let decode_timer = metrics::HANDLE_BULK_INSERT_ELAPSED
.with_label_values(&["decode_request"])
.start_timer();
@@ -48,6 +59,20 @@ impl Inserter {
return Ok(0);
};
decode_timer.observe_duration();
if let Some((min, max)) = compute_timestamp_range(
&record_batch,
&table
.table_info()
.meta
.schema
.timestamp_column()
.as_ref()
.unwrap()
.name,
)? {
// notify flownode to update dirty time windows.
self.update_flow_dirty_window(table_id, min, max);
}
metrics::BULK_REQUEST_MESSAGE_SIZE.observe(body_size as f64);
metrics::BULK_REQUEST_ROWS
.with_label_values(&["raw"])
@@ -216,4 +241,88 @@ impl Inserter {
crate::metrics::DIST_INGEST_ROW_COUNT.inc_by(rows_inserted as u64);
Ok(rows_inserted)
}
fn update_flow_dirty_window(&self, table_id: TableId, min: i64, max: i64) {
let table_flownode_set_cache = self.table_flownode_set_cache.clone();
let node_manager = self.node_manager.clone();
common_runtime::spawn_global(async move {
let result = table_flownode_set_cache
.get(table_id)
.await
.context(error::RequestInsertsSnafu);
let flownodes = match result {
Ok(flownodes) => flownodes.unwrap_or_default(),
Err(e) => {
error!(e; "Failed to get flownodes for table id: {}", table_id);
return;
}
};
let peers: HashSet<_> = flownodes.values().cloned().collect();
for peer in peers {
let node_manager = node_manager.clone();
common_runtime::spawn_global(async move {
if let Err(e) = node_manager
.flownode(&peer)
.await
.handle_mark_window_dirty(DirtyWindowRequest {
table_id,
dirty_time_ranges: vec![WindowRange {
start_value: min,
end_value: max,
}],
})
.await
.context(error::RequestInsertsSnafu)
{
error!(e; "Failed to mark time window as dirty, table: {}, min: {}, max: {}", table_id, min, max);
}
});
}
});
}
}
/// Calculate the timestamp range of record batch. Return `None` if record batch is empty.
fn compute_timestamp_range(
rb: &RecordBatch,
timestamp_index_name: &str,
) -> error::Result<Option<(i64, i64)>> {
let ts_col = rb
.column_by_name(timestamp_index_name)
.context(error::ColumnNotFoundSnafu {
msg: timestamp_index_name,
})?;
if rb.num_rows() == 0 {
return Ok(None);
}
let primitive = match ts_col.data_type() {
DataType::Timestamp(unit, _) => match unit {
TimeUnit::Second => ts_col
.as_any()
.downcast_ref::<TimestampSecondArray>()
.unwrap()
.reinterpret_cast::<Int64Type>(),
TimeUnit::Millisecond => ts_col
.as_any()
.downcast_ref::<TimestampMillisecondArray>()
.unwrap()
.reinterpret_cast::<Int64Type>(),
TimeUnit::Microsecond => ts_col
.as_any()
.downcast_ref::<TimestampMicrosecondArray>()
.unwrap()
.reinterpret_cast::<Int64Type>(),
TimeUnit::Nanosecond => ts_col
.as_any()
.downcast_ref::<TimestampNanosecondArray>()
.unwrap()
.reinterpret_cast::<Int64Type>(),
},
t => {
return error::InvalidTimeIndexTypeSnafu { ty: t.clone() }.fail();
}
};
Ok(arrow::compute::min(&primitive).zip(arrow::compute::max(&primitive)))
}

View File

@@ -837,6 +837,13 @@ pub enum Error {
#[snafu(implicit)]
location: Location,
},
#[snafu(display("Invalid time index type: {}", ty))]
InvalidTimeIndexType {
ty: arrow::datatypes::DataType,
#[snafu(implicit)]
location: Location,
},
}
pub type Result<T> = std::result::Result<T, Error>;
@@ -964,6 +971,7 @@ impl ErrorExt for Error {
Error::ColumnOptions { source, .. } => source.status_code(),
Error::DecodeFlightData { source, .. } => source.status_code(),
Error::ComputeArrow { .. } => StatusCode::Internal,
Error::InvalidTimeIndexType { .. } => StatusCode::InvalidArguments,
}
}

View File

@@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use api::v1::flow::FlowRequestHeader;
use api::v1::flow::{AdjustFlow, FlowRequestHeader};
use async_trait::async_trait;
use common_error::ext::BoxedError;
use common_function::handlers::FlowServiceHandler;
@@ -22,6 +22,7 @@ use common_query::error::Result;
use common_telemetry::tracing_context::TracingContext;
use futures::stream::FuturesUnordered;
use futures::StreamExt;
use serde_json::json;
use session::context::QueryContextRef;
use snafu::{OptionExt, ResultExt};
@@ -57,9 +58,96 @@ impl FlowServiceHandler for FlowServiceOperator {
) -> Result<api::v1::flow::FlowResponse> {
self.flush_inner(catalog, flow, ctx).await
}
async fn adjust(
&self,
catalog: &str,
flow: &str,
min_run_interval_secs: u64,
max_filter_num_per_query: usize,
ctx: QueryContextRef,
) -> Result<api::v1::flow::FlowResponse> {
self.adjust_inner(
catalog,
flow,
min_run_interval_secs,
max_filter_num_per_query,
ctx,
)
.await
}
}
impl FlowServiceOperator {
async fn adjust_inner(
&self,
catalog: &str,
flow: &str,
min_run_interval_secs: u64,
max_filter_num_per_query: usize,
ctx: QueryContextRef,
) -> Result<api::v1::flow::FlowResponse> {
let id = self
.flow_metadata_manager
.flow_name_manager()
.get(catalog, flow)
.await
.map_err(BoxedError::new)
.context(common_query::error::ExecuteSnafu)?
.context(common_meta::error::FlowNotFoundSnafu {
flow_name: format!("{}.{}", catalog, flow),
})
.map_err(BoxedError::new)
.context(common_query::error::ExecuteSnafu)?
.flow_id();
let all_flownode_peers = self
.flow_metadata_manager
.flow_route_manager()
.routes(id)
.await
.map_err(BoxedError::new)
.context(common_query::error::ExecuteSnafu)?;
// order of flownodes doesn't matter here
let all_flow_nodes = FuturesUnordered::from_iter(
all_flownode_peers
.iter()
.map(|(_key, peer)| self.node_manager.flownode(peer.peer())),
)
.collect::<Vec<_>>()
.await;
// TODO(discord9): use proper type for flow options
let options = json!({
"min_run_interval_secs": min_run_interval_secs,
"max_filter_num_per_query": max_filter_num_per_query,
});
for node in all_flow_nodes {
let _res = {
use api::v1::flow::{flow_request, FlowRequest};
let flush_req = FlowRequest {
header: Some(FlowRequestHeader {
tracing_context: TracingContext::from_current_span().to_w3c(),
query_context: Some(
common_meta::rpc::ddl::QueryContext::from(ctx.clone()).into(),
),
}),
body: Some(flow_request::Body::Adjust(AdjustFlow {
flow_id: Some(api::v1::FlowId { id }),
options: options.to_string(),
})),
};
node.handle(flush_req)
.await
.map_err(BoxedError::new)
.context(common_query::error::ExecuteSnafu)?
};
}
Ok(Default::default())
}
/// Flush the flownodes according to the flow id.
async fn flush_inner(
&self,

View File

@@ -78,7 +78,7 @@ pub struct Inserter {
catalog_manager: CatalogManagerRef,
pub(crate) partition_manager: PartitionRuleManagerRef,
pub(crate) node_manager: NodeManagerRef,
table_flownode_set_cache: TableFlownodeSetCacheRef,
pub(crate) table_flownode_set_cache: TableFlownodeSetCacheRef,
}
pub type InserterRef = Arc<Inserter>;

View File

@@ -478,6 +478,21 @@ impl QueryEngine for DatafusionQueryEngine {
fn engine_context(&self, query_ctx: QueryContextRef) -> QueryEngineContext {
let mut state = self.state.session_state();
state.config_mut().set_extension(query_ctx.clone());
// note that hints in "x-greptime-hints" is automatically parsed
// and set to query context's extension, so we can get it from query context.
if let Some(parallelism) = query_ctx.extension("query_parallelism") {
if let Ok(n) = parallelism.parse::<u64>() {
if n > 0 {
let new_cfg = state.config().clone().with_target_partitions(n as usize);
*state.config_mut() = new_cfg;
}
} else {
common_telemetry::warn!(
"Failed to parse query_parallelism: {}, using default value",
parallelism
);
}
}
QueryEngineContext::new(state, query_ctx)
}

View File

@@ -15,6 +15,7 @@
use std::collections::HashSet;
use std::sync::Arc;
use common_telemetry::debug;
use datafusion::datasource::DefaultTableSource;
use datafusion::error::Result as DfResult;
use datafusion_common::config::ConfigOptions;
@@ -148,12 +149,13 @@ struct PlanRewriter {
level: usize,
/// Simulated stack for the `rewrite` recursion
stack: Vec<(LogicalPlan, usize)>,
/// Stages to be expanded
/// Stages to be expanded, will be added as parent node of merge scan one by one
stage: Vec<LogicalPlan>,
status: RewriterStatus,
/// Partition columns of the table in current pass
partition_cols: Option<Vec<String>>,
column_requirements: HashSet<Column>,
expand_on_next_call: bool,
}
impl PlanRewriter {
@@ -174,6 +176,10 @@ impl PlanRewriter {
{
return true;
}
if self.expand_on_next_call {
self.expand_on_next_call = false;
return true;
}
match Categorizer::check_plan(plan, self.partition_cols.clone()) {
Commutativity::Commutative => {}
Commutativity::PartialCommutative => {
@@ -190,12 +196,20 @@ impl PlanRewriter {
self.stage.push(plan)
}
}
Commutativity::TransformedCommutative(transformer) => {
Commutativity::TransformedCommutative {
transformer,
expand_on_parent,
} => {
if let Some(transformer) = transformer
&& let Some(plan) = transformer(plan)
&& let Some(changed_plan) = transformer(plan)
{
self.update_column_requirements(&plan);
self.stage.push(plan)
debug!("PlanRewriter: transformed plan: {changed_plan:#?} from {plan}");
if let Some(last_stage) = changed_plan.last() {
// update the column requirements from the last stage
self.update_column_requirements(last_stage);
}
self.stage.extend(changed_plan.into_iter().rev());
self.expand_on_next_call = expand_on_parent;
}
}
Commutativity::NonCommutative
@@ -391,10 +405,21 @@ impl TreeNodeRewriter for PlanRewriter {
return Ok(Transformed::yes(node));
};
let parent = parent.clone();
// TODO(ruihang): avoid this clone
if self.should_expand(&parent.clone()) {
if self.should_expand(&parent) {
// TODO(ruihang): does this work for nodes with multiple children?;
let node = self.expand(node)?;
debug!("PlanRewriter: should expand child:\n {node}\n Of Parent: {parent}");
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));
}

View File

@@ -15,7 +15,12 @@
use std::collections::HashSet;
use std::sync::Arc;
use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNode};
use common_function::aggrs::approximate::hll::{HllState, HLL_NAME};
use common_function::aggrs::approximate::uddsketch::{UddSketchState, UDDSKETCH_STATE_NAME};
use common_telemetry::debug;
use datafusion::functions_aggregate::sum::sum_udaf;
use datafusion_common::Column;
use datafusion_expr::{Expr, LogicalPlan, Projection, UserDefinedLogicalNode};
use promql::extension_plan::{
EmptyMetric, InstantManipulate, RangeManipulate, SeriesDivide, SeriesNormalize,
};
@@ -23,12 +28,190 @@ use promql::extension_plan::{
use crate::dist_plan::merge_sort::{merge_sort_transformer, MergeSortLogicalPlan};
use crate::dist_plan::MergeScanLogicalPlan;
/// generate the upper aggregation plan that will execute on the frontend.
/// Basically a logical plan resembling the following:
/// Projection:
/// Aggregate:
///
/// from Aggregate
///
/// The upper Projection exists sole to make sure parent plan can recognize the output
/// of the upper aggregation plan.
pub fn step_aggr_to_upper_aggr(
aggr_plan: &LogicalPlan,
) -> datafusion_common::Result<Vec<LogicalPlan>> {
let LogicalPlan::Aggregate(input_aggr) = aggr_plan else {
return Err(datafusion_common::DataFusionError::Plan(
"step_aggr_to_upper_aggr only accepts Aggregate plan".to_string(),
));
};
if !is_all_aggr_exprs_steppable(&input_aggr.aggr_expr) {
return Err(datafusion_common::DataFusionError::NotImplemented(
"Some aggregate expressions are not steppable".to_string(),
));
}
let mut upper_aggr_expr = vec![];
for aggr_expr in &input_aggr.aggr_expr {
let Some(aggr_func) = get_aggr_func(aggr_expr) else {
return Err(datafusion_common::DataFusionError::NotImplemented(
"Aggregate function not found".to_string(),
));
};
let col_name = aggr_expr.qualified_name();
let input_column = Expr::Column(datafusion_common::Column::new(col_name.0, col_name.1));
let upper_func = match aggr_func.func.name() {
"sum" | "min" | "max" | "last_value" | "first_value" => {
// aggr_calc(aggr_merge(input_column))) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.args = vec![input_column.clone()];
new_aggr_func
}
"count" => {
// sum(input_column) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.func = sum_udaf();
new_aggr_func.args = vec![input_column.clone()];
new_aggr_func
}
UDDSKETCH_STATE_NAME => {
// udd_merge(bucket_size, error_rate input_column) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.func = Arc::new(UddSketchState::merge_udf_impl());
new_aggr_func.args[2] = input_column.clone();
new_aggr_func
}
HLL_NAME => {
// hll_merge(input_column) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.func = Arc::new(HllState::merge_udf_impl());
new_aggr_func.args = vec![input_column.clone()];
new_aggr_func
}
_ => {
return Err(datafusion_common::DataFusionError::NotImplemented(format!(
"Aggregate function {} is not supported for Step aggregation",
aggr_func.func.name()
)))
}
};
// deal with nested alias case
let mut new_aggr_expr = aggr_expr.clone();
{
let new_aggr_func = get_aggr_func_mut(&mut new_aggr_expr).unwrap();
*new_aggr_func = upper_func;
}
upper_aggr_expr.push(new_aggr_expr);
}
let mut new_aggr = input_aggr.clone();
// use lower aggregate plan as input, this will be replace by merge scan plan later
new_aggr.input = Arc::new(LogicalPlan::Aggregate(input_aggr.clone()));
new_aggr.aggr_expr = upper_aggr_expr;
// group by expr also need to be all ref by column to avoid duplicated computing
let mut new_group_expr = new_aggr.group_expr.clone();
for expr in &mut new_group_expr {
if let Expr::Column(_) = expr {
// already a column, no need to change
continue;
}
let col_name = expr.qualified_name();
let input_column = Expr::Column(datafusion_common::Column::new(col_name.0, col_name.1));
*expr = input_column;
}
new_aggr.group_expr = new_group_expr.clone();
let mut new_projection_exprs = new_group_expr;
// the upper aggr expr need to be aliased to the input aggr expr's name,
// so that the parent plan can recognize it.
for (lower_aggr_expr, upper_aggr_expr) in
input_aggr.aggr_expr.iter().zip(new_aggr.aggr_expr.iter())
{
let lower_col_name = lower_aggr_expr.qualified_name();
let (table, col_name) = upper_aggr_expr.qualified_name();
let aggr_out_column = Column::new(table, col_name);
let aliased_output_aggr_expr =
Expr::Column(aggr_out_column).alias_qualified(lower_col_name.0, lower_col_name.1);
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()))?;
let projection = LogicalPlan::Projection(new_projection);
// return the new logical plan
Ok(vec![projection, upper_aggr_plan])
}
/// Check if the given aggregate expression is steppable.
/// As in if it can be split into multiple steps:
/// i.e. on datanode first call `state(input)` then
/// on frontend call `calc(merge(state))` to get the final result.
///
pub fn is_all_aggr_exprs_steppable(aggr_exprs: &[Expr]) -> bool {
let step_action = HashSet::from([
"sum",
"count",
"min",
"max",
"first_value",
"last_value",
UDDSKETCH_STATE_NAME,
HLL_NAME,
]);
aggr_exprs.iter().all(|expr| {
if let Some(aggr_func) = get_aggr_func(expr) {
if aggr_func.distinct {
// Distinct aggregate functions are not steppable(yet).
return false;
}
step_action.contains(aggr_func.func.name())
} else {
false
}
})
}
pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFunction> {
let mut expr_ref = expr;
while let Expr::Alias(alias) = expr_ref {
expr_ref = &alias.expr;
}
if let Expr::AggregateFunction(aggr_func) = expr_ref {
Some(aggr_func)
} else {
None
}
}
pub fn get_aggr_func_mut(expr: &mut Expr) -> Option<&mut datafusion_expr::expr::AggregateFunction> {
let mut expr_ref = expr;
while let Expr::Alias(alias) = expr_ref {
expr_ref = &mut alias.expr;
}
if let Expr::AggregateFunction(aggr_func) = expr_ref {
Some(aggr_func)
} else {
None
}
}
#[allow(dead_code)]
pub enum Commutativity {
Commutative,
PartialCommutative,
ConditionalCommutative(Option<Transformer>),
TransformedCommutative(Option<Transformer>),
TransformedCommutative {
/// Return plans from parent to child order
transformer: Option<StageTransformer>,
/// whether the transformer changes the child to parent
expand_on_parent: bool,
},
NonCommutative,
Unimplemented,
/// For unrelated plans like DDL
@@ -55,7 +238,21 @@ impl Categorizer {
LogicalPlan::Filter(filter) => Self::check_expr(&filter.predicate),
LogicalPlan::Window(_) => Commutativity::Unimplemented,
LogicalPlan::Aggregate(aggr) => {
if !Self::check_partition(&aggr.group_expr, &partition_cols) {
let is_all_steppable = is_all_aggr_exprs_steppable(&aggr.aggr_expr);
let matches_partition = Self::check_partition(&aggr.group_expr, &partition_cols);
if !matches_partition && is_all_steppable {
debug!("Plan is steppable: {plan}");
return Commutativity::TransformedCommutative {
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()
})),
expand_on_parent: true,
};
}
if !matches_partition {
return Commutativity::NonCommutative;
}
for expr in &aggr.aggr_expr {
@@ -169,7 +366,8 @@ impl Categorizer {
| Expr::Negative(_)
| Expr::Between(_)
| Expr::Exists(_)
| Expr::InList(_) => Commutativity::Commutative,
| Expr::InList(_)
| Expr::Case(_) => Commutativity::Commutative,
Expr::ScalarFunction(_udf) => Commutativity::Commutative,
Expr::AggregateFunction(_udaf) => Commutativity::Commutative,
@@ -177,7 +375,6 @@ impl Categorizer {
| Expr::SimilarTo(_)
| Expr::IsUnknown(_)
| Expr::IsNotUnknown(_)
| Expr::Case(_)
| Expr::Cast(_)
| Expr::TryCast(_)
| Expr::WindowFunction(_)
@@ -217,6 +414,8 @@ impl Categorizer {
pub type Transformer = Arc<dyn Fn(&LogicalPlan) -> Option<LogicalPlan>>;
pub type StageTransformer = Arc<dyn Fn(&LogicalPlan) -> Option<Vec<LogicalPlan>>>;
pub fn partial_commutative_transformer(plan: &LogicalPlan) -> Option<LogicalPlan> {
Some(plan.clone())
}

View File

@@ -40,7 +40,7 @@ use futures_util::StreamExt;
use session::context::{QueryContext, QueryContextBuilder, QueryContextRef};
use session::hints::READ_PREFERENCE_HINT;
use snafu::{OptionExt, ResultExt};
use table::metadata::TableId;
use table::TableRef;
use tokio::sync::mpsc;
use crate::error::Error::UnsupportedAuthScheme;
@@ -149,8 +149,8 @@ impl GreptimeRequestHandler {
.clone()
.unwrap_or_else(common_runtime::global_runtime);
runtime.spawn(async move {
// Cached table id
let mut table_id: Option<TableId> = None;
// Cached table ref
let mut table_ref: Option<TableRef> = None;
let mut decoder = FlightDecoder::default();
while let Some(request) = stream.next().await {
@@ -169,7 +169,7 @@ impl GreptimeRequestHandler {
let timer = metrics::GRPC_BULK_INSERT_ELAPSED.start_timer();
let result = handler
.put_record_batch(&table_name, &mut table_id, &mut decoder, data)
.put_record_batch(&table_name, &mut table_ref, &mut decoder, data)
.await
.inspect_err(|e| error!(e; "Failed to handle flight record batches"));
timer.observe_duration();

View File

@@ -23,8 +23,8 @@ use common_grpc::flight::FlightDecoder;
use common_query::Output;
use session::context::QueryContextRef;
use snafu::ResultExt;
use table::metadata::TableId;
use table::table_name::TableName;
use table::TableRef;
use crate::error::{self, Result};
@@ -45,8 +45,8 @@ pub trait GrpcQueryHandler {
async fn put_record_batch(
&self,
table: &TableName,
table_id: &mut Option<TableId>,
table_name: &TableName,
table_ref: &mut Option<TableRef>,
decoder: &mut FlightDecoder,
flight_data: FlightData,
) -> std::result::Result<AffectedRows, Self::Error>;
@@ -77,13 +77,13 @@ where
async fn put_record_batch(
&self,
table: &TableName,
table_id: &mut Option<TableId>,
table_name: &TableName,
table_ref: &mut Option<TableRef>,
decoder: &mut FlightDecoder,
data: FlightData,
) -> Result<AffectedRows> {
self.0
.put_record_batch(table, table_id, decoder, data)
.put_record_batch(table_name, table_ref, decoder, data)
.await
.map_err(BoxedError::new)
.context(error::ExecuteGrpcRequestSnafu)

View File

@@ -34,7 +34,6 @@ use servers::query_handler::sql::{ServerSqlQueryHandlerRef, SqlQueryHandler};
use session::context::QueryContextRef;
use snafu::ensure;
use sql::statements::statement::Statement;
use table::metadata::TableId;
use table::table_name::TableName;
use table::TableRef;
@@ -160,15 +159,11 @@ impl GrpcQueryHandler for DummyInstance {
async fn put_record_batch(
&self,
table: &TableName,
table_id: &mut Option<TableId>,
decoder: &mut FlightDecoder,
data: FlightData,
_table_name: &TableName,
_table_ref: &mut Option<TableRef>,
_decoder: &mut FlightDecoder,
_data: FlightData,
) -> std::result::Result<AffectedRows, Self::Error> {
let _ = table;
let _ = data;
let _ = table_id;
let _ = decoder;
unimplemented!()
}
}

View File

@@ -14,10 +14,12 @@
// For the given format: `x-greptime-hints: auto_create_table=true, ttl=7d`
pub const HINTS_KEY: &str = "x-greptime-hints";
/// Deprecated, use `HINTS_KEY` instead. Notes if "x-greptime-hints" is set, keys with this prefix will be ignored.
pub const HINTS_KEY_PREFIX: &str = "x-greptime-hint-";
pub const READ_PREFERENCE_HINT: &str = "read_preference";
/// Deprecated, use `HINTS_KEY` instead.
pub const HINT_KEYS: [&str; 7] = [
"x-greptime-hint-auto_create_table",
"x-greptime-hint-ttl",

View File

@@ -44,7 +44,7 @@ explain analyze SELECT count(*) FROM system_metrics;
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[count(system_REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(system_REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 1_|
+-+-+-+

View File

@@ -0,0 +1,59 @@
CREATE TABLE IF NOT EXISTS base_table (
"time" TIMESTAMP(3) NOT NULL,
env STRING NULL,
service_name STRING NULL,
city STRING NULL,
page STRING NULL,
lcp BIGINT NULL,
fmp BIGINT NULL,
fcp BIGINT NULL,
fp BIGINT NULL,
tti BIGINT NULL,
fid BIGINT NULL,
shard_key BIGINT NULL,
TIME INDEX ("time"),
PRIMARY KEY (env, service_name)
) PARTITION ON COLUMNS (shard_key) (
shard_key < 4,
shard_key >= 4
AND shard_key < 8,
shard_key >= 8
AND shard_key < 12,
shard_key >= 12
AND shard_key < 16,
shard_key >= 16
AND shard_key < 20,
shard_key >= 20
AND shard_key < 24,
shard_key >= 24
AND shard_key < 28,
shard_key >= 28
AND shard_key < 32,
shard_key >= 32
AND shard_key < 36,
shard_key >= 36
AND shard_key < 40,
shard_key >= 40
AND shard_key < 44,
shard_key >= 44
AND shard_key < 48,
shard_key >= 48
AND shard_key < 52,
shard_key >= 52
AND shard_key < 56,
shard_key >= 56
AND shard_key < 60,
shard_key >= 60
) ENGINE = mito WITH(
'append_mode' = 'true',
'compaction.twcs.max_output_file_size' = "2GB",
'compaction.twcs.time_window' = "1h",
'compaction.type' = "twcs",
);
EXPLAIN SELECT count(*) from base_table where time >= now();
-- ERROR: Internal error: 1003
EXPLAIN ANALYZE SELECT count(*) from base_table where time >= now();
DROP TABLE base_table;

View File

@@ -111,7 +111,7 @@ EXPLAIN ANALYZE SELECT i, t AS alias_ts FROM test_pk ORDER BY t DESC LIMIT 5;
|_|_|_WindowedSortExec: expr=test_pk.t__temp__0@2 DESC num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=test_pk.t__temp__0@2 DESC num_ranges=REDACTED limit=5 REDACTED
|_|_|_ProjectionExec: expr=[i@0 as i, t@1 as t, t@1 as test_pk.t__temp__0] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -133,7 +133,7 @@ EXPLAIN ANALYZE SELECT i, t AS alias_ts FROM test_pk ORDER BY alias_ts DESC LIMI
|_|_|_SortPreservingMergeExec: [t@1 DESC], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@1 DESC num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@1 DESC num_ranges=REDACTED limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+

View File

@@ -0,0 +1,409 @@
CREATE TABLE integers(
host STRING,
i BIGINT,
ts TIMESTAMP TIME INDEX
) PARTITION ON COLUMNS (host) (
host < '550-A',
host >= '550-A'
AND host < '550-W',
host >= '550-W'
);
Affected Rows: 0
INSERT INTO integers (host, i, ts) VALUES
('550-A', 1, '2023-01-01 00:00:00'),
('550-A', 2, '2023-01-01 01:00:00'),
('550-W', 3, '2023-01-01 02:00:00'),
('550-W', 4, '2023-01-01 03:00:00');
Affected Rows: 4
-- count
EXPLAIN SELECT
count(i)
FROM
integers;
+---------------+-------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i)) AS count(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[count(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
count(i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i)) AS count(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
count(i)
FROM
integers
GROUP BY
ts;
+---------------+-----------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-----------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: count(integers.i) |
| | Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i)) AS count(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i)]] |
| | TableScan: integers] |
| physical_plan | ProjectionExec: expr=[count(integers.i)@1 as count(integers.i)] |
| | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-----------------------------------------------------------------------------------------------------------+
-- sum
EXPLAIN SELECT
sum(i)
FROM
integers;
+---------------+-------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[sum(sum(integers.i)) AS sum(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[sum(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[sum(integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[sum(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
sum(i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[sum(sum(integers.i)) AS sum(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[sum(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
sum(i)
FROM
integers
GROUP BY
ts;
+---------------+-----------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-----------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: sum(integers.i) |
| | Aggregate: groupBy=[[integers.ts]], aggr=[[sum(sum(integers.i)) AS sum(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[sum(integers.i)]] |
| | TableScan: integers] |
| physical_plan | ProjectionExec: expr=[sum(integers.i)@1 as sum(integers.i)] |
| | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-----------------------------------------------------------------------------------------------------------+
-- min
EXPLAIN SELECT
min(i)
FROM
integers;
+---------------+-------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[min(min(integers.i)) AS min(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[min(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[min(integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[min(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
min(i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[min(min(integers.i)) AS min(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[min(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[min(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[min(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
min(i)
FROM
integers
GROUP BY
ts;
+---------------+-----------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-----------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: min(integers.i) |
| | Aggregate: groupBy=[[integers.ts]], aggr=[[min(min(integers.i)) AS min(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[min(integers.i)]] |
| | TableScan: integers] |
| physical_plan | ProjectionExec: expr=[min(integers.i)@1 as min(integers.i)] |
| | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[min(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[min(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-----------------------------------------------------------------------------------------------------------+
-- max
EXPLAIN SELECT
max(i)
FROM
integers;
+---------------+-------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[max(max(integers.i)) AS max(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[max(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[max(integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[max(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
max(i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[max(max(integers.i)) AS max(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[max(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[max(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[max(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
max(i)
FROM
integers
GROUP BY
ts;
+---------------+-----------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-----------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: max(integers.i) |
| | Aggregate: groupBy=[[integers.ts]], aggr=[[max(max(integers.i)) AS max(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[max(integers.i)]] |
| | TableScan: integers] |
| physical_plan | ProjectionExec: expr=[max(integers.i)@1 as max(integers.i)] |
| | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[max(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[max(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-----------------------------------------------------------------------------------------------------------+
-- uddsketch_state
EXPLAIN SELECT
uddsketch_state(128, 0.01, i)
FROM
integers;
+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64))]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
uddsketch_state(128, 0.01, i)
FROM
integers
GROUP BY
ts;
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64))]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
uddsketch_state(128, 0.01, i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: uddsketch_state(Int64(128),Float64(0.01),integers.i) |
| | Aggregate: groupBy=[[integers.ts]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64))]] |
| | TableScan: integers] |
| physical_plan | ProjectionExec: expr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)@1 as uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[uddsketch_state(Int64(128),Float64(0.01),integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-- hll
EXPLAIN SELECT
hll(i)
FROM
integers;
+---------------+----------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+----------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[hll_merge(hll(integers.i)) AS hll(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[hll(CAST(integers.i AS Utf8))]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[hll(integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[hll(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+----------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
hll(i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[hll_merge(hll(integers.i)) AS hll(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[hll(CAST(integers.i AS Utf8))]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[hll(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[hll(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
hll(i)
FROM
integers
GROUP BY
ts;
+---------------+-----------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-----------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: hll(integers.i) |
| | Aggregate: groupBy=[[integers.ts]], aggr=[[hll_merge(hll(integers.i)) AS hll(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[hll(CAST(integers.i AS Utf8))]] |
| | TableScan: integers] |
| physical_plan | ProjectionExec: expr=[hll(integers.i)@1 as hll(integers.i)] |
| | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[hll(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[hll(integers.i)] |
| | MergeScanExec: peers=[4402341478400(1025, 0), 4402341478401(1025, 1), 4402341478402(1025, 2), ] |
| | |
+---------------+-----------------------------------------------------------------------------------------------------------------------+
DROP TABLE integers;
Affected Rows: 0

View File

@@ -0,0 +1,144 @@
CREATE TABLE integers(
host STRING,
i BIGINT,
ts TIMESTAMP TIME INDEX
) PARTITION ON COLUMNS (host) (
host < '550-A',
host >= '550-A'
AND host < '550-W',
host >= '550-W'
);
INSERT INTO integers (host, i, ts) VALUES
('550-A', 1, '2023-01-01 00:00:00'),
('550-A', 2, '2023-01-01 01:00:00'),
('550-W', 3, '2023-01-01 02:00:00'),
('550-W', 4, '2023-01-01 03:00:00');
-- count
EXPLAIN SELECT
count(i)
FROM
integers;
EXPLAIN SELECT
ts,
count(i)
FROM
integers
GROUP BY
ts;
EXPLAIN SELECT
count(i)
FROM
integers
GROUP BY
ts;
-- sum
EXPLAIN SELECT
sum(i)
FROM
integers;
EXPLAIN SELECT
ts,
sum(i)
FROM
integers
GROUP BY
ts;
EXPLAIN SELECT
sum(i)
FROM
integers
GROUP BY
ts;
-- min
EXPLAIN SELECT
min(i)
FROM
integers;
EXPLAIN SELECT
ts,
min(i)
FROM
integers
GROUP BY
ts;
EXPLAIN SELECT
min(i)
FROM
integers
GROUP BY
ts;
-- max
EXPLAIN SELECT
max(i)
FROM
integers;
EXPLAIN SELECT
ts,
max(i)
FROM
integers
GROUP BY
ts;
EXPLAIN SELECT
max(i)
FROM
integers
GROUP BY
ts;
-- uddsketch_state
EXPLAIN SELECT
uddsketch_state(128, 0.01, i)
FROM
integers;
EXPLAIN SELECT
ts,
uddsketch_state(128, 0.01, i)
FROM
integers
GROUP BY
ts;
EXPLAIN SELECT
uddsketch_state(128, 0.01, i)
FROM
integers
GROUP BY
ts;
-- hll
EXPLAIN SELECT
hll(i)
FROM
integers;
EXPLAIN SELECT
ts,
hll(i)
FROM
integers
GROUP BY
ts;
EXPLAIN SELECT
hll(i)
FROM
integers
GROUP BY
ts;
DROP TABLE integers;

View File

@@ -0,0 +1,80 @@
CREATE TABLE integers(
host STRING,
i BIGINT,
ts TIMESTAMP TIME INDEX
) PARTITION ON COLUMNS (host) (
host < '550-A',
host >= '550-A'
AND host < '550-W',
host >= '550-W'
);
Affected Rows: 0
-- count
EXPLAIN SELECT
count(i)
FROM
integers;
+---------------+-------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i)) AS count(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[]], aggr=[[count(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] |
| | CoalescePartitionsExec |
| | AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] |
| | MergeScanExec: peers=[4398046511104(1024, 0), 4398046511105(1024, 1), 4398046511106(1024, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
ts,
count(i)
FROM
integers
GROUP BY
ts;
+---------------+---------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i)) AS count(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([ts@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)] |
| | MergeScanExec: peers=[4398046511104(1024, 0), 4398046511105(1024, 1), 4398046511106(1024, 2), ] |
| | |
+---------------+---------------------------------------------------------------------------------------------------------+
EXPLAIN SELECT
date_bin('1 hour'::INTERVAL, ts),
count(i)
FROM
integers
GROUP BY
date_bin('1 hour'::INTERVAL, ts);
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[date_bin(Utf8("1 hour"),integers.ts) AS date_bin(Utf8("1 hour"),integers.ts)]], aggr=[[sum(count(integers.i)) AS count(integers.i)]] |
| | MergeScan [is_placeholder=false, input=Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]] |
| | TableScan: integers] |
| physical_plan | AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] |
| | CoalesceBatchesExec: target_batch_size=8192 |
| | RepartitionExec: partitioning=Hash([date_bin(Utf8("1 hour"),integers.ts)@0], 20), input_partitions=20 |
| | AggregateExec: mode=Partial, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] |
| | MergeScanExec: peers=[4398046511104(1024, 0), 4398046511105(1024, 1), 4398046511106(1024, 2), ] |
| | |
+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+
DROP TABLE integers;
Affected Rows: 0

View File

@@ -0,0 +1,90 @@
CREATE TABLE integers(
host STRING PRIMARY KEY,
i BIGINT,
ts TIMESTAMP TIME INDEX
) PARTITION ON COLUMNS (host) (
host < '550-A',
host >= '550-A'
AND host < '550-W',
host >= '550-W'
);
INSERT INTO integers (host, i, ts) VALUES
('550-A', 1, '2023-01-01 00:00:00'),
('550-B', 7, '2023-01-01 00:00:00'),
('550-A', 2, '2023-01-01 01:00:00'),
('550-W', 3, '2023-01-01 02:00:00'),
('550-W', 4, '2023-01-01 03:00:00');
-- count
SELECT
count(i)
FROM
integers;
EXPLAIN
SELECT
count(i)
FROM
integers;
EXPLAIN ANALYZE
SELECT
count(i)
FROM
integers;
SELECT
ts,
count(i)
FROM
integers
GROUP BY
ts;
EXPLAIN
SELECT
ts,
count(i)
FROM
integers
GROUP BY
ts;
EXPLAIN ANALYZE
SELECT
ts,
count(i)
FROM
integers
GROUP BY
ts;
SELECT
date_bin('1 hour' :: INTERVAL, ts),
count(i)
FROM
integers
GROUP BY
date_bin('1 hour' :: INTERVAL, ts);
EXPLAIN
SELECT
date_bin('1 hour' :: INTERVAL, ts),
count(i)
FROM
integers
GROUP BY
date_bin('1 hour' :: INTERVAL, ts);
EXPLAIN ANALYZE
SELECT
date_bin('1 hour' :: INTERVAL, ts),
count(i)
FROM
integers
GROUP BY
date_bin('1 hour' :: INTERVAL, ts);
DROP TABLE integers;

View File

@@ -0,0 +1,604 @@
-- base table, massive row insert to this table
CREATE TABLE IF NOT EXISTS base_table (
"time" TIMESTAMP(3) NOT NULL,
env STRING NULL,
service_name STRING NULL,
city STRING NULL,
page STRING NULL,
lcp BIGINT NULL,
fmp BIGINT NULL,
fcp BIGINT NULL,
fp BIGINT NULL,
tti BIGINT NULL,
fid BIGINT NULL,
shard_key BIGINT NULL,
TIME INDEX ("time"),
PRIMARY KEY (env, service_name)
) PARTITION ON COLUMNS (shard_key) (
shard_key < 4,
shard_key >= 4
AND shard_key < 8,
shard_key >= 8
AND shard_key < 12,
shard_key >= 12
AND shard_key < 16,
shard_key >= 16
AND shard_key < 20,
shard_key >= 20
AND shard_key < 24,
shard_key >= 24
AND shard_key < 28,
shard_key >= 28
AND shard_key < 32,
shard_key >= 32
AND shard_key < 36,
shard_key >= 36
AND shard_key < 40,
shard_key >= 40
AND shard_key < 44,
shard_key >= 44
AND shard_key < 48,
shard_key >= 48
AND shard_key < 52,
shard_key >= 52
AND shard_key < 56,
shard_key >= 56
AND shard_key < 60,
shard_key >= 60
) ENGINE = mito WITH(
'append_mode' = 'true',
'compaction.twcs.max_output_file_size' = "2GB",
'compaction.twcs.time_window' = "1h",
'compaction.type' = "twcs",
);
EXPLAIN SELECT
env,
service_name,
city,
page,
uddsketch_state(
128,
0.01,
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS lcp_state,
max(
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS max_lcp,
min(
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS min_lcp,
uddsketch_state(
128,
0.01,
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS fmp_state,
max(
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS max_fmp,
min(
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS min_fmp,
uddsketch_state(
128,
0.01,
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS fcp_state,
max(
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS max_fcp,
min(
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS min_fcp,
uddsketch_state(
128,
0.01,
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS fp_state,
max(
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS max_fp,
min(
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS min_fp,
uddsketch_state(
128,
0.01,
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS tti_state,
max(
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS max_tti,
min(
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS min_tti,
uddsketch_state(
128,
0.01,
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS fid_state,
max(
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS max_fid,
min(
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS min_fid,
max(shard_key) AS shard_key,
date_bin('60 seconds' :: INTERVAL, time) :: TIMESTAMP(0)
FROM
base_table
WHERE
(
(
lcp > 0
AND lcp < 3000000
)
OR (
fmp > 0
AND fmp < 3000000
)
OR (
fcp > 0
AND fcp < 3000000
)
OR (
fp > 0
AND fp < 3000000
)
OR (
tti > 0
AND tti < 3000000
)
OR (
fid > 0
AND fid < 3000000
)
) AND time >= now()
GROUP BY
env,
service_name,
city,
page,
date_bin('60 seconds' :: INTERVAL, time) :: TIMESTAMP(0);
EXPLAIN ANALYZE SELECT
env,
service_name,
city,
page,
uddsketch_state(
128,
0.01,
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS lcp_state,
max(
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS max_lcp,
min(
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS min_lcp,
uddsketch_state(
128,
0.01,
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS fmp_state,
max(
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS max_fmp,
min(
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS min_fmp,
uddsketch_state(
128,
0.01,
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS fcp_state,
max(
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS max_fcp,
min(
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS min_fcp,
uddsketch_state(
128,
0.01,
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS fp_state,
max(
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS max_fp,
min(
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS min_fp,
uddsketch_state(
128,
0.01,
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS tti_state,
max(
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS max_tti,
min(
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS min_tti,
uddsketch_state(
128,
0.01,
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS fid_state,
max(
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS max_fid,
min(
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS min_fid,
max(shard_key) AS shard_key,
date_bin('60 seconds' :: INTERVAL, time) :: TIMESTAMP(0)
FROM
base_table
WHERE
(
(
lcp > 0
AND lcp < 3000000
)
OR (
fmp > 0
AND fmp < 3000000
)
OR (
fcp > 0
AND fcp < 3000000
)
OR (
fp > 0
AND fp < 3000000
)
OR (
tti > 0
AND tti < 3000000
)
OR (
fid > 0
AND fid < 3000000
)
) AND time >= now()
GROUP BY
env,
service_name,
city,
page,
date_bin('60 seconds' :: INTERVAL, time) :: TIMESTAMP(0);
EXPLAIN SELECT
env,
service_name,
city,
page,
uddsketch_state(
128,
0.01,
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS lcp_state,
max(
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS max_lcp,
min(
CASE
WHEN lcp > 0
AND lcp < 3000000 THEN lcp
ELSE NULL
END
) AS min_lcp,
uddsketch_state(
128,
0.01,
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS fmp_state,
max(
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS max_fmp,
min(
CASE
WHEN fmp > 0
AND fmp < 3000000 THEN fmp
ELSE NULL
END
) AS min_fmp,
uddsketch_state(
128,
0.01,
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS fcp_state,
max(
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS max_fcp,
min(
CASE
WHEN fcp > 0
AND fcp < 3000000 THEN fcp
ELSE NULL
END
) AS min_fcp,
uddsketch_state(
128,
0.01,
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS fp_state,
max(
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS max_fp,
min(
CASE
WHEN fp > 0
AND fp < 3000000 THEN fp
ELSE NULL
END
) AS min_fp,
uddsketch_state(
128,
0.01,
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS tti_state,
max(
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS max_tti,
min(
CASE
WHEN tti > 0
AND tti < 3000000 THEN tti
ELSE NULL
END
) AS min_tti,
uddsketch_state(
128,
0.01,
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS fid_state,
max(
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS max_fid,
min(
CASE
WHEN fid > 0
AND fid < 3000000 THEN fid
ELSE NULL
END
) AS min_fid,
max(shard_key) AS shard_key,
date_bin('60 seconds' :: INTERVAL, time) :: TIMESTAMP(0)
FROM
base_table
WHERE
(
(
lcp > 0
AND lcp < 3000000
)
OR (
fmp > 0
AND fmp < 3000000
)
OR (
fcp > 0
AND fcp < 3000000
)
OR (
fp > 0
AND fp < 3000000
)
OR (
tti > 0
AND tti < 3000000
)
OR (
fid > 0
AND fid < 3000000
)
) AND time >= now()
GROUP BY
env,
service_name,
city,
page,
date_bin('60 seconds' :: INTERVAL, time) :: TIMESTAMP(0);
EXPLAIN SELECT count(*) from base_table where time >= now();
-- ERROR: Internal error: 1003
EXPLAIN ANALYZE SELECT count(*) from base_table where time >= now();
DROP TABLE base_table;

View File

@@ -72,7 +72,7 @@ ORDER BY
| 1_| 0_|_ProjectionExec: expr=[collect_time_utc@0 as collect_time, peak_current@1 as peak_current] REDACTED
|_|_|_SortPreservingMergeExec: [collect_time_utc@0 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[collect_time_utc@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+
@@ -121,7 +121,7 @@ ORDER BY
| 1_| 0_|_ProjectionExec: expr=[collect_time_utc@0 as collect_time_0, peak_current@1 as peak_current] REDACTED
|_|_|_SortPreservingMergeExec: [collect_time_utc@0 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[collect_time_utc@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+
@@ -173,7 +173,7 @@ ORDER BY
|_|_|_SortPreservingMergeExec: [collect_time@0 DESC] REDACTED
|_|_|_WindowedSortExec: expr=collect_time@0 DESC num_ranges=REDACTED REDACTED
|_|_|_PartSortExec: expr=collect_time@0 DESC num_ranges=REDACTED REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+
@@ -226,7 +226,7 @@ ORDER BY
|_|_|_WindowedSortExec: expr=collect_time@1 DESC num_ranges=REDACTED REDACTED
|_|_|_PartSortExec: expr=collect_time@1 DESC num_ranges=REDACTED REDACTED
|_|_|_ProjectionExec: expr=[collect_time_utc@1 as collect_time_utc, collect_time@0 as collect_time, peak_current@2 as peak_current] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+

View File

@@ -23,7 +23,7 @@ TQL ANALYZE (0, 10, '5s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -45,7 +45,7 @@ TQL ANALYZE (0, 10, '1s', '2s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[2000], interval=[1000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -66,7 +66,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -89,7 +89,7 @@ TQL ANALYZE VERBOSE (0, 10, '5s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: 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
|_|_|_SeriesScan: region=REDACTED, {"partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries", "projection": ["i", "j", "k"], "filters": ["j >= TimestampMillisecond(-300000, None)", "j <= TimestampMillisecond(310000, None)"], "REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -120,11 +120,11 @@ TQL ANALYZE (0, 10, '5s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
| 1_| 1_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+
@@ -151,7 +151,7 @@ TQL ANALYZE (0, 10, '5s') rate(test[10s]);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[5000], eval range=[10000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
| 1_| 1_|_ProjectionExec: expr=[j@0 as j, prom_rate(j_range,i,test.j,Int64(10000))@1 as prom_rate(j_range,i,j,Int64(10000)), k@2 as k, l@3 as l] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
@@ -160,7 +160,7 @@ TQL ANALYZE (0, 10, '5s') rate(test[10s]);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[5000], eval range=[10000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+

View File

@@ -98,7 +98,7 @@ EXPLAIN ANALYZE SELECT DISTINCT a FROM test ORDER BY a;
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 2_|
+-+-+-+

View File

@@ -32,14 +32,14 @@ select sum(val) from t group by host;
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[host@1 as host], aggr=[sum(t.val)] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_ProjectionExec: expr=[sum(t.val)@1 as sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[host@0 as host], aggr=[sum(t.val)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[host@1 as host], aggr=[sum(t.val)] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+
@@ -62,9 +62,9 @@ select sum(val) from t;
|_|_|_ProjectionExec: expr=[val@1 as val] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 0_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 1_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 1_|
+-+-+-+
@@ -90,9 +90,9 @@ select sum(val) from t group by idc;
|_|_|_ProjectionExec: expr=[val@1 as val, idc@3 as idc] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 0_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 1_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+

View File

@@ -302,13 +302,13 @@ explain analyze select tag from t where num > 6 order by ts desc limit 2;
|_|_|_WindowedSortExec: expr=ts@1 DESC num_ranges=REDACTED fetch=2 REDACTED
|_|_|_PartSortExec: expr=ts@1 DESC num_ranges=REDACTED limit=2 REDACTED
|_|_|_FilterExec: num@2 > 6, projection=[tag@0, ts@1] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_SortPreservingMergeExec: [ts@1 DESC], fetch=2 REDACTED
|_|_|_WindowedSortExec: expr=ts@1 DESC num_ranges=REDACTED fetch=2 REDACTED
|_|_|_PartSortExec: expr=ts@1 DESC num_ranges=REDACTED limit=2 REDACTED
|_|_|_FilterExec: num@2 > 6, projection=[tag@0, ts@1] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 2_|
+-+-+-+

View File

@@ -70,7 +70,7 @@ EXPLAIN ANALYZE SELECT * FROM test ORDER BY t LIMIT 5;
|_|_|_|
| 1_| 0_|_SortPreservingMergeExec: [t@1 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@1 ASC NULLS LAST num_ranges=REDACTED fetch=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -103,7 +103,7 @@ EXPLAIN ANALYZE SELECT * FROM test ORDER BY t DESC LIMIT 5;
| 1_| 0_|_SortPreservingMergeExec: [t@1 DESC], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@1 DESC num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@1 DESC num_ranges=REDACTED limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -136,7 +136,7 @@ EXPLAIN ANALYZE SELECT * FROM test where i > 2 ORDER BY t LIMIT 4;
| 1_| 0_|_SortPreservingMergeExec: [t@1 ASC NULLS LAST], fetch=4 REDACTED
|_|_|_WindowedSortExec: expr=t@1 ASC NULLS LAST num_ranges=REDACTED fetch=4 REDACTED
|_|_|_FilterExec: i@0 > 2 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -170,7 +170,7 @@ EXPLAIN ANALYZE SELECT * FROM test where i > 2 ORDER BY t DESC LIMIT 4;
|_|_|_WindowedSortExec: expr=t@1 DESC num_ranges=REDACTED fetch=4 REDACTED
|_|_|_PartSortExec: expr=t@1 DESC num_ranges=REDACTED limit=4 REDACTED
|_|_|_FilterExec: i@0 > 2 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -203,7 +203,7 @@ EXPLAIN ANALYZE SELECT * FROM test where t > 8 ORDER BY t DESC LIMIT 4;
| 1_| 0_|_SortPreservingMergeExec: [t@1 DESC], fetch=4 REDACTED
|_|_|_WindowedSortExec: expr=t@1 DESC num_ranges=REDACTED fetch=4 REDACTED
|_|_|_PartSortExec: expr=t@1 DESC num_ranges=REDACTED limit=4 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=2 (1 memtable ranges, 1 file 1 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":2, "mem_ranges":1, "files":1, "file_ranges":1} REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -285,7 +285,7 @@ EXPLAIN ANALYZE SELECT * FROM test_pk ORDER BY t LIMIT 5;
| 1_| 0_|_SortPreservingMergeExec: [t@2 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -307,7 +307,7 @@ EXPLAIN ANALYZE VERBOSE SELECT * FROM test_pk ORDER BY t LIMIT 5;
| 1_| 0_|_SortPreservingMergeExec: [t@2 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges), projection=["pk", "i", "t"], REDACTED
|_|_|_SeqScan: region=REDACTED, {"partition_count":{"count":4, "mem_ranges":1, "REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -340,7 +340,7 @@ EXPLAIN ANALYZE SELECT * FROM test_pk ORDER BY t DESC LIMIT 5;
| 1_| 0_|_SortPreservingMergeExec: [t@2 DESC], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 DESC num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 DESC num_ranges=REDACTED limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -374,7 +374,7 @@ EXPLAIN ANALYZE SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
| 1_| 0_|_SortPreservingMergeExec: [t@2 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED limit=5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=4 (1 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":4, "mem_ranges":1, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -396,7 +396,7 @@ EXPLAIN ANALYZE VERBOSE SELECT * FROM test_pk where pk > 7 ORDER BY t LIMIT 5;
| 1_| 0_|_SortPreservingMergeExec: [t@2 ASC NULLS LAST], fetch=5 REDACTED
|_|_|_WindowedSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 ASC NULLS LAST num_ranges=REDACTED 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
|_|_|_SeqScan: region=REDACTED, {"partition_count":{"count":4, "mem_ranges":1, "REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+

View File

@@ -78,7 +78,7 @@ EXPLAIN ANALYZE SELECT ts, host, min(val) RANGE '5s' FROM host ALIGN '5s';
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 0_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 10_|
+-+-+-+

View File

@@ -89,7 +89,7 @@ explain analyze select * from demo where idc='idc1';
+-+-+-+
| 0_| 0_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 0_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 2_|
+-+-+-+
@@ -115,7 +115,7 @@ explain analyze SELECT * FROM demo where host in ('test1');
+-+-+-+
| 0_| 0_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 0_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 1_|
+-+-+-+

View File

@@ -62,7 +62,7 @@ EXPLAIN ANALYZE SELECT * FROM skipping_table WHERE id = 'id2' ORDER BY `name`;
|_|_|_SortExec: expr=[name@2 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: id@1 = id2 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=3 (0 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":3, "mem_ranges":0, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 1_|
+-+-+-+
@@ -84,7 +84,7 @@ EXPLAIN ANALYZE SELECT * FROM skipping_table WHERE id = 'id5' ORDER BY `name`;
|_|_|_SortExec: expr=[name@2 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: id@1 = id5 REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=3 (0 memtable ranges, 3 file 3 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":3, "mem_ranges":0, "files":3, "file_ranges":3} REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+

View File

@@ -21,7 +21,7 @@ tql analyze (1, 3, '1s') t1{ a = "a" };
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[1000..3000], lookback=[300000], interval=[1000], time index=[b] REDACTED
|_|_|_PromSeriesDivideExec: tags=["a"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 3_|
+-+-+-+
@@ -41,7 +41,7 @@ tql analyze (1, 3, '1s') t1{ a =~ ".*" };
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[1000..3000], lookback=[300000], interval=[1000], time index=[b] REDACTED
|_|_|_PromSeriesDivideExec: tags=["a"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 6_|
+-+-+-+
@@ -61,7 +61,7 @@ tql analyze (1, 3, '1s') t1{ a =~ "a.*" };
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[1000..3000], lookback=[300000], interval=[1000], time index=[b] REDACTED
|_|_|_PromSeriesDivideExec: tags=["a"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 3_|
+-+-+-+

View File

@@ -47,7 +47,7 @@ TQL analyze (0, 10, '1s') sum by(job) (irate(cpu_usage{job="fire"}[5s])) / 1e9;
|_|_|_FilterExec: job@0 = fire AND ts@2 >= -305000000000 AND ts@2 <= 310000000000 REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SeqScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges) REDACTED
| 1_| 0_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+

View File

@@ -35,7 +35,7 @@ tql analyze (0, 10, '1s') 100 - (avg by (k) (irate(t[1m])) * 100);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[1000], eval range=[60000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+
@@ -86,7 +86,7 @@ tql analyze (0, 10, '1s') 100 - (avg by (k) (irate(t[1m])) * 100);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[1000], eval range=[60000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries 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_irate(j_range,i)@1 IS NOT NULL REDACTED
@@ -94,7 +94,7 @@ tql analyze (0, 10, '1s') 100 - (avg by (k) (irate(t[1m])) * 100);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[1000], eval range=[60000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+
@@ -146,10 +146,10 @@ tql analyze (0, 10, '1s') 100 - (avg by (k) (irate(t[1m])) * 100);
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_SortPreservingMergeExec: [k@2 ASC, l@3 ASC, j@1 ASC] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
| 1_| 1_|_SortPreservingMergeExec: [k@2 ASC, l@3 ASC, j@1 ASC] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+

View File

@@ -46,7 +46,7 @@ explain analyze
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[host@1 as host], aggr=[last_value(t.host) ORDER BY [t.ts ASC NULLS LAST], last_value(t.not_pk) ORDER BY [t.ts ASC NULLS LAST], last_value(t.val) ORDER BY [t.ts ASC NULLS LAST]] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), selector=LastRow REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "selector":"LastRow" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+

View File

@@ -104,7 +104,7 @@ EXPLAIN ANALYZE SELECT i, t AS alias_ts FROM test_pk ORDER BY t DESC LIMIT 5;
|_|_|_WindowedSortExec: expr=t@2 DESC num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=t@2 DESC num_ranges=REDACTED limit=5 REDACTED
|_|_|_ProjectionExec: expr=[i@0 as i, t@1 as alias_ts, t@1 as t] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+
@@ -126,7 +126,7 @@ EXPLAIN ANALYZE SELECT i, t AS alias_ts FROM test_pk ORDER BY alias_ts DESC LIMI
|_|_|_WindowedSortExec: expr=alias_ts@1 DESC num_ranges=REDACTED fetch=5 REDACTED
|_|_|_PartSortExec: expr=alias_ts@1 DESC num_ranges=REDACTED limit=5 REDACTED
|_|_|_ProjectionExec: expr=[i@0 as i, t@1 as alias_ts] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 5_|
+-+-+-+

View File

@@ -72,7 +72,7 @@ ORDER BY
| 1_| 0_|_SortPreservingMergeExec: [collect_time@0 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[collect_time@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_ProjectionExec: expr=[collect_time_utc@0 as collect_time, peak_current@1 as peak_current] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+
@@ -120,7 +120,7 @@ ORDER BY
| 1_| 0_|_SortPreservingMergeExec: [collect_time_0@0 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[collect_time_0@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_ProjectionExec: expr=[collect_time_utc@0 as collect_time_0, peak_current@1 as peak_current] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+
@@ -172,7 +172,7 @@ ORDER BY
|_|_|_WindowedSortExec: expr=true_collect_time@0 DESC num_ranges=REDACTED REDACTED
|_|_|_PartSortExec: expr=true_collect_time@0 DESC num_ranges=REDACTED REDACTED
|_|_|_ProjectionExec: expr=[collect_time@0 as true_collect_time, collect_time_utc@1 as collect_time, peak_current@2 as peak_current] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+
@@ -224,7 +224,7 @@ ORDER BY
|_|_|_WindowedSortExec: expr=true_collect_time@1 DESC num_ranges=REDACTED REDACTED
|_|_|_PartSortExec: expr=true_collect_time@1 DESC num_ranges=REDACTED REDACTED
|_|_|_ProjectionExec: expr=[collect_time_utc@1 as collect_time, collect_time@0 as true_collect_time, peak_current@2 as peak_current] REDACTED
|_|_|_SeqScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges) REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 8_|
+-+-+-+

View File

@@ -23,7 +23,7 @@ TQL ANALYZE (0, 10, '5s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -45,7 +45,7 @@ TQL ANALYZE (0, 10, '1s', '2s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[2000], interval=[1000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -66,7 +66,7 @@ TQL ANALYZE ('1970-01-01T00:00:00'::timestamp, '1970-01-01T00:00:00'::timestamp
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=1 (1 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -89,7 +89,7 @@ TQL ANALYZE VERBOSE (0, 10, '5s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k"] REDACTED
|_|_|_SeriesScan: 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
|_|_|_SeriesScan: region=REDACTED, {"partition_count":{"count":1, "mem_ranges":1, "files":0, "file_ranges":0}, "distribution":"PerSeries", "projection": ["i", "j", "k"], "filters": ["j >= TimestampMillisecond(-300000, None)", "j <= TimestampMillisecond(310000, None)"], "REDACTED
|_|_|_|
|_|_| Total rows: 4_|
+-+-+-+
@@ -120,11 +120,11 @@ TQL ANALYZE (0, 10, '5s') test;
|_|_|_|
| 1_| 0_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
| 1_| 1_|_PromInstantManipulateExec: range=[0..10000], lookback=[300000], interval=[5000], time index=[j] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+
@@ -150,7 +150,7 @@ TQL ANALYZE (0, 10, '5s') rate(test[10s]);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[5000], eval range=[10000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries 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_rate(j_range,i,j,Int64(10000))@1 IS NOT NULL REDACTED
@@ -158,7 +158,7 @@ TQL ANALYZE (0, 10, '5s') rate(test[10s]);
|_|_|_PromRangeManipulateExec: req range=[0..10000], interval=[5000], eval range=[10000], time index=[j] REDACTED
|_|_|_PromSeriesNormalizeExec: offset=[0], time index=[j], filter NaN: [true] REDACTED
|_|_|_PromSeriesDivideExec: tags=["k", "l"] REDACTED
|_|_|_SeriesScan: region=REDACTED, partition_count=0 (0 memtable ranges, 0 file 0 ranges), distribution=PerSeries REDACTED
|_|_|_SeriesScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0}, "distribution":"PerSeries" REDACTED
|_|_|_|
|_|_| Total rows: 0_|
+-+-+-+