chore: update datafusion family (#6675)

* chore: update datafusion family

Signed-off-by: luofucong <luofc@foxmail.com>

* fix ci

Signed-off-by: luofucong <luofc@foxmail.com>

* use official otel-arrow-rust

Signed-off-by: luofucong <luofc@foxmail.com>

* rebase

Signed-off-by: luofucong <luofc@foxmail.com>

* use the official orc-rust

Signed-off-by: luofucong <luofc@foxmail.com>

* resolve PR comments

Signed-off-by: luofucong <luofc@foxmail.com>

* remove the empty lines

Signed-off-by: luofucong <luofc@foxmail.com>

* try following PR comments

Signed-off-by: luofucong <luofc@foxmail.com>

---------

Signed-off-by: luofucong <luofc@foxmail.com>
This commit is contained in:
LFC
2025-08-15 20:41:49 +08:00
committed by GitHub
parent dfc29eb3b3
commit f9d2a89a0c
198 changed files with 4932 additions and 4196 deletions

View File

@@ -709,7 +709,7 @@ jobs:
- name: Install toolchain
uses: actions-rust-lang/setup-rust-toolchain@v1
with:
cache: false
cache: false
- name: Rust Cache
uses: Swatinem/rust-cache@v2
with:

4429
Cargo.lock generated

File diff suppressed because it is too large Load Diff

View File

@@ -98,11 +98,11 @@ rust.unexpected_cfgs = { level = "warn", check-cfg = ['cfg(tokio_unstable)'] }
# See for more detaiils: https://github.com/rust-lang/cargo/issues/11329
ahash = { version = "0.8", features = ["compile-time-rng"] }
aquamarine = "0.6"
arrow = { version = "54.2", features = ["prettyprint"] }
arrow-array = { version = "54.2", default-features = false, features = ["chrono-tz"] }
arrow-flight = "54.2"
arrow-ipc = { version = "54.2", default-features = false, features = ["lz4", "zstd"] }
arrow-schema = { version = "54.2", features = ["serde"] }
arrow = { version = "56.0", features = ["prettyprint"] }
arrow-array = { version = "56.0", default-features = false, features = ["chrono-tz"] }
arrow-flight = "56.0"
arrow-ipc = { version = "56.0", default-features = false, features = ["lz4", "zstd"] }
arrow-schema = { version = "56.0", features = ["serde"] }
async-stream = "0.3"
async-trait = "0.1"
# Remember to update axum-extra, axum-macros when updating axum
@@ -121,26 +121,27 @@ clap = { version = "4.4", features = ["derive"] }
config = "0.13.0"
crossbeam-utils = "0.8"
dashmap = "6.1"
datafusion = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-common = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-expr = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-functions = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-functions-aggregate-common = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-optimizer = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-physical-expr = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-physical-plan = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-sql = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion-substrait = { git = "https://github.com/waynexia/arrow-datafusion.git", rev = "12c0381babd52c681043957e9d6ee083a03f7646" }
datafusion = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-common = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-expr = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-functions = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-functions-aggregate-common = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-optimizer = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-orc = { git = "https://github.com/GreptimeTeam/datafusion-orc", rev = "a0a5f902158f153119316eaeec868cff3fc8a99d" }
datafusion-physical-expr = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-physical-plan = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-sql = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
datafusion-substrait = { git = "https://github.com/GreptimeTeam/datafusion.git", rev = "7d5214512740b4dfb742b6b3d91ed9affcc2c9d0" }
deadpool = "0.12"
deadpool-postgres = "0.14"
derive_builder = "0.20"
dotenv = "0.15"
either = "1.15"
etcd-client = "0.14"
etcd-client = { git = "https://github.com/GreptimeTeam/etcd-client", rev = "f62df834f0cffda355eba96691fe1a9a332b75a7" }
fst = "0.4.7"
futures = "0.3"
futures-util = "0.3"
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "9edc8c4d2a7e3c7d8e4982a6af94426da4057687" }
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "03007c30a2d2bf1acb4374cf5e92df9b0bd8844e" }
hex = "0.4"
http = "1"
humantime = "2.1"
@@ -159,7 +160,7 @@ nalgebra = "0.33"
nix = { version = "0.30.1", default-features = false, features = ["event", "fs", "process"] }
notify = "8.0"
num_cpus = "1.16"
object_store_opendal = "0.50"
object_store_opendal = { git = "https://github.com/apache/opendal", rev = "0ba8574b6d08d209056704d28a9a114beb3c1022" }
once_cell = "1.18"
opentelemetry-proto = { version = "0.27", features = [
"gen-tonic",
@@ -170,7 +171,7 @@ opentelemetry-proto = { version = "0.27", features = [
] }
ordered-float = { version = "4.3", features = ["serde"] }
parking_lot = "0.12"
parquet = { version = "54.2", default-features = false, features = ["arrow", "async", "object_store"] }
parquet = { version = "56.0", default-features = false, features = ["arrow", "async", "object_store"] }
paste = "1.0"
pin-project = "1.0"
pretty_assertions = "1.4.0"
@@ -205,10 +206,10 @@ simd-json = "0.15"
similar-asserts = "1.6.0"
smallvec = { version = "1", features = ["serde"] }
snafu = "0.8"
sqlparser = { git = "https://github.com/GreptimeTeam/sqlparser-rs.git", rev = "df6fcca80ce903f5beef7002cd2c1b062e7024f8", features = [
sqlparser = { git = "https://github.com/GreptimeTeam/sqlparser-rs.git", rev = "39e4fc94c3c741981f77e9d63b5ce8c02e0a27ea", features = [
"visitor",
"serde",
] } # branch = "v0.54.x"
] } # branch = "v0.55.x"
sqlx = { version = "0.8", features = [
"runtime-tokio-rustls",
"mysql",
@@ -218,20 +219,20 @@ sqlx = { version = "0.8", features = [
strum = { version = "0.27", features = ["derive"] }
sysinfo = "0.33"
tempfile = "3"
tokio = { version = "1.40", features = ["full"] }
tokio = { version = "1.47", features = ["full"] }
tokio-postgres = "0.7"
tokio-rustls = { version = "0.26.2", default-features = false }
tokio-stream = "0.1"
tokio-util = { version = "0.7", features = ["io-util", "compat"] }
toml = "0.8.8"
tonic = { version = "0.12", features = ["tls", "gzip", "zstd"] }
tonic = { version = "0.13", features = ["tls-ring", "gzip", "zstd"] }
tower = "0.5"
tower-http = "0.6"
tracing = "0.1"
tracing-appender = "0.2"
tracing-subscriber = { version = "0.3", features = ["env-filter", "json", "fmt"] }
typetag = "0.2"
uuid = { version = "1.7", features = ["serde", "v4", "fast-rng"] }
uuid = { version = "1.17", features = ["serde", "v4", "fast-rng"] }
vrl = "0.25"
zstd = "0.13"
# DO_NOT_REMOVE_THIS: END_OF_EXTERNAL_DEPENDENCIES
@@ -290,7 +291,7 @@ mito-codec = { path = "src/mito-codec" }
mito2 = { path = "src/mito2" }
object-store = { path = "src/object-store" }
operator = { path = "src/operator" }
otel-arrow-rust = { git = "https://github.com/open-telemetry/otel-arrow", rev = "5d551412d2a12e689cde4d84c14ef29e36784e51", features = [
otel-arrow-rust = { git = "https://github.com/GreptimeTeam/otel-arrow", rev = "2d64b7c0fa95642028a8205b36fe9ea0b023ec59", features = [
"server",
] }
partition = { path = "src/partition" }

View File

@@ -19,6 +19,3 @@ paste.workspace = true
prost.workspace = true
serde_json.workspace = true
snafu.workspace = true
[build-dependencies]
tonic-build = "0.11"

View File

@@ -133,7 +133,7 @@ impl Predicate {
let Expr::Column(c) = *expr else {
unreachable!();
};
let Expr::Literal(ScalarValue::Utf8(Some(pattern))) = *pattern else {
let Expr::Literal(ScalarValue::Utf8(Some(pattern)), _) = *pattern else {
unreachable!();
};
@@ -148,8 +148,8 @@ impl Predicate {
// left OP right
Expr::BinaryExpr(bin) => match (*bin.left, bin.op, *bin.right) {
// left == right
(Expr::Literal(scalar), Operator::Eq, Expr::Column(c))
| (Expr::Column(c), Operator::Eq, Expr::Literal(scalar)) => {
(Expr::Literal(scalar, _), Operator::Eq, Expr::Column(c))
| (Expr::Column(c), Operator::Eq, Expr::Literal(scalar, _)) => {
let Ok(v) = Value::try_from(scalar) else {
return None;
};
@@ -157,8 +157,8 @@ impl Predicate {
Some(Predicate::Eq(c.name, v))
}
// left != right
(Expr::Literal(scalar), Operator::NotEq, Expr::Column(c))
| (Expr::Column(c), Operator::NotEq, Expr::Literal(scalar)) => {
(Expr::Literal(scalar, _), Operator::NotEq, Expr::Column(c))
| (Expr::Column(c), Operator::NotEq, Expr::Literal(scalar, _)) => {
let Ok(v) = Value::try_from(scalar) else {
return None;
};
@@ -189,7 +189,7 @@ impl Predicate {
let mut values = Vec::with_capacity(list.len());
for scalar in list {
// Safety: checked by `is_all_scalars`
let Expr::Literal(scalar) = scalar else {
let Expr::Literal(scalar, _) = scalar else {
unreachable!();
};
@@ -237,7 +237,7 @@ fn like_utf8(s: &str, pattern: &str, case_insensitive: &bool) -> Option<bool> {
}
fn is_string_literal(expr: &Expr) -> bool {
matches!(expr, Expr::Literal(ScalarValue::Utf8(Some(_))))
matches!(expr, Expr::Literal(ScalarValue::Utf8(Some(_)), _))
}
fn is_column(expr: &Expr) -> bool {
@@ -286,14 +286,14 @@ impl Predicates {
/// Returns true when the values are all [`DfExpr::Literal`].
fn is_all_scalars(list: &[Expr]) -> bool {
list.iter().all(|v| matches!(v, Expr::Literal(_)))
list.iter().all(|v| matches!(v, Expr::Literal(_, _)))
}
#[cfg(test)]
mod tests {
use datafusion::common::{Column, ScalarValue};
use datafusion::common::Column;
use datafusion::logical_expr::expr::InList;
use datafusion::logical_expr::BinaryExpr;
use datafusion::logical_expr::{BinaryExpr, Literal};
use super::*;
@@ -378,7 +378,7 @@ mod tests {
let expr = Expr::Like(Like {
negated: false,
expr: Box::new(column("a")),
pattern: Box::new(string_literal("%abc")),
pattern: Box::new("%abc".lit()),
case_insensitive: true,
escape_char: None,
});
@@ -405,7 +405,7 @@ mod tests {
let expr = Expr::Like(Like {
negated: false,
expr: Box::new(column("a")),
pattern: Box::new(string_literal("%abc")),
pattern: Box::new("%abc".lit()),
case_insensitive: false,
escape_char: None,
});
@@ -425,7 +425,7 @@ mod tests {
let expr = Expr::Like(Like {
negated: true,
expr: Box::new(column("a")),
pattern: Box::new(string_literal("%abc")),
pattern: Box::new("%abc".lit()),
case_insensitive: true,
escape_char: None,
});
@@ -440,10 +440,6 @@ mod tests {
Expr::Column(Column::from_name(name))
}
fn string_literal(v: &str) -> Expr {
Expr::Literal(ScalarValue::Utf8(Some(v.to_string())))
}
fn match_string_value(v: &Value, expected: &str) -> bool {
matches!(v, Value::String(bs) if bs.as_utf8() == expected)
}
@@ -463,13 +459,13 @@ mod tests {
let expr1 = Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("a")),
op: Operator::Eq,
right: Box::new(string_literal("a_value")),
right: Box::new("a_value".lit()),
});
let expr2 = Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("b")),
op: Operator::NotEq,
right: Box::new(string_literal("b_value")),
right: Box::new("b_value".lit()),
});
(expr1, expr2)
@@ -508,7 +504,7 @@ mod tests {
let inlist_expr = Expr::InList(InList {
expr: Box::new(column("a")),
list: vec![string_literal("a1"), string_literal("a2")],
list: vec!["a1".lit(), "a2".lit()],
negated: false,
});
@@ -518,7 +514,7 @@ mod tests {
let inlist_expr = Expr::InList(InList {
expr: Box::new(column("a")),
list: vec![string_literal("a1"), string_literal("a2")],
list: vec!["a1".lit(), "a2".lit()],
negated: true,
});
let inlist_p = Predicate::from_expr(inlist_expr).unwrap();

View File

@@ -32,7 +32,7 @@ use dummy_catalog::DummyCatalogList;
use table::TableRef;
use crate::error::{
CastManagerSnafu, DatafusionSnafu, DecodePlanSnafu, GetViewCacheSnafu, ProjectViewColumnsSnafu,
CastManagerSnafu, DecodePlanSnafu, GetViewCacheSnafu, ProjectViewColumnsSnafu,
QueryAccessDeniedSnafu, Result, TableNotExistSnafu, ViewInfoNotFoundSnafu,
ViewPlanColumnsChangedSnafu,
};
@@ -199,10 +199,10 @@ impl DfTableSourceProvider {
logical_plan
};
Ok(Arc::new(
ViewTable::try_new(logical_plan, Some(view_info.definition.to_string()))
.context(DatafusionSnafu)?,
))
Ok(Arc::new(ViewTable::new(
logical_plan,
Some(view_info.definition.to_string()),
)))
}
}

View File

@@ -25,19 +25,17 @@ common-error.workspace = true
common-macro.workspace = true
common-recordbatch.workspace = true
common-runtime.workspace = true
common-telemetry.workspace = true
datafusion.workspace = true
datafusion-orc.workspace = true
datatypes.workspace = true
derive_builder.workspace = true
futures.workspace = true
lazy_static.workspace = true
object-store.workspace = true
object_store_opendal.workspace = true
orc-rust = { git = "https://github.com/datafusion-contrib/orc-rust", rev = "3134cab581a8e91b942d6a23aca2916ea965f6bb", default-features = false, features = [
"async",
] }
orc-rust = { version = "0.6.3", default-features = false, features = ["async"] }
parquet.workspace = true
paste.workspace = true
rand.workspace = true
regex = "1.7"
serde.workspace = true
snafu.workspace = true
@@ -47,6 +45,4 @@ tokio-util.workspace = true
url = "2.3"
[dev-dependencies]
common-telemetry.workspace = true
common-test-util.workspace = true
uuid.workspace = true

View File

@@ -12,16 +12,11 @@
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use arrow_schema::{ArrowError, Schema, SchemaRef};
use arrow_schema::Schema;
use async_trait::async_trait;
use bytes::Bytes;
use common_recordbatch::adapter::RecordBatchStreamTypeAdapter;
use datafusion::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener};
use datafusion::error::{DataFusionError, Result as DfResult};
use futures::future::BoxFuture;
use futures::{FutureExt, StreamExt, TryStreamExt};
use futures::FutureExt;
use object_store::ObjectStore;
use orc_rust::arrow_reader::ArrowReaderBuilder;
use orc_rust::async_arrow_reader::ArrowStreamReader;
@@ -97,67 +92,6 @@ impl FileFormat for OrcFormat {
}
}
#[derive(Debug, Clone)]
pub struct OrcOpener {
object_store: Arc<ObjectStore>,
output_schema: SchemaRef,
projection: Option<Vec<usize>>,
}
impl OrcOpener {
pub fn new(
object_store: ObjectStore,
output_schema: SchemaRef,
projection: Option<Vec<usize>>,
) -> Self {
Self {
object_store: Arc::from(object_store),
output_schema,
projection,
}
}
}
impl FileOpener for OrcOpener {
fn open(&self, meta: FileMeta) -> DfResult<FileOpenFuture> {
let object_store = self.object_store.clone();
let projected_schema = if let Some(projection) = &self.projection {
let projected_schema = self
.output_schema
.project(projection)
.map_err(|e| DataFusionError::External(Box::new(e)))?;
Arc::new(projected_schema)
} else {
self.output_schema.clone()
};
let projection = self.projection.clone();
Ok(Box::pin(async move {
let path = meta.location().to_string();
let meta = object_store
.stat(&path)
.await
.map_err(|e| DataFusionError::External(Box::new(e)))?;
let reader = object_store
.reader(&path)
.await
.map_err(|e| DataFusionError::External(Box::new(e)))?;
let stream_reader =
new_orc_stream_reader(ReaderAdapter::new(reader, meta.content_length()))
.await
.map_err(|e| DataFusionError::External(Box::new(e)))?;
let stream =
RecordBatchStreamTypeAdapter::new(projected_schema, stream_reader, projection);
let adopted = stream.map_err(|e| ArrowError::ExternalError(Box::new(e)));
Ok(adopted.boxed())
}))
}
}
#[cfg(test)]
mod tests {
use common_test_util::find_workspace_path;

View File

@@ -31,6 +31,7 @@ use datatypes::schema::SchemaRef;
use futures::future::BoxFuture;
use futures::StreamExt;
use object_store::{FuturesAsyncReader, ObjectStore};
use parquet::arrow::arrow_reader::ArrowReaderOptions;
use parquet::arrow::AsyncArrowWriter;
use parquet::basic::{Compression, Encoding, ZstdLevel};
use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder};
@@ -65,7 +66,7 @@ impl FileFormat for ParquetFormat {
.compat();
let metadata = reader
.get_metadata()
.get_metadata(None)
.await
.context(error::ReadParquetSnafuSnafu)?;
@@ -146,7 +147,7 @@ impl LazyParquetFileReader {
impl AsyncFileReader for LazyParquetFileReader {
fn get_bytes(
&mut self,
range: std::ops::Range<usize>,
range: std::ops::Range<u64>,
) -> BoxFuture<'_, ParquetResult<bytes::Bytes>> {
Box::pin(async move {
self.maybe_initialize()
@@ -157,13 +158,16 @@ impl AsyncFileReader for LazyParquetFileReader {
})
}
fn get_metadata(&mut self) -> BoxFuture<'_, ParquetResult<Arc<ParquetMetaData>>> {
fn get_metadata<'a>(
&'a mut self,
options: Option<&'a ArrowReaderOptions>,
) -> BoxFuture<'a, parquet::errors::Result<Arc<ParquetMetaData>>> {
Box::pin(async move {
self.maybe_initialize()
.await
.map_err(|e| ParquetError::External(Box::new(e)))?;
// Safety: Must initialized
self.reader.as_mut().unwrap().get_metadata().await
self.reader.as_mut().unwrap().get_metadata(options).await
})
}
}

View File

@@ -19,35 +19,39 @@ use std::vec;
use common_test_util::find_workspace_path;
use datafusion::assert_batches_eq;
use datafusion::datasource::file_format::file_compression_type::FileCompressionType;
use datafusion::datasource::physical_plan::{
CsvConfig, CsvOpener, FileOpener, FileScanConfig, FileStream, JsonOpener, ParquetExec,
CsvSource, FileScanConfig, FileSource, FileStream, JsonSource, ParquetSource,
};
use datafusion::datasource::source::DataSourceExec;
use datafusion::execution::context::TaskContext;
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::prelude::SessionContext;
use datafusion_orc::OrcSource;
use futures::StreamExt;
use object_store::ObjectStore;
use super::FORMAT_TYPE;
use crate::file_format::orc::{OrcFormat, OrcOpener};
use crate::file_format::parquet::DefaultParquetFileReaderFactory;
use crate::file_format::{FileFormat, Format};
use crate::file_format::{FileFormat, Format, OrcFormat};
use crate::test_util::{scan_config, test_basic_schema, test_store};
use crate::{error, test_util};
struct Test<'a, T: FileOpener> {
struct Test<'a> {
config: FileScanConfig,
opener: T,
file_source: Arc<dyn FileSource>,
expected: Vec<&'a str>,
}
impl<T: FileOpener> Test<'_, T> {
pub async fn run(self) {
impl Test<'_> {
async fn run(self, store: &ObjectStore) {
let store = Arc::new(object_store_opendal::OpendalStore::new(store.clone()));
let file_opener = self.file_source.create_file_opener(store, &self.config, 0);
let result = FileStream::new(
&self.config,
0,
self.opener,
file_opener,
&ExecutionPlanMetricsSet::new(),
)
.unwrap()
@@ -62,26 +66,16 @@ impl<T: FileOpener> Test<'_, T> {
#[tokio::test]
async fn test_json_opener() {
let store = test_store("/");
let store = Arc::new(object_store_opendal::OpendalStore::new(store));
let schema = test_basic_schema();
let json_opener = || {
JsonOpener::new(
test_util::TEST_BATCH_SIZE,
schema.clone(),
FileCompressionType::UNCOMPRESSED,
store.clone(),
)
};
let file_source = Arc::new(JsonSource::new()).with_batch_size(test_util::TEST_BATCH_SIZE);
let path = &find_workspace_path("/src/common/datasource/tests/json/basic.json")
.display()
.to_string();
let tests = [
Test {
config: scan_config(schema.clone(), None, path),
opener: json_opener(),
config: scan_config(schema.clone(), None, path, file_source.clone()),
file_source: file_source.clone(),
expected: vec![
"+-----+-------+",
"| num | str |",
@@ -93,8 +87,8 @@ async fn test_json_opener() {
],
},
Test {
config: scan_config(schema.clone(), Some(1), path),
opener: json_opener(),
config: scan_config(schema, Some(1), path, file_source.clone()),
file_source,
expected: vec![
"+-----+------+",
"| num | str |",
@@ -106,37 +100,26 @@ async fn test_json_opener() {
];
for test in tests {
test.run().await;
test.run(&store).await;
}
}
#[tokio::test]
async fn test_csv_opener() {
let store = test_store("/");
let store = Arc::new(object_store_opendal::OpendalStore::new(store));
let schema = test_basic_schema();
let path = &find_workspace_path("/src/common/datasource/tests/csv/basic.csv")
.display()
.to_string();
let csv_config = Arc::new(CsvConfig::new(
test_util::TEST_BATCH_SIZE,
schema.clone(),
None,
true,
b',',
b'"',
None,
store,
None,
));
let csv_opener = || CsvOpener::new(csv_config.clone(), FileCompressionType::UNCOMPRESSED);
let file_source = CsvSource::new(true, b',', b'"')
.with_batch_size(test_util::TEST_BATCH_SIZE)
.with_schema(schema.clone());
let tests = [
Test {
config: scan_config(schema.clone(), None, path),
opener: csv_opener(),
config: scan_config(schema.clone(), None, path, file_source.clone()),
file_source: file_source.clone(),
expected: vec![
"+-----+-------+",
"| num | str |",
@@ -148,8 +131,8 @@ async fn test_csv_opener() {
],
},
Test {
config: scan_config(schema.clone(), Some(1), path),
opener: csv_opener(),
config: scan_config(schema, Some(1), path, file_source.clone()),
file_source,
expected: vec![
"+-----+------+",
"| num | str |",
@@ -161,7 +144,7 @@ async fn test_csv_opener() {
];
for test in tests {
test.run().await;
test.run(&store).await;
}
}
@@ -174,12 +157,12 @@ async fn test_parquet_exec() {
let path = &find_workspace_path("/src/common/datasource/tests/parquet/basic.parquet")
.display()
.to_string();
let base_config = scan_config(schema.clone(), None, path);
let exec = ParquetExec::builder(base_config)
.with_parquet_file_reader_factory(Arc::new(DefaultParquetFileReaderFactory::new(store)))
.build();
let parquet_source = ParquetSource::default()
.with_parquet_file_reader_factory(Arc::new(DefaultParquetFileReaderFactory::new(store)));
let config = scan_config(schema, None, path, Arc::new(parquet_source));
let exec = DataSourceExec::from_data_source(config);
let ctx = SessionContext::new();
let context = Arc::new(TaskContext::from(&ctx));
@@ -208,20 +191,18 @@ async fn test_parquet_exec() {
#[tokio::test]
async fn test_orc_opener() {
let root = find_workspace_path("/src/common/datasource/tests/orc")
let path = &find_workspace_path("/src/common/datasource/tests/orc/test.orc")
.display()
.to_string();
let store = test_store(&root);
let schema = OrcFormat.infer_schema(&store, "test.orc").await.unwrap();
let schema = Arc::new(schema);
let orc_opener = OrcOpener::new(store.clone(), schema.clone(), None);
let path = "test.orc";
let store = test_store("/");
let schema = Arc::new(OrcFormat.infer_schema(&store, path).await.unwrap());
let file_source = Arc::new(OrcSource::default());
let tests = [
Test {
config: scan_config(schema.clone(), None, path),
opener: orc_opener.clone(),
config: scan_config(schema.clone(), None, path, file_source.clone()),
file_source: file_source.clone(),
expected: vec![
"+----------+-----+-------+------------+-----+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+----------------------------+-------------+",
"| double_a | a | b | str_direct | d | e | f | int_short_repeated | int_neg_short_repeated | int_delta | int_neg_delta | int_direct | int_neg_direct | bigint_direct | bigint_neg_direct | bigint_other | utf8_increase | utf8_decrease | timestamp_simple | date_simple |",
@@ -235,8 +216,8 @@ async fn test_orc_opener() {
],
},
Test {
config: scan_config(schema.clone(), Some(1), path),
opener: orc_opener.clone(),
config: scan_config(schema.clone(), Some(1), path, file_source.clone()),
file_source,
expected: vec![
"+----------+-----+------+------------+---+-----+-------+--------------------+------------------------+-----------+---------------+------------+----------------+---------------+-------------------+--------------+---------------+---------------+-------------------------+-------------+",
"| double_a | a | b | str_direct | d | e | f | int_short_repeated | int_neg_short_repeated | int_delta | int_neg_delta | int_direct | int_neg_direct | bigint_direct | bigint_neg_direct | bigint_other | utf8_increase | utf8_decrease | timestamp_simple | date_simple |",
@@ -248,7 +229,7 @@ async fn test_orc_opener() {
];
for test in tests {
test.run().await;
test.run(&store).await;
}
}

View File

@@ -16,12 +16,12 @@ use std::sync::Arc;
use arrow_schema::{DataType, Field, Schema, SchemaRef};
use common_test_util::temp_dir::{create_temp_dir, TempDir};
use datafusion::common::{Constraints, Statistics};
use datafusion::datasource::file_format::file_compression_type::FileCompressionType;
use datafusion::datasource::listing::PartitionedFile;
use datafusion::datasource::object_store::ObjectStoreUrl;
use datafusion::datasource::physical_plan::{
CsvConfig, CsvOpener, FileScanConfig, FileStream, JsonOpener,
CsvSource, FileGroup, FileScanConfig, FileScanConfigBuilder, FileSource, FileStream,
JsonOpener, JsonSource,
};
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
use object_store::services::Fs;
@@ -68,21 +68,20 @@ pub fn test_basic_schema() -> SchemaRef {
Arc::new(schema)
}
pub fn scan_config(file_schema: SchemaRef, limit: Option<usize>, filename: &str) -> FileScanConfig {
pub(crate) fn scan_config(
file_schema: SchemaRef,
limit: Option<usize>,
filename: &str,
file_source: Arc<dyn FileSource>,
) -> FileScanConfig {
// object_store only recognize the Unix style path, so make it happy.
let filename = &filename.replace('\\', "/");
let statistics = Statistics::new_unknown(file_schema.as_ref());
FileScanConfig {
object_store_url: ObjectStoreUrl::parse("empty://").unwrap(), // won't be used
file_schema,
file_groups: vec![vec![PartitionedFile::new(filename.to_string(), 10)]],
constraints: Constraints::empty(),
statistics,
projection: None,
limit,
table_partition_cols: vec![],
output_ordering: vec![],
}
let file_group = FileGroup::new(vec![PartitionedFile::new(filename.to_string(), 4096)]);
FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema, file_source)
.with_file_group(file_group)
.with_limit(limit)
.build()
}
pub async fn setup_stream_to_json_test(origin_path: &str, threshold: impl Fn(usize) -> usize) {
@@ -99,9 +98,14 @@ pub async fn setup_stream_to_json_test(origin_path: &str, threshold: impl Fn(usi
let size = store.read(origin_path).await.unwrap().len();
let config = scan_config(schema.clone(), None, origin_path);
let stream = FileStream::new(&config, 0, json_opener, &ExecutionPlanMetricsSet::new()).unwrap();
let config = scan_config(schema, None, origin_path, Arc::new(JsonSource::new()));
let stream = FileStream::new(
&config,
0,
Arc::new(json_opener),
&ExecutionPlanMetricsSet::new(),
)
.unwrap();
let (tmp_store, dir) = test_tmp_store("test_stream_to_json");
@@ -127,24 +131,17 @@ pub async fn setup_stream_to_csv_test(origin_path: &str, threshold: impl Fn(usiz
let schema = test_basic_schema();
let csv_config = Arc::new(CsvConfig::new(
TEST_BATCH_SIZE,
schema.clone(),
None,
true,
b',',
b'"',
None,
Arc::new(object_store_opendal::OpendalStore::new(store.clone())),
None,
));
let csv_opener = CsvOpener::new(csv_config, FileCompressionType::UNCOMPRESSED);
let csv_source = CsvSource::new(true, b',', b'"')
.with_schema(schema.clone())
.with_batch_size(TEST_BATCH_SIZE);
let config = scan_config(schema, None, origin_path, csv_source.clone());
let size = store.read(origin_path).await.unwrap().len();
let config = scan_config(schema.clone(), None, origin_path);
let csv_opener = csv_source.create_file_opener(
Arc::new(object_store_opendal::OpendalStore::new(store.clone())),
&config,
0,
);
let stream = FileStream::new(&config, 0, csv_opener, &ExecutionPlanMetricsSet::new()).unwrap();
let (tmp_store, dir) = test_tmp_store("test_stream_to_csv");

View File

@@ -25,14 +25,14 @@
use std::sync::Arc;
use arrow::array::StructArray;
use arrow_schema::Fields;
use arrow_schema::{FieldRef, Fields};
use common_telemetry::debug;
use datafusion::functions_aggregate::all_default_aggregate_functions;
use datafusion::optimizer::analyzer::type_coercion::TypeCoercion;
use datafusion::optimizer::AnalyzerRule;
use datafusion::physical_planner::create_aggregate_expr_and_maybe_filter;
use datafusion_common::{Column, ScalarValue};
use datafusion_expr::expr::AggregateFunction;
use datafusion_expr::expr::{AggregateFunction, AggregateFunctionParams};
use datafusion_expr::function::StateFieldsArgs;
use datafusion_expr::{
Accumulator, Aggregate, AggregateUDF, AggregateUDFImpl, Expr, ExprSchemable, LogicalPlan,
@@ -146,6 +146,7 @@ impl StateMergeHelper {
};
let original_input_types = aggr_func
.params
.args
.iter()
.map(|e| e.get_type(&aggr.input.schema()))
@@ -156,11 +157,7 @@ impl StateMergeHelper {
let expr = AggregateFunction {
func: Arc::new(state_func.into()),
args: aggr_func.args.clone(),
distinct: aggr_func.distinct,
filter: aggr_func.filter.clone(),
order_by: aggr_func.order_by.clone(),
null_treatment: aggr_func.null_treatment,
params: aggr_func.params.clone(),
};
let expr = Expr::AggregateFunction(expr);
let lower_state_output_col_name = expr.schema_name().to_string();
@@ -182,11 +179,10 @@ impl StateMergeHelper {
let arg = Expr::Column(Column::new_unqualified(lower_state_output_col_name));
let expr = AggregateFunction {
func: Arc::new(merge_func.into()),
args: vec![arg],
distinct: aggr_func.distinct,
filter: aggr_func.filter.clone(),
order_by: aggr_func.order_by.clone(),
null_treatment: aggr_func.null_treatment,
params: AggregateFunctionParams {
args: vec![arg],
..aggr_func.params.clone()
},
};
// alias to the original aggregate expr's schema name, so parent plan can refer to it
@@ -247,15 +243,8 @@ impl StateWrapper {
pub fn deduce_aggr_return_type(
&self,
acc_args: &datafusion_expr::function::AccumulatorArgs,
) -> datafusion_common::Result<DataType> {
let input_exprs = acc_args.exprs;
let input_schema = acc_args.schema;
let input_types = input_exprs
.iter()
.map(|e| e.data_type(input_schema))
.collect::<Result<Vec<_>, _>>()?;
let return_type = self.inner.return_type(&input_types)?;
Ok(return_type)
) -> datafusion_common::Result<FieldRef> {
self.inner.return_field(acc_args.schema.fields())
}
}
@@ -265,14 +254,13 @@ impl AggregateUDFImpl for StateWrapper {
acc_args: datafusion_expr::function::AccumulatorArgs<'b>,
) -> datafusion_common::Result<Box<dyn Accumulator>> {
// fix and recover proper acc args for the original aggregate function.
let state_type = acc_args.return_type.clone();
let state_type = acc_args.return_type().clone();
let inner = {
let old_return_type = self.deduce_aggr_return_type(&acc_args)?;
let acc_args = datafusion_expr::function::AccumulatorArgs {
return_type: &old_return_type,
return_field: self.deduce_aggr_return_type(&acc_args)?,
schema: acc_args.schema,
ignore_nulls: acc_args.ignore_nulls,
ordering_req: acc_args.ordering_req,
order_bys: acc_args.order_bys,
is_reversed: acc_args.is_reversed,
name: acc_args.name,
is_distinct: acc_args.is_distinct,
@@ -297,11 +285,15 @@ impl AggregateUDFImpl for StateWrapper {
/// Return state_fields as the output struct type.
///
fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result<DataType> {
let old_return_type = self.inner.return_type(arg_types)?;
let input_fields = &arg_types
.iter()
.map(|x| Arc::new(Field::new("x", x.clone(), false)))
.collect::<Vec<_>>();
let state_fields_args = StateFieldsArgs {
name: self.inner().name(),
input_types: arg_types,
return_type: &old_return_type,
input_fields,
return_field: self.inner.return_field(input_fields)?,
// TODO(discord9): how to get this?, probably ok?
ordering_fields: &[],
is_distinct: false,
@@ -315,12 +307,11 @@ impl AggregateUDFImpl for StateWrapper {
fn state_fields(
&self,
args: datafusion_expr::function::StateFieldsArgs,
) -> datafusion_common::Result<Vec<Field>> {
let old_return_type = self.inner.return_type(args.input_types)?;
) -> datafusion_common::Result<Vec<FieldRef>> {
let state_fields_args = StateFieldsArgs {
name: args.name,
input_types: args.input_types,
return_type: &old_return_type,
input_fields: args.input_fields,
return_field: self.inner.return_field(args.input_fields)?,
ordering_fields: args.ordering_fields,
is_distinct: args.is_distinct,
};
@@ -502,7 +493,7 @@ impl AggregateUDFImpl for MergeWrapper {
fn state_fields(
&self,
_args: datafusion_expr::function::StateFieldsArgs,
) -> datafusion_common::Result<Vec<Field>> {
) -> datafusion_common::Result<Vec<FieldRef>> {
self.original_phy_expr.state_fields()
}
}

View File

@@ -35,7 +35,7 @@ use datafusion::prelude::SessionContext;
use datafusion_common::{Column, TableReference};
use datafusion_expr::expr::AggregateFunction;
use datafusion_expr::sqlparser::ast::NullTreatment;
use datafusion_expr::{Aggregate, Expr, LogicalPlan, SortExpr, TableScan};
use datafusion_expr::{lit, Aggregate, Expr, LogicalPlan, SortExpr, TableScan};
use datafusion_physical_expr::aggregate::AggregateExprBuilder;
use datafusion_physical_expr::{EquivalenceProperties, Partitioning};
use datatypes::arrow_array::StringArray;
@@ -234,7 +234,7 @@ async fn test_sum_udaf() {
vec![Expr::Column(Column::new_unqualified("number"))],
false,
None,
None,
vec![],
None,
))],
)
@@ -250,7 +250,7 @@ async fn test_sum_udaf() {
vec![Expr::Column(Column::new_unqualified("number"))],
false,
None,
None,
vec![],
None,
))],
)
@@ -290,7 +290,7 @@ async fn test_sum_udaf() {
vec![Expr::Column(Column::new_unqualified("__sum_state(number)"))],
false,
None,
None,
vec![],
None,
))
.alias("sum(number)")],
@@ -378,7 +378,7 @@ async fn test_avg_udaf() {
vec![Expr::Column(Column::new_unqualified("number"))],
false,
None,
None,
vec![],
None,
))],
)
@@ -395,7 +395,7 @@ async fn test_avg_udaf() {
vec![Expr::Column(Column::new_unqualified("number"))],
false,
None,
None,
vec![],
None,
))],
)
@@ -449,7 +449,7 @@ async fn test_avg_udaf() {
vec![Expr::Column(Column::new_unqualified("__avg_state(number)"))],
false,
None,
None,
vec![],
None,
))
.alias("avg(number)")],
@@ -551,7 +551,7 @@ async fn test_udaf_correct_eval_result() {
expected_fn: Option<ExpectedFn>,
distinct: bool,
filter: Option<Box<Expr>>,
order_by: Option<Vec<SortExpr>>,
order_by: Vec<SortExpr>,
null_treatment: Option<NullTreatment>,
}
type ExpectedFn = fn(ArrayRef) -> bool;
@@ -575,7 +575,7 @@ async fn test_udaf_correct_eval_result() {
expected_fn: None,
distinct: false,
filter: None,
order_by: None,
order_by: vec![],
null_treatment: None,
},
TestCase {
@@ -596,7 +596,7 @@ async fn test_udaf_correct_eval_result() {
expected_fn: None,
distinct: false,
filter: None,
order_by: None,
order_by: vec![],
null_treatment: None,
},
TestCase {
@@ -619,7 +619,7 @@ async fn test_udaf_correct_eval_result() {
expected_fn: None,
distinct: false,
filter: None,
order_by: None,
order_by: vec![],
null_treatment: None,
},
TestCase {
@@ -630,8 +630,8 @@ async fn test_udaf_correct_eval_result() {
true,
)])),
args: vec![
Expr::Literal(ScalarValue::Int64(Some(128))),
Expr::Literal(ScalarValue::Float64(Some(0.05))),
lit(128i64),
lit(0.05f64),
Expr::Column(Column::new_unqualified("number")),
],
input: vec![Arc::new(Float64Array::from(vec![
@@ -659,7 +659,7 @@ async fn test_udaf_correct_eval_result() {
}),
distinct: false,
filter: None,
order_by: None,
order_by: vec![],
null_treatment: None,
},
TestCase {
@@ -690,7 +690,7 @@ async fn test_udaf_correct_eval_result() {
}),
distinct: false,
filter: None,
order_by: None,
order_by: vec![],
null_treatment: None,
},
// TODO(discord9): udd_merge/hll_merge/geo_path/quantile_aggr tests

View File

@@ -41,7 +41,7 @@ use datatypes::arrow::array::{
Array, ArrayRef, AsArray, BooleanArray, Int64Array, ListArray, UInt64Array,
};
use datatypes::arrow::buffer::{OffsetBuffer, ScalarBuffer};
use datatypes::arrow::datatypes::{DataType, Field};
use datatypes::arrow::datatypes::{DataType, Field, FieldRef};
use crate::function_registry::FunctionRegistry;
@@ -94,14 +94,14 @@ impl AggregateUDFImpl for CountHash {
false
}
fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<Field>> {
Ok(vec![Field::new_list(
fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<FieldRef>> {
Ok(vec![Arc::new(Field::new_list(
format_state_name(args.name, "count_hash"),
Field::new_list_field(DataType::UInt64, true),
// For count_hash accumulator, null list item stands for an
// empty value set (i.e., all NULL value so far for that group).
true,
)])
))])
}
fn accumulator(&self, acc_args: AccumulatorArgs) -> Result<Box<dyn Accumulator>> {

View File

@@ -23,6 +23,7 @@ use datafusion::logical_expr::Volatility;
use datatypes::value::{Value, ValueRef};
use session::context::QueryContextRef;
use snafu::{ensure, ResultExt};
use sql::ast::ObjectNamePartExt;
use sql::parser::ParserContext;
use store_api::storage::ConcreteDataType;
@@ -85,9 +86,9 @@ fn parse_flush_flow(
let (catalog_name, flow_name) = match &obj_name.0[..] {
[flow_name] => (
query_ctx.current_catalog().to_string(),
flow_name.value.clone(),
flow_name.to_string_unquoted(),
),
[catalog, flow_name] => (catalog.value.clone(), flow_name.value.clone()),
[catalog, flow_name] => (catalog.to_string_unquoted(), flow_name.to_string_unquoted()),
_ => {
return InvalidFuncArgsSnafu {
err_msg: format!(

View File

@@ -113,6 +113,8 @@ mod tests {
use common_query::prelude::ScalarValue;
use datafusion::arrow::array::BooleanArray;
use datafusion_common::config::ConfigOptions;
use datatypes::arrow::datatypes::Field;
use datatypes::data_type::ConcreteDataType;
use datatypes::prelude::VectorRef;
use datatypes::vectors::{BooleanVector, ConstantVector};
@@ -162,10 +164,21 @@ mod tests {
]))),
];
let arg_fields = vec![
Arc::new(Field::new("a", args[0].data_type(), false)),
Arc::new(Field::new("b", args[1].data_type(), false)),
];
let return_field = Arc::new(Field::new(
"x",
ConcreteDataType::boolean_datatype().as_arrow_type(),
false,
));
let args = ScalarFunctionArgs {
args,
arg_fields,
number_rows: 4,
return_type: &ConcreteDataType::boolean_datatype().as_arrow_type(),
return_field,
config_options: Arc::new(ConfigOptions::default()),
};
match udf.invoke_with_args(args).unwrap() {
datafusion_expr::ColumnarValue::Array(x) => {

View File

@@ -20,7 +20,7 @@ common-telemetry.workspace = true
common-time.workspace = true
dashmap.workspace = true
datatypes.workspace = true
flatbuffers = "24"
flatbuffers = "25.2"
hyper.workspace = true
lazy_static.workspace = true
prost.workspace = true

View File

@@ -25,7 +25,7 @@ use common_recordbatch::DfRecordBatch;
use datatypes::arrow;
use datatypes::arrow::array::ArrayRef;
use datatypes::arrow::buffer::Buffer;
use datatypes::arrow::datatypes::{Schema as ArrowSchema, SchemaRef};
use datatypes::arrow::datatypes::{DataType, Schema as ArrowSchema, SchemaRef};
use datatypes::arrow::error::ArrowError;
use datatypes::arrow::ipc::{convert, reader, root_as_message, writer, MessageHeader};
use flatbuffers::FlatBufferBuilder;
@@ -91,7 +91,15 @@ impl FlightEncoder {
/// be encoded to exactly one [FlightData].
pub fn encode(&mut self, flight_message: FlightMessage) -> Vec1<FlightData> {
match flight_message {
FlightMessage::Schema(schema) => vec1![self.encode_schema(schema.as_ref())],
FlightMessage::Schema(schema) => {
schema.fields().iter().for_each(|x| {
if matches!(x.data_type(), DataType::Dictionary(_, _)) {
self.dictionary_tracker.next_dict_id();
}
});
vec1![self.encode_schema(schema.as_ref())]
}
FlightMessage::RecordBatch(record_batch) => {
let (encoded_dictionaries, encoded_batch) = self
.data_gen

View File

@@ -24,7 +24,7 @@ use datafusion::error::Result as DatafusionResult;
use datafusion::logical_expr::{LogicalPlan, LogicalPlanBuilder};
use datafusion_common::{Column, TableReference};
use datafusion_expr::dml::InsertOp;
use datafusion_expr::{col, DmlStatement, WriteOp};
use datafusion_expr::{col, DmlStatement, TableSource, WriteOp};
pub use expr::{build_filter_from_timestamp, build_same_type_ts_filter};
use snafu::ResultExt;
@@ -131,7 +131,7 @@ pub fn breakup_insert_plan(
/// create a `insert into table_name <input>` logical plan
pub fn add_insert_to_logical_plan(
table_name: TableName,
table_schema: datafusion_common::DFSchemaRef,
target: Arc<dyn TableSource>,
input: LogicalPlan,
) -> Result<LogicalPlan> {
let table_name = TableReference::Full {
@@ -142,7 +142,7 @@ pub fn add_insert_to_logical_plan(
let plan = LogicalPlan::Dml(DmlStatement::new(
table_name,
table_schema,
target,
WriteOp::Insert(InsertOp::Append),
Arc::new(input),
));

View File

@@ -87,29 +87,31 @@ fn timestamp_to_literal(timestamp: &Timestamp) -> Expr {
TimeUnit::Microsecond => ScalarValue::TimestampMicrosecond(Some(timestamp.value()), None),
TimeUnit::Nanosecond => ScalarValue::TimestampNanosecond(Some(timestamp.value()), None),
};
Expr::Literal(scalar_value)
Expr::Literal(scalar_value, None)
}
#[cfg(test)]
mod tests {
use datafusion_expr::Literal;
use super::*;
#[test]
fn test_timestamp_to_literal() {
let timestamp = Timestamp::new(123456789, TimeUnit::Second);
let expected = Expr::Literal(ScalarValue::TimestampSecond(Some(123456789), None));
let expected = ScalarValue::TimestampSecond(Some(123456789), None).lit();
assert_eq!(timestamp_to_literal(&timestamp), expected);
let timestamp = Timestamp::new(123456789, TimeUnit::Millisecond);
let expected = Expr::Literal(ScalarValue::TimestampMillisecond(Some(123456789), None));
let expected = ScalarValue::TimestampMillisecond(Some(123456789), None).lit();
assert_eq!(timestamp_to_literal(&timestamp), expected);
let timestamp = Timestamp::new(123456789, TimeUnit::Microsecond);
let expected = Expr::Literal(ScalarValue::TimestampMicrosecond(Some(123456789), None));
let expected = ScalarValue::TimestampMicrosecond(Some(123456789), None).lit();
assert_eq!(timestamp_to_literal(&timestamp), expected);
let timestamp = Timestamp::new(123456789, TimeUnit::Nanosecond);
let expected = Expr::Literal(ScalarValue::TimestampNanosecond(Some(123456789), None));
let expected = ScalarValue::TimestampNanosecond(Some(123456789), None).lit();
assert_eq!(timestamp_to_literal(&timestamp), expected);
}
}

View File

@@ -26,7 +26,7 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs};
use datafusion_expr::{
Accumulator, AccumulatorFactoryFunction, AggregateUDF as DfAggregateUdf, AggregateUDFImpl,
};
use datatypes::arrow::datatypes::DataType as ArrowDataType;
use datatypes::arrow::datatypes::{DataType as ArrowDataType, FieldRef};
use datatypes::data_type::DataType;
use crate::function::{
@@ -129,14 +129,14 @@ impl AggregateUDFImpl for DfUdafAdapter {
(self.accumulator)(acc_args)
}
fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<Field>> {
fn state_fields(&self, args: StateFieldsArgs) -> Result<Vec<FieldRef>> {
let state_types = self.creator.state_types()?;
let fields = state_types
.into_iter()
.enumerate()
.map(|(i, t)| {
let name = format!("{}_{i}", args.name);
Field::new(name, t.as_arrow_type(), true)
Arc::new(Field::new(name, t.as_arrow_type(), true))
})
.collect::<Vec<_>>();
Ok(fields)

View File

@@ -134,11 +134,11 @@ where
b.and_then(|b| {
let projected_column = b.project(&projection)?;
if projected_column.schema().fields.len() != projected_schema.fields.len() {
return Err(DataFusionError::ArrowError(ArrowError::SchemaError(format!(
return Err(DataFusionError::ArrowError(Box::new(ArrowError::SchemaError(format!(
"Trying to cast a RecordBatch into an incompatible schema. RecordBatch: {}, Target: {}",
projected_column.schema(),
projected_schema,
)), None));
))), None));
}
let mut columns = Vec::with_capacity(projected_schema.fields.len());
@@ -360,7 +360,7 @@ impl ExecutionPlanVisitor for MetricCollector {
// skip if no metric available
let Some(metric) = plan.metrics() else {
self.record_batch_metrics.plan_metrics.push(PlanMetrics {
plan: std::any::type_name::<Self>().to_string(),
plan: plan.name().to_string(),
level: self.current_level,
metrics: vec![],
});

View File

@@ -77,7 +77,7 @@ impl SimpleFilterEvaluator {
_ => return None,
}
let Expr::Literal(val) = lit.lit() else {
let Expr::Literal(val, _) = lit.lit() else {
return None;
};
@@ -143,8 +143,8 @@ impl SimpleFilterEvaluator {
// swap the expr if it is in the form of `literal` `op` `col`
let mut op = binary.op;
let (lhs, rhs) = match (&*binary.left, &*binary.right) {
(Expr::Column(ref col), Expr::Literal(ref lit)) => (col, lit),
(Expr::Literal(ref lit), Expr::Column(ref col)) => {
(Expr::Column(col), Expr::Literal(lit, _)) => (col, lit),
(Expr::Literal(lit, _), Expr::Column(col)) => {
// safety: The previous check ensures the operator is able to swap.
op = op.swap().unwrap();
(col, lit)
@@ -359,15 +359,15 @@ mod test {
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("foo"))),
op: Operator::Plus,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1.lit()),
});
assert!(SimpleFilterEvaluator::try_new(&expr).is_none());
// two literal is not supported
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
left: Box::new(1.lit()),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1.lit()),
});
assert!(SimpleFilterEvaluator::try_new(&expr).is_none());
@@ -384,10 +384,10 @@ mod test {
left: Box::new(Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("foo"))),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1.lit()),
})),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1.lit()),
});
assert!(SimpleFilterEvaluator::try_new(&expr).is_none());
}
@@ -398,13 +398,13 @@ mod test {
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("foo"))),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1.lit()),
});
let _ = SimpleFilterEvaluator::try_new(&expr).unwrap();
// swap operands
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
left: Box::new(1.lit()),
op: Operator::Lt,
right: Box::new(Expr::Column(Column::from_name("foo"))),
});
@@ -418,7 +418,7 @@ mod test {
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("foo"))),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1i64.lit()),
});
let evaluator = SimpleFilterEvaluator::try_new(&expr).unwrap();
@@ -440,7 +440,7 @@ mod test {
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("foo"))),
op: Operator::Lt,
right: Box::new(Expr::Literal(ScalarValue::Int64(Some(1)))),
right: Box::new(1i64.lit()),
});
let evaluator = SimpleFilterEvaluator::try_new(&expr).unwrap();

View File

@@ -16,6 +16,7 @@ use common_time::timezone::Timezone;
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::constraint::{CURRENT_TIMESTAMP, CURRENT_TIMESTAMP_FN};
use datatypes::schema::ColumnDefaultConstraint;
use sqlparser::ast::ValueWithSpan;
pub use sqlparser::ast::{
visit_expressions_mut, visit_statements_mut, BinaryOperator, ColumnDef, ColumnOption,
ColumnOptionDef, DataType, Expr, Function, FunctionArg, FunctionArgExpr, FunctionArguments,
@@ -38,7 +39,7 @@ pub fn parse_column_default_constraint(
{
let default_constraint = match &opt.option {
ColumnOption::Default(Expr::Value(v)) => ColumnDefaultConstraint::Value(
sql_value_to_value(column_name, data_type, v, timezone, None, false)?,
sql_value_to_value(column_name, data_type, &v.value, timezone, None, false)?,
),
ColumnOption::Default(Expr::Function(func)) => {
let mut func = format!("{func}").to_lowercase();
@@ -54,8 +55,13 @@ pub fn parse_column_default_constraint(
// Specialized process for handling numerical inputs to prevent
// overflow errors during the parsing of negative numbers,
// See https://github.com/GreptimeTeam/greptimedb/issues/4351
if let (UnaryOperator::Minus, Expr::Value(SqlValue::Number(n, _))) =
(op, expr.as_ref())
if let (
UnaryOperator::Minus,
Expr::Value(ValueWithSpan {
value: SqlValue::Number(n, _),
span: _,
}),
) = (op, expr.as_ref())
{
return Ok(Some(ColumnDefaultConstraint::Value(sql_number_to_value(
data_type,
@@ -64,8 +70,14 @@ pub fn parse_column_default_constraint(
}
if let Expr::Value(v) = &**expr {
let value =
sql_value_to_value(column_name, data_type, v, timezone, Some(*op), false)?;
let value = sql_value_to_value(
column_name,
data_type,
&v.value,
timezone,
Some(*op),
false,
)?;
ColumnDefaultConstraint::Value(value)
} else {
return UnsupportedDefaultValueSnafu {
@@ -85,7 +97,7 @@ pub fn parse_column_default_constraint(
_ => {
return UnsupportedDefaultValueSnafu {
column_name,
expr: Expr::Value(SqlValue::Null),
expr: Expr::Value(SqlValue::Null.into()),
}
.fail();
}
@@ -113,7 +125,7 @@ mod test {
let opts = vec![
ColumnOptionDef {
name: None,
option: ColumnOption::Default(Expr::Value(bool_value)),
option: ColumnOption::Default(Expr::Value(bool_value.into())),
},
ColumnOptionDef {
name: None,
@@ -139,7 +151,9 @@ mod test {
name: None,
option: ColumnOption::Default(Expr::UnaryOp {
op: UnaryOperator::Minus,
expr: Box::new(Expr::Value(SqlValue::Number("32768".to_string(), false))),
expr: Box::new(Expr::Value(
SqlValue::Number("32768".to_string(), false).into(),
)),
}),
}];
@@ -161,10 +175,9 @@ mod test {
fn test_incorrect_default_value_issue_3479() {
let opts = vec![ColumnOptionDef {
name: None,
option: ColumnOption::Default(Expr::Value(SqlValue::Number(
"0.047318541668048164".into(),
false,
))),
option: ColumnOption::Default(Expr::Value(
SqlValue::Number("0.047318541668048164".into(), false).into(),
)),
}];
let constraint = parse_column_default_constraint(
"coll",

View File

@@ -444,8 +444,12 @@ impl TryFrom<&ArrowDataType> for ConcreteDataType {
ArrowDataType::Date32 => Self::date_datatype(),
ArrowDataType::Timestamp(u, _) => ConcreteDataType::from_arrow_time_unit(u),
ArrowDataType::Interval(u) => ConcreteDataType::from_arrow_interval_unit(u),
ArrowDataType::Binary | ArrowDataType::LargeBinary => Self::binary_datatype(),
ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 => Self::string_datatype(),
ArrowDataType::Binary | ArrowDataType::LargeBinary | ArrowDataType::BinaryView => {
Self::binary_datatype()
}
ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 | ArrowDataType::Utf8View => {
Self::string_datatype()
}
ArrowDataType::List(field) => Self::List(ListType::new(
ConcreteDataType::from_arrow_type(field.data_type()),
)),
@@ -466,8 +470,6 @@ impl TryFrom<&ArrowDataType> for ConcreteDataType {
ArrowDataType::Float16
| ArrowDataType::Date64
| ArrowDataType::FixedSizeBinary(_)
| ArrowDataType::BinaryView
| ArrowDataType::Utf8View
| ArrowDataType::ListView(_)
| ArrowDataType::FixedSizeList(_, _)
| ArrowDataType::LargeList(_)
@@ -475,7 +477,9 @@ impl TryFrom<&ArrowDataType> for ConcreteDataType {
| ArrowDataType::Union(_, _)
| ArrowDataType::Decimal256(_, _)
| ArrowDataType::Map(_, _)
| ArrowDataType::RunEndEncoded(_, _) => {
| ArrowDataType::RunEndEncoded(_, _)
| ArrowDataType::Decimal32(_, _)
| ArrowDataType::Decimal64(_, _) => {
return error::UnsupportedArrowTypeSnafu {
arrow_type: dt.clone(),
}

View File

@@ -264,7 +264,9 @@ impl Helper {
ArrowDataType::Null => Arc::new(NullVector::try_from_arrow_array(array)?),
ArrowDataType::Boolean => Arc::new(BooleanVector::try_from_arrow_array(array)?),
ArrowDataType::Binary => Arc::new(BinaryVector::try_from_arrow_array(array)?),
ArrowDataType::LargeBinary | ArrowDataType::FixedSizeBinary(_) => {
ArrowDataType::LargeBinary
| ArrowDataType::FixedSizeBinary(_)
| ArrowDataType::BinaryView => {
let array = arrow::compute::cast(array.as_ref(), &ArrowDataType::Binary)
.context(crate::error::ArrowComputeSnafu)?;
Arc::new(BinaryVector::try_from_arrow_array(array)?)
@@ -280,7 +282,7 @@ impl Helper {
ArrowDataType::Float32 => Arc::new(Float32Vector::try_from_arrow_array(array)?),
ArrowDataType::Float64 => Arc::new(Float64Vector::try_from_arrow_array(array)?),
ArrowDataType::Utf8 => Arc::new(StringVector::try_from_arrow_array(array)?),
ArrowDataType::LargeUtf8 => {
ArrowDataType::LargeUtf8 | ArrowDataType::Utf8View => {
let array = arrow::compute::cast(array.as_ref(), &ArrowDataType::Utf8)
.context(crate::error::ArrowComputeSnafu)?;
Arc::new(StringVector::try_from_arrow_array(array)?)
@@ -377,11 +379,11 @@ impl Helper {
| ArrowDataType::Decimal256(_, _)
| ArrowDataType::Map(_, _)
| ArrowDataType::RunEndEncoded(_, _)
| ArrowDataType::BinaryView
| ArrowDataType::Utf8View
| ArrowDataType::ListView(_)
| ArrowDataType::LargeListView(_)
| ArrowDataType::Date64 => {
| ArrowDataType::Date64
| ArrowDataType::Decimal32(_, _)
| ArrowDataType::Decimal64(_, _) => {
return error::UnsupportedArrowTypeSnafu {
arrow_type: array.as_ref().data_type().clone(),
}

View File

@@ -26,6 +26,7 @@ common-test-util = { workspace = true, optional = true }
common-time.workspace = true
datafusion.workspace = true
datafusion-expr.workspace = true
datafusion-orc.workspace = true
datatypes.workspace = true
futures.workspace = true
object-store.workspace = true

View File

@@ -33,7 +33,7 @@ use futures::Stream;
use snafu::{ensure, OptionExt, ResultExt};
use store_api::storage::ScanRequest;
use self::file_stream::{CreateScanPlanContext, ScanPlanConfig};
use self::file_stream::ScanPlanConfig;
use crate::error::{
BuildBackendSnafu, CreateDefaultSnafu, ExtractColumnFromFilterSnafu,
MissingColumnNoDefaultSnafu, ProjectSchemaSnafu, ProjectionOutOfBoundsSnafu, Result,
@@ -50,7 +50,6 @@ impl FileRegion {
let file_stream = file_stream::create_stream(
&self.format,
&CreateScanPlanContext::default(),
&ScanPlanConfig {
file_schema,
files: &self.file_options.files,

View File

@@ -15,18 +15,17 @@
use std::sync::Arc;
use common_datasource::file_format::csv::CsvFormat;
use common_datasource::file_format::json::JsonFormat;
use common_datasource::file_format::orc::{OrcFormat, OrcOpener};
use common_datasource::file_format::parquet::{DefaultParquetFileReaderFactory, ParquetFormat};
use common_datasource::file_format::parquet::DefaultParquetFileReaderFactory;
use common_datasource::file_format::Format;
use common_recordbatch::adapter::RecordBatchStreamAdapter;
use common_recordbatch::SendableRecordBatchStream;
use datafusion::common::{Constraints, Statistics, ToDFSchema};
use datafusion::common::ToDFSchema;
use datafusion::datasource::listing::PartitionedFile;
use datafusion::datasource::object_store::ObjectStoreUrl;
use datafusion::datasource::physical_plan::{
CsvConfig, CsvOpener, FileOpener, FileScanConfig, FileStream, JsonOpener, ParquetExec,
CsvSource, FileGroup, FileScanConfigBuilder, FileSource, FileStream, JsonSource, ParquetSource,
};
use datafusion::datasource::source::DataSourceExec;
use datafusion::physical_expr::create_physical_expr;
use datafusion::physical_expr::execution_props::ExecutionProps;
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
@@ -34,6 +33,7 @@ use datafusion::physical_plan::ExecutionPlan;
use datafusion::prelude::SessionContext;
use datafusion_expr::expr::Expr;
use datafusion_expr::utils::conjunction;
use datafusion_orc::OrcSource;
use datatypes::arrow::datatypes::Schema as ArrowSchema;
use datatypes::schema::SchemaRef;
use object_store::ObjectStore;
@@ -43,86 +43,37 @@ use crate::error::{self, Result};
const DEFAULT_BATCH_SIZE: usize = 8192;
#[derive(Debug, Clone, Copy, Default)]
pub struct CreateScanPlanContext {}
fn build_csv_opener(
fn build_record_batch_stream(
scan_plan_config: &ScanPlanConfig,
file_schema: Arc<ArrowSchema>,
config: &ScanPlanConfig,
format: &CsvFormat,
) -> CsvOpener {
let csv_config = Arc::new(CsvConfig::new(
DEFAULT_BATCH_SIZE,
file_schema,
config.projection.cloned(),
format.has_header,
format.delimiter,
b'"',
None,
Arc::new(object_store_opendal::OpendalStore::new(
config.store.clone(),
)),
None,
));
CsvOpener::new(csv_config, format.compression_type.into())
}
fn build_json_opener(
file_schema: Arc<ArrowSchema>,
config: &ScanPlanConfig,
format: &JsonFormat,
) -> Result<JsonOpener> {
let projected_schema = if let Some(projection) = config.projection {
Arc::new(
file_schema
.project(projection)
.context(error::ProjectArrowSchemaSnafu)?,
)
} else {
file_schema
};
let store = object_store_opendal::OpendalStore::new(config.store.clone());
Ok(JsonOpener::new(
DEFAULT_BATCH_SIZE,
projected_schema,
format.compression_type.into(),
Arc::new(store),
))
}
fn build_orc_opener(output_schema: Arc<ArrowSchema>, config: &ScanPlanConfig) -> Result<OrcOpener> {
Ok(OrcOpener::new(
config.store.clone(),
output_schema,
config.projection.cloned(),
))
}
fn build_record_batch_stream<T: FileOpener + Send + 'static>(
opener: T,
file_schema: Arc<ArrowSchema>,
files: &[String],
projection: Option<&Vec<usize>>,
limit: Option<usize>,
file_source: Arc<dyn FileSource>,
) -> Result<SendableRecordBatchStream> {
let statistics = Statistics::new_unknown(file_schema.as_ref());
let files = scan_plan_config
.files
.iter()
.map(|filename| PartitionedFile::new(filename.to_string(), 0))
.collect::<Vec<_>>();
let config = FileScanConfigBuilder::new(
ObjectStoreUrl::local_filesystem(),
file_schema,
file_source.clone(),
)
.with_projection(scan_plan_config.projection.cloned())
.with_limit(limit)
.with_file_group(FileGroup::new(files))
.build();
let store = Arc::new(object_store_opendal::OpendalStore::new(
scan_plan_config.store.clone(),
));
let file_opener = file_source.create_file_opener(store, &config, 0);
let stream = FileStream::new(
&FileScanConfig {
object_store_url: ObjectStoreUrl::parse("empty://").unwrap(), // won't be used
file_schema,
file_groups: vec![files
.iter()
.map(|filename| PartitionedFile::new(filename.to_string(), 0))
.collect::<Vec<_>>()],
statistics,
projection: projection.cloned(),
limit,
table_partition_cols: vec![],
output_ordering: vec![],
constraints: Constraints::empty(),
},
&config,
0, // partition: hard-code
opener,
file_opener,
&ExecutionPlanMetricsSet::new(),
)
.context(error::BuildStreamSnafu)?;
@@ -132,34 +83,32 @@ fn build_record_batch_stream<T: FileOpener + Send + 'static>(
}
fn new_csv_stream(
_ctx: &CreateScanPlanContext,
config: &ScanPlanConfig,
format: &CsvFormat,
) -> Result<SendableRecordBatchStream> {
let file_schema = config.file_schema.arrow_schema().clone();
let opener = build_csv_opener(file_schema.clone(), config, format);
// push down limit only if there is no filter
let limit = config.filters.is_empty().then_some(config.limit).flatten();
build_record_batch_stream(opener, file_schema, config.files, config.projection, limit)
let csv_source = CsvSource::new(format.has_header, format.delimiter, b'"')
.with_schema(file_schema.clone())
.with_batch_size(DEFAULT_BATCH_SIZE);
build_record_batch_stream(config, file_schema, limit, csv_source)
}
fn new_json_stream(
_ctx: &CreateScanPlanContext,
config: &ScanPlanConfig,
format: &JsonFormat,
) -> Result<SendableRecordBatchStream> {
fn new_json_stream(config: &ScanPlanConfig) -> Result<SendableRecordBatchStream> {
let file_schema = config.file_schema.arrow_schema().clone();
let opener = build_json_opener(file_schema.clone(), config, format)?;
// push down limit only if there is no filter
let limit = config.filters.is_empty().then_some(config.limit).flatten();
build_record_batch_stream(opener, file_schema, config.files, config.projection, limit)
let file_source = JsonSource::new().with_batch_size(DEFAULT_BATCH_SIZE);
build_record_batch_stream(config, file_schema, limit, file_source)
}
fn new_parquet_stream_with_exec_plan(
_ctx: &CreateScanPlanContext,
config: &ScanPlanConfig,
_format: &ParquetFormat,
) -> Result<SendableRecordBatchStream> {
fn new_parquet_stream_with_exec_plan(config: &ScanPlanConfig) -> Result<SendableRecordBatchStream> {
let file_schema = config.file_schema.arrow_schema().clone();
let ScanPlanConfig {
files,
@@ -170,25 +119,20 @@ fn new_parquet_stream_with_exec_plan(
..
} = config;
// construct config for ParquetExec
let scan_config = FileScanConfig {
object_store_url: ObjectStoreUrl::parse("empty://").unwrap(), // won't be used
file_schema: file_schema.clone(),
file_groups: vec![files
let file_group = FileGroup::new(
files
.iter()
.map(|filename| PartitionedFile::new(filename.to_string(), 0))
.collect::<Vec<_>>()],
constraints: Constraints::empty(),
statistics: Statistics::new_unknown(file_schema.as_ref()),
projection: projection.cloned(),
limit: *limit,
table_partition_cols: vec![],
output_ordering: vec![],
};
.collect::<Vec<_>>(),
);
let mut parquet_source = ParquetSource::default().with_parquet_file_reader_factory(Arc::new(
DefaultParquetFileReaderFactory::new(store.clone()),
));
// build predicate filter
let filters = filters.to_vec();
let filters = if let Some(expr) = conjunction(filters) {
if let Some(expr) = conjunction(filters) {
let df_schema = file_schema
.clone()
.to_dfschema_ref()
@@ -196,22 +140,23 @@ fn new_parquet_stream_with_exec_plan(
let filters = create_physical_expr(&expr, &df_schema, &ExecutionProps::new())
.context(error::ParquetScanPlanSnafu)?;
Some(filters)
} else {
None
parquet_source = parquet_source.with_predicate(filters);
};
let file_scan_config = FileScanConfigBuilder::new(
ObjectStoreUrl::local_filesystem(),
file_schema,
Arc::new(parquet_source),
)
.with_file_group(file_group)
.with_projection(projection.cloned())
.with_limit(*limit)
.build();
// TODO(ruihang): get this from upper layer
let task_ctx = SessionContext::default().task_ctx();
let mut builder = ParquetExec::builder(scan_config);
if let Some(filters) = filters {
builder = builder.with_predicate(filters);
}
let parquet_exec = builder
.with_parquet_file_reader_factory(Arc::new(DefaultParquetFileReaderFactory::new(
store.clone(),
)))
.build();
let parquet_exec = DataSourceExec::from_data_source(file_scan_config);
let stream = parquet_exec
.execute(0, task_ctx)
.context(error::ParquetScanPlanSnafu)?;
@@ -221,16 +166,14 @@ fn new_parquet_stream_with_exec_plan(
))
}
fn new_orc_stream(
_ctx: &CreateScanPlanContext,
config: &ScanPlanConfig,
_format: &OrcFormat,
) -> Result<SendableRecordBatchStream> {
fn new_orc_stream(config: &ScanPlanConfig) -> Result<SendableRecordBatchStream> {
let file_schema = config.file_schema.arrow_schema().clone();
let opener = build_orc_opener(file_schema.clone(), config)?;
// push down limit only if there is no filter
let limit = config.filters.is_empty().then_some(config.limit).flatten();
build_record_batch_stream(opener, file_schema, config.files, config.projection, limit)
let file_source = OrcSource::default().with_batch_size(DEFAULT_BATCH_SIZE);
build_record_batch_stream(config, file_schema, limit, file_source)
}
#[derive(Debug, Clone)]
@@ -245,13 +188,12 @@ pub struct ScanPlanConfig<'a> {
pub fn create_stream(
format: &Format,
ctx: &CreateScanPlanContext,
config: &ScanPlanConfig,
) -> Result<SendableRecordBatchStream> {
match format {
Format::Csv(format) => new_csv_stream(ctx, config, format),
Format::Json(format) => new_json_stream(ctx, config, format),
Format::Parquet(format) => new_parquet_stream_with_exec_plan(ctx, config, format),
Format::Orc(format) => new_orc_stream(ctx, config, format),
Format::Csv(format) => new_csv_stream(config, format),
Format::Json(_) => new_json_stream(config),
Format::Parquet(_) => new_parquet_stream_with_exec_plan(config),
Format::Orc(_) => new_orc_stream(config),
}
}

View File

@@ -24,8 +24,7 @@ use common_query::logical_plan::breakup_insert_plan;
use common_telemetry::tracing::warn;
use common_telemetry::{debug, info};
use common_time::Timestamp;
use datafusion::optimizer::analyzer::count_wildcard_rule::CountWildcardRule;
use datafusion::optimizer::AnalyzerRule;
use datafusion::datasource::DefaultTableSource;
use datafusion::sql::unparser::expr_to_sql;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::DFSchemaRef;
@@ -40,6 +39,7 @@ use snafu::{ensure, OptionExt, ResultExt};
use sql::parser::{ParseOptions, ParserContext};
use sql::statements::statement::Statement;
use substrait::{DFLogicalSubstraitConvertor, SubstraitPlan};
use table::table::adapter::DfTableProviderAdapter;
use tokio::sync::oneshot;
use tokio::sync::oneshot::error::TryRecvError;
use tokio::time::Instant;
@@ -252,7 +252,11 @@ impl BatchingTask {
.await?;
let new_query = self
.gen_query_with_time_window(engine.clone(), &table.meta.schema, max_window_cnt)
.gen_query_with_time_window(
engine.clone(),
&table.table_info().meta.schema,
max_window_cnt,
)
.await?;
let insert_into = if let Some((new_query, _column_cnt)) = new_query {
@@ -274,6 +278,10 @@ impl BatchingTask {
}
);
}
let table_provider = Arc::new(DfTableProviderAdapter::new(table));
let table_source = Arc::new(DefaultTableSource::new(table_provider));
// update_at& time index placeholder (if exists) should have default value
LogicalPlan::Dml(DmlStatement::new(
datafusion_common::TableReference::Full {
@@ -281,7 +289,7 @@ impl BatchingTask {
schema: self.config.sink_table_name[1].clone().into(),
table: self.config.sink_table_name[2].clone().into(),
},
df_schema,
table_source,
WriteOp::Insert(datafusion_expr::dml::InsertOp::Append),
Arc::new(new_query),
))
@@ -324,7 +332,7 @@ impl BatchingTask {
let schema = &self.config.sink_table_name[1];
// fix all table ref by make it fully qualified, i.e. "table_name" => "catalog_name.schema_name.table_name"
let fixed_plan = plan
let plan = plan
.clone()
.transform_down_with_subqueries(|p| {
if let LogicalPlan::TableScan(mut table_scan) = p {
@@ -340,16 +348,6 @@ impl BatchingTask {
})?
.data;
let expanded_plan = CountWildcardRule::new()
.analyze(fixed_plan.clone(), &Default::default())
.with_context(|_| DatafusionSnafu {
context: format!(
"Failed to expand wildcard in logical plan, plan={:?}",
fixed_plan
),
})?;
let plan = expanded_plan;
let mut peer_desc = None;
let res = {

View File

@@ -36,7 +36,7 @@ use snafu::{ensure, OptionExt, ResultExt};
use sql::parser::{ParseOptions, ParserContext};
use sql::statements::statement::Statement;
use sql::statements::tql::Tql;
use table::metadata::TableInfo;
use table::TableRef;
use crate::adapter::AUTO_CREATED_PLACEHOLDER_TS_COL;
use crate::df_optimizer::apply_df_optimizer;
@@ -46,7 +46,7 @@ use crate::{Error, TableName};
pub async fn get_table_info_df_schema(
catalog_mr: CatalogManagerRef,
table_name: TableName,
) -> Result<(Arc<TableInfo>, Arc<DFSchema>), Error> {
) -> Result<(TableRef, Arc<DFSchema>), Error> {
let full_table_name = table_name.clone().join(".");
let table = catalog_mr
.table(&table_name[0], &table_name[1], &table_name[2], None)
@@ -56,7 +56,7 @@ pub async fn get_table_info_df_schema(
.context(TableNotFoundSnafu {
name: &full_table_name,
})?;
let table_info = table.table_info().clone();
let table_info = table.table_info();
let schema = table_info.meta.schema.clone();
@@ -72,7 +72,7 @@ pub async fn get_table_info_df_schema(
),
})?,
);
Ok((table_info, df_schema))
Ok((table, df_schema))
}
/// Convert sql to datafusion logical plan
@@ -426,7 +426,7 @@ impl TreeNodeRewriter for AddFilterRewriter {
return Ok(Transformed::no(node));
}
match node {
LogicalPlan::Filter(mut filter) if !filter.having => {
LogicalPlan::Filter(mut filter) => {
filter.predicate = filter.predicate.and(self.extra_filter.clone());
self.is_rewritten = true;
Ok(Transformed::yes(LogicalPlan::Filter(filter)))

View File

@@ -29,7 +29,6 @@ use datafusion::optimizer::analyzer::type_coercion::TypeCoercion;
use datafusion::optimizer::common_subexpr_eliminate::CommonSubexprEliminate;
use datafusion::optimizer::optimize_projections::OptimizeProjections;
use datafusion::optimizer::simplify_expressions::SimplifyExpressions;
use datafusion::optimizer::unwrap_cast_in_comparison::UnwrapCastInComparison;
use datafusion::optimizer::utils::NamePreserver;
use datafusion::optimizer::{Analyzer, AnalyzerRule, Optimizer, OptimizerContext};
use datafusion_common::tree_node::{
@@ -38,8 +37,8 @@ use datafusion_common::tree_node::{
use datafusion_common::{Column, DFSchema, ScalarValue};
use datafusion_expr::utils::merge_schema;
use datafusion_expr::{
BinaryExpr, ColumnarValue, Expr, Operator, Projection, ScalarFunctionArgs, ScalarUDFImpl,
Signature, TypeSignature, Volatility,
BinaryExpr, ColumnarValue, Expr, Literal, Operator, Projection, ScalarFunctionArgs,
ScalarUDFImpl, Signature, TypeSignature, Volatility,
};
use query::optimizer::count_wildcard::CountWildcardToTimeIndexRule;
use query::parser::QueryLanguageParser;
@@ -80,7 +79,6 @@ pub async fn apply_df_optimizer(
Arc::new(OptimizeProjections::new()),
Arc::new(CommonSubexprEliminate::new()),
Arc::new(SimplifyExpressions::new()),
Arc::new(UnwrapCastInComparison::new()),
]);
let plan = optimizer
.optimize(plan, &ctx, |_, _| {})
@@ -305,11 +303,11 @@ impl TreeNodeRewriter for ExpandAvgRewriter<'_> {
BinaryExpr::new(Box::new(sum_cast), Operator::Divide, Box::new(count_expr));
let div_expr = Box::new(Expr::BinaryExpr(div));
let zero = Box::new(Expr::Literal(ScalarValue::Int64(Some(0))));
let zero = Box::new(0.lit());
let not_zero =
BinaryExpr::new(Box::new(count_expr_ref), Operator::NotEq, zero.clone());
let not_zero = Box::new(Expr::BinaryExpr(not_zero));
let null = Box::new(Expr::Literal(ScalarValue::Null));
let null = Box::new(Expr::Literal(ScalarValue::Null, None));
let case_when =
datafusion_expr::Case::new(None, vec![(not_zero, div_expr)], Some(null));

View File

@@ -88,6 +88,7 @@ use servers::query_handler::sql::SqlQueryHandler;
use session::context::{Channel, QueryContextRef};
use session::table_name::table_idents_to_full_name;
use snafu::prelude::*;
use sql::ast::ObjectNamePartExt;
use sql::dialect::Dialect;
use sql::parser::{ParseOptions, ParserContext};
use sql::statements::copy::{CopyDatabase, CopyTable};
@@ -1010,9 +1011,9 @@ fn validate_database(name: &ObjectName, query_ctx: &QueryContextRef) -> Result<(
let (catalog, schema) = match &name.0[..] {
[schema] => (
query_ctx.current_catalog().to_string(),
schema.value.clone(),
schema.to_string_unquoted(),
),
[catalog, schema] => (catalog.value.clone(), schema.value.clone()),
[catalog, schema] => (catalog.to_string_unquoted(), schema.to_string_unquoted()),
_ => InvalidSqlSnafu {
err_msg: format!(
"expect database name to be <catalog>.<schema> or <schema>, actual: {name}",

View File

@@ -31,17 +31,19 @@ use common_grpc::FlightData;
use common_query::logical_plan::add_insert_to_logical_plan;
use common_query::Output;
use common_telemetry::tracing::{self};
use datafusion::datasource::DefaultTableSource;
use query::parser::PromQuery;
use servers::interceptor::{GrpcQueryInterceptor, GrpcQueryInterceptorRef};
use servers::query_handler::grpc::GrpcQueryHandler;
use servers::query_handler::sql::SqlQueryHandler;
use session::context::QueryContextRef;
use snafu::{ensure, OptionExt, ResultExt};
use table::table::adapter::DfTableProviderAdapter;
use table::table_name::TableName;
use table::TableRef;
use crate::error::{
CatalogSnafu, DataFusionSnafu, Error, ExternalSnafu, InFlightWriteBytesExceededSnafu,
CatalogSnafu, Error, ExternalSnafu, InFlightWriteBytesExceededSnafu,
IncompleteGrpcRequestSnafu, NotSupportedSnafu, PermissionSnafu, PlanStatementSnafu, Result,
SubstraitDecodeLogicalPlanSnafu, TableNotFoundSnafu, TableOperationSnafu,
};
@@ -372,20 +374,10 @@ impl Instance {
]
.join("."),
})?;
let table_provider = Arc::new(DfTableProviderAdapter::new(table));
let table_source = Arc::new(DefaultTableSource::new(table_provider));
let table_info = table.table_info();
let df_schema = Arc::new(
table_info
.meta
.schema
.arrow_schema()
.clone()
.try_into()
.context(DataFusionSnafu)?,
);
let insert_into = add_insert_to_logical_plan(table_name, df_schema, logical_plan)
let insert_into = add_insert_to_logical_plan(table_name, table_source, logical_plan)
.context(SubstraitDecodeLogicalPlanSnafu)?;
let engine_ctx = self.query_engine().engine_context(ctx.clone());

View File

@@ -30,6 +30,7 @@ use common_recordbatch::util;
use datafusion::dataframe::DataFrame;
use datafusion::execution::context::SessionContext;
use datafusion::execution::SessionStateBuilder;
use datafusion_expr::select_expr::SelectExpr;
use datafusion_expr::{col, lit, lit_timestamp_nano, wildcard, Expr, SortExpr};
use datatypes::value::ValueRef;
use query::QueryEngineRef;
@@ -61,7 +62,7 @@ impl JaegerQueryHandler for Instance {
ctx,
self.catalog_manager(),
self.query_engine(),
vec![col(SERVICE_NAME_COLUMN)],
vec![SelectExpr::from(col(SERVICE_NAME_COLUMN))],
vec![],
vec![],
None,
@@ -118,10 +119,10 @@ impl JaegerQueryHandler for Instance {
self.catalog_manager(),
self.query_engine(),
vec![
col(SPAN_NAME_COLUMN),
col(SPAN_KIND_COLUMN),
col(SERVICE_NAME_COLUMN),
col(TIMESTAMP_COLUMN),
SelectExpr::from(col(SPAN_NAME_COLUMN)),
SelectExpr::from(col(SPAN_KIND_COLUMN)),
SelectExpr::from(col(SERVICE_NAME_COLUMN)),
SelectExpr::from(col(TIMESTAMP_COLUMN)),
],
filters,
vec![col(SPAN_NAME_COLUMN).sort(true, false)], // Sort by span_name in ascending order.
@@ -287,7 +288,7 @@ async fn query_trace_table(
ctx: QueryContextRef,
catalog_manager: &CatalogManagerRef,
query_engine: &QueryEngineRef,
selects: Vec<Expr>,
selects: Vec<SelectExpr>,
filters: Vec<Expr>,
sorts: Vec<SortExpr>,
limit: Option<usize>,
@@ -300,7 +301,10 @@ async fn query_trace_table(
// If only select services, use the trace services table.
let table_name = {
if selects.len() == 1 && selects[0] == col(SERVICE_NAME_COLUMN) {
if match selects.as_slice() {
[SelectExpr::Expression(x)] => x == &col(SERVICE_NAME_COLUMN),
_ => false,
} {
&trace_services_table_name(trace_table_name)
} else {
trace_table_name

View File

@@ -21,8 +21,8 @@ use common_error::ext::BoxedError;
use common_query::prelude::GREPTIME_PHYSICAL_TABLE;
use common_telemetry::tracing;
use opentelemetry_proto::tonic::collector::logs::v1::ExportLogsServiceRequest;
use opentelemetry_proto::tonic::collector::metrics::v1::ExportMetricsServiceRequest;
use opentelemetry_proto::tonic::collector::trace::v1::ExportTraceServiceRequest;
use otel_arrow_rust::proto::opentelemetry::collector::metrics::v1::ExportMetricsServiceRequest;
use pipeline::{GreptimePipelineParams, PipelineWay};
use servers::error::{self, AuthSnafu, InFlightWriteBytesExceededSnafu, Result as ServerResult};
use servers::http::prom_store::PHYSICAL_TABLE_PARAM;

View File

@@ -239,8 +239,7 @@ pub async fn bootstrap_metasrv_with_router(
info!("gRPC server is bound to: {}", real_bind_addr);
let incoming =
TcpIncoming::from_listener(listener, true, None).context(error::TcpIncomingSnafu)?;
let incoming = TcpIncoming::from(listener).with_nodelay(Some(true));
let _handle = common_runtime::spawn_global(async move {
let result = router

View File

@@ -253,12 +253,6 @@ pub enum Error {
location: Location,
},
#[snafu(display("Failed to convert to TcpIncoming"))]
TcpIncoming {
#[snafu(source)]
error: Box<dyn std::error::Error + Send + Sync>,
},
#[snafu(display("Failed to start gRPC server"))]
StartGrpc {
#[snafu(source)]
@@ -964,7 +958,6 @@ impl ErrorExt for Error {
Error::EtcdFailed { .. }
| Error::ConnectEtcd { .. }
| Error::TcpBind { .. }
| Error::TcpIncoming { .. }
| Error::SerializeToJson { .. }
| Error::DeserializeFromJson { .. }
| Error::NoLeader { .. }

View File

@@ -30,10 +30,8 @@ use std::task::{Context, Poll};
use axum::http::StatusCode;
use axum::response::IntoResponse;
use axum::{routing, Router as AxumRouter};
use bytes::Bytes;
use http_body_util::{BodyExt, Full};
use tonic::body::BoxBody;
use tonic::codegen::{empty_body, http, BoxFuture, Service};
use tonic::body::Body;
use tonic::codegen::{http, BoxFuture, Service};
use tonic::server::NamedService;
use crate::metasrv::Metasrv;
@@ -129,8 +127,8 @@ impl NamedService for Admin {
const NAME: &'static str = "admin";
}
impl Service<http::Request<BoxBody>> for Admin {
type Response = http::Response<BoxBody>;
impl Service<http::Request<Body>> for Admin {
type Response = http::Response<Body>;
type Error = Infallible;
type Future = BoxFuture<Self::Response, Self::Error>;
@@ -138,7 +136,7 @@ impl Service<http::Request<BoxBody>> for Admin {
Poll::Ready(Ok(()))
}
fn call(&mut self, req: http::Request<BoxBody>) -> Self::Future {
fn call(&mut self, req: http::Request<Body>) -> Self::Future {
let router = self.router.clone();
let query_params = req
.uri()
@@ -202,22 +200,22 @@ impl Router {
path: &str,
method: http::Method,
params: HashMap<String, String>,
) -> Result<http::Response<BoxBody>, Infallible> {
) -> Result<http::Response<Body>, Infallible> {
let handler = match self.handlers.get(path) {
Some(handler) => handler,
None => {
return Ok(http::Response::builder()
.status(http::StatusCode::NOT_FOUND)
.body(empty_body())
.body(Body::empty())
.unwrap())
}
};
let res = match handler.handle(path, method, &params).await {
Ok(res) => res.map(boxed),
Ok(res) => res.map(Body::new),
Err(e) => http::Response::builder()
.status(http::StatusCode::INTERNAL_SERVER_ERROR)
.body(boxed(e.to_string()))
.body(Body::new(e.to_string()))
.unwrap(),
};
@@ -231,14 +229,6 @@ fn check_path(path: &str) {
}
}
/// Returns a [BoxBody] from a string.
/// The implementation follows [empty_body()].
fn boxed(body: String) -> BoxBody {
Full::new(Bytes::from(body))
.map_err(|err| match err {})
.boxed_unsync()
}
/// Expose admin HTTP endpoints as an Axum router for the main HTTP server.
pub fn admin_axum_router(metasrv: Arc<Metasrv>) -> AxumRouter {
let node_lease_handler = Arc::new(NodeLeaseHandler {

View File

@@ -154,8 +154,8 @@ mod tests {
use std::sync::Arc;
use api::v1::SemanticType;
use datafusion_common::{Column, ScalarValue};
use datafusion_expr::{BinaryExpr, Expr, Operator};
use datafusion_common::Column;
use datafusion_expr::{BinaryExpr, Expr, Literal, Operator};
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use datatypes::value::ValueRef;
@@ -226,7 +226,7 @@ mod tests {
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name(column_name))),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Utf8(Some(value.to_string())))),
right: Box::new(value.lit()),
});
SimpleFilterEvaluator::try_new(&expr).unwrap()
}

View File

@@ -383,8 +383,8 @@ mod tests {
use api::v1::value::ValueData;
use api::v1::{Mutation, OpType, Rows, SemanticType};
use common_time::Timestamp;
use datafusion_common::{Column, ScalarValue};
use datafusion_expr::{BinaryExpr, Expr, Operator};
use datafusion_common::Column;
use datafusion_expr::{BinaryExpr, Expr, Literal, Operator};
use datatypes::data_type::ConcreteDataType;
use datatypes::prelude::Vector;
use datatypes::scalars::ScalarVector;
@@ -631,7 +631,7 @@ mod tests {
let expr = Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("k1"))),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::UInt32(Some(i)))),
right: Box::new((i as u32).lit()),
});
let iter = memtable
.iter(None, Some(Predicate::new(vec![expr])), None)

View File

@@ -465,7 +465,10 @@ impl DisplayAs for SeqScan {
self.stream_ctx.input.mapper.metadata().region_id
)?;
match t {
DisplayFormatType::Default => self.stream_ctx.format_for_explain(false, f),
// TODO(LFC): Implement all the "TreeRender" display format.
DisplayFormatType::Default | DisplayFormatType::TreeRender => {
self.stream_ctx.format_for_explain(false, f)
}
DisplayFormatType::Verbose => {
self.stream_ctx.format_for_explain(true, f)?;
self.metrics_list.format_verbose_metrics(f)

View File

@@ -328,7 +328,9 @@ impl DisplayAs for SeriesScan {
self.stream_ctx.input.mapper.metadata().region_id
)?;
match t {
DisplayFormatType::Default => self.stream_ctx.format_for_explain(false, f),
DisplayFormatType::Default | DisplayFormatType::TreeRender => {
self.stream_ctx.format_for_explain(false, f)
}
DisplayFormatType::Verbose => {
self.stream_ctx.format_for_explain(true, f)?;
self.metrics_list.format_verbose_metrics(f)

View File

@@ -331,7 +331,9 @@ impl DisplayAs for UnorderedScan {
self.stream_ctx.input.mapper.metadata().region_id
)?;
match t {
DisplayFormatType::Default => self.stream_ctx.format_for_explain(false, f),
DisplayFormatType::Default | DisplayFormatType::TreeRender => {
self.stream_ctx.format_for_explain(false, f)
}
DisplayFormatType::Verbose => {
self.stream_ctx.format_for_explain(true, f)?;
self.metrics_list.format_verbose_metrics(f)

View File

@@ -301,7 +301,7 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
/// Helper function to get non-null literal value
fn nonnull_lit(expr: &Expr) -> Option<&ScalarValue> {
match expr {
Expr::Literal(lit) if !lit.is_null() => Some(lit),
Expr::Literal(lit, _) if !lit.is_null() => Some(lit),
_ => None,
}
}
@@ -312,8 +312,8 @@ impl<'a> BloomFilterIndexApplierBuilder<'a> {
right: &'b Expr,
) -> Result<Option<(&'b Column, &'b ScalarValue)>> {
let (col, lit) = match (left, right) {
(Expr::Column(col), Expr::Literal(lit)) => (col, lit),
(Expr::Literal(lit), Expr::Column(col)) => (col, lit),
(Expr::Column(col), Expr::Literal(lit, _)) => (col, lit),
(Expr::Literal(lit, _), Expr::Column(col)) => (col, lit),
_ => return Ok(None),
};
Ok(Some((col, lit)))
@@ -335,7 +335,7 @@ fn encode_lit(lit: &ScalarValue, data_type: ConcreteDataType) -> Result<Bytes> {
mod tests {
use api::v1::SemanticType;
use datafusion_common::Column;
use datafusion_expr::{col, lit};
use datafusion_expr::{col, lit, Literal};
use datatypes::schema::ColumnSchema;
use object_store::services::Memory;
use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder};
@@ -385,10 +385,6 @@ mod tests {
Expr::Column(Column::from_name(name))
}
fn string_lit(s: impl Into<String>) -> Expr {
Expr::Literal(ScalarValue::Utf8(Some(s.into())))
}
#[test]
fn test_build_with_exprs() {
let (_d, factory) = PuffinManagerFactory::new_for_test_block("test_build_with_exprs_");
@@ -403,7 +399,7 @@ mod tests {
let exprs = vec![Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
op: Operator::Eq,
right: Box::new(string_lit("value1")),
right: Box::new("value1".lit()),
})];
let result = builder.build(&exprs).unwrap();
assert!(result.is_some());
@@ -423,7 +419,7 @@ mod tests {
}
fn int64_lit(i: i64) -> Expr {
Expr::Literal(ScalarValue::Int64(Some(i)))
i.lit()
}
#[test]
@@ -495,7 +491,7 @@ mod tests {
assert!(or_chain_predicates.contains(&encode_str("value4")));
// Test with null value
let expr = col("column1").eq(Expr::Literal(ScalarValue::Utf8(None)));
let expr = col("column1").eq(Expr::Literal(ScalarValue::Utf8(None), None));
let result = builder().build(&[expr]).unwrap();
assert!(result.is_none());
@@ -529,7 +525,7 @@ mod tests {
left: Box::new(Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
op: Operator::Eq,
right: Box::new(string_lit("value1")),
right: Box::new("value1".lit()),
})),
op: Operator::And,
right: Box::new(Expr::BinaryExpr(BinaryExpr {
@@ -563,13 +559,13 @@ mod tests {
Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Utf8(None))),
right: Box::new(Expr::Literal(ScalarValue::Utf8(None), None)),
}),
Expr::InList(InList {
expr: Box::new(column("column2")),
list: vec![
int64_lit(1),
Expr::Literal(ScalarValue::Int64(None)),
Expr::Literal(ScalarValue::Int64(None), None),
int64_lit(3),
],
negated: false,
@@ -601,13 +597,13 @@ mod tests {
Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
op: Operator::Gt,
right: Box::new(string_lit("value1")),
right: Box::new("value1".lit()),
}),
// Non-existent column
Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("non_existent")),
op: Operator::Eq,
right: Box::new(string_lit("value")),
right: Box::new("value".lit()),
}),
// Negated IN list
Expr::InList(InList {
@@ -636,11 +632,11 @@ mod tests {
Expr::BinaryExpr(BinaryExpr {
left: Box::new(column("column1")),
op: Operator::Eq,
right: Box::new(string_lit("value1")),
right: Box::new("value1".lit()),
}),
Expr::InList(InList {
expr: Box::new(column("column1")),
list: vec![string_lit("value2"), string_lit("value3")],
list: vec!["value2".lit(), "value3".lit()],
negated: false,
}),
];

View File

@@ -207,7 +207,7 @@ impl<'a> FulltextIndexApplierBuilder<'a> {
return None;
}
let Expr::Literal(ScalarValue::Utf8(Some(query))) = &f.args[1] else {
let Expr::Literal(ScalarValue::Utf8(Some(query)), _) = &f.args[1] else {
return None;
};
@@ -248,7 +248,7 @@ impl<'a> FulltextIndexApplierBuilder<'a> {
return None;
}
let Expr::Literal(ScalarValue::Utf8(Some(term))) = &f.args[1] else {
let Expr::Literal(ScalarValue::Utf8(Some(term)), _) = &f.args[1] else {
return None;
};
@@ -290,7 +290,7 @@ mod tests {
use datafusion::functions::string::lower;
use datafusion_common::Column;
use datafusion_expr::expr::ScalarFunction;
use datafusion_expr::ScalarUDF;
use datafusion_expr::{Literal, ScalarUDF};
use datatypes::schema::ColumnSchema;
use store_api::metadata::{ColumnMetadata, RegionMetadataBuilder};
use store_api::storage::RegionId;
@@ -337,10 +337,7 @@ mod tests {
let metadata = mock_metadata();
let func = ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("text")), "foo".lit()],
func: matches_func(),
};
@@ -367,10 +364,7 @@ mod tests {
let metadata = mock_metadata();
let func = ScalarFunction {
args: vec![
Expr::Column(Column::from_name("not_found")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("not_found")), "foo".lit()],
func: matches_func(),
};
@@ -382,10 +376,7 @@ mod tests {
let metadata = mock_metadata();
let func = ScalarFunction {
args: vec![
Expr::Column(Column::from_name("ts")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("ts")), "foo".lit()],
func: matches_func(),
};
@@ -397,10 +388,7 @@ mod tests {
let metadata = mock_metadata();
let func = ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Int64(Some(42))),
],
args: vec![Expr::Column(Column::from_name("text")), 42.lit()],
func: matches_func(),
};
@@ -412,10 +400,7 @@ mod tests {
let metadata = mock_metadata();
let func = ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("text")), "foo".lit()],
func: matches_term_func(),
};
@@ -441,10 +426,7 @@ mod tests {
};
let func = ScalarFunction {
args: vec![
Expr::ScalarFunction(lower_func_expr),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::ScalarFunction(lower_func_expr), "foo".lit()],
func: matches_term_func(),
};
@@ -477,10 +459,7 @@ mod tests {
let metadata = mock_metadata();
let func = ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("text")), "foo".lit()],
func: matches_func(), // Using 'matches' instead of 'matches_term'
};
@@ -519,10 +498,7 @@ mod tests {
// Create a matches expression
let matches_expr = Expr::ScalarFunction(ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("text")), "foo".lit()],
func: matches_func(),
});
@@ -542,19 +518,13 @@ mod tests {
// Create a matches expression
let matches_expr = Expr::ScalarFunction(ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("foo".to_string()))),
],
args: vec![Expr::Column(Column::from_name("text")), "foo".lit()],
func: matches_func(),
});
// Create a matches_term expression
let matches_term_expr = Expr::ScalarFunction(ScalarFunction {
args: vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("bar".to_string()))),
],
args: vec![Expr::Column(Column::from_name("text")), "bar".lit()],
func: matches_term_func(),
});

View File

@@ -220,7 +220,7 @@ impl<'a> InvertedIndexApplierBuilder<'a> {
/// Helper function to get a non-null literal.
fn nonnull_lit(expr: &Expr) -> Option<&ScalarValue> {
match expr {
Expr::Literal(lit) if !lit.is_null() => Some(lit),
Expr::Literal(lit, _) if !lit.is_null() => Some(lit),
_ => None,
}
}
@@ -248,7 +248,7 @@ impl<'a> InvertedIndexApplierBuilder<'a> {
mod tests {
use api::v1::SemanticType;
use datafusion_common::Column;
use datafusion_expr::Between;
use datafusion_expr::{Between, Literal};
use datatypes::data_type::ConcreteDataType;
use datatypes::schema::ColumnSchema;
use index::inverted_index::search::predicate::{
@@ -313,11 +313,11 @@ mod tests {
}
pub(crate) fn string_lit(s: impl Into<String>) -> Expr {
Expr::Literal(ScalarValue::Utf8(Some(s.into())))
s.into().lit()
}
pub(crate) fn int64_lit(i: impl Into<i64>) -> Expr {
Expr::Literal(ScalarValue::Int64(Some(i.into())))
i.into().lit()
}
pub(crate) fn encoded_string(s: impl Into<String>) -> Vec<u8> {

View File

@@ -31,7 +31,7 @@ impl InvertedIndexApplierBuilder<'_> {
if !data_type.is_string() {
return Ok(());
}
let DfExpr::Literal(ScalarValue::Utf8(Some(pattern))) = pattern else {
let DfExpr::Literal(ScalarValue::Utf8(Some(pattern)), _) = pattern else {
return Ok(());
};

View File

@@ -93,7 +93,7 @@ mod tests {
use common_time::Timestamp;
use datafusion_common::{Column, ScalarValue};
use datafusion_expr::{col, lit, BinaryExpr, Expr, Operator};
use datafusion_expr::{col, lit, BinaryExpr, Expr, Literal, Operator};
use datatypes::arrow;
use datatypes::arrow::array::{RecordBatch, UInt64Array};
use datatypes::arrow::datatypes::{DataType, Field, Schema};
@@ -383,7 +383,7 @@ mod tests {
let predicate = Some(Predicate::new(vec![Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("tag_0"))),
op: Operator::Eq,
right: Box::new(Expr::Literal(ScalarValue::Utf8(Some("a".to_string())))),
right: Box::new("a".lit()),
})]));
let builder = ParquetReaderBuilder::new(
@@ -485,7 +485,7 @@ mod tests {
let predicate = Some(Predicate::new(vec![Expr::BinaryExpr(BinaryExpr {
left: Box::new(Expr::Column(Column::from_name("field_0"))),
op: Operator::GtEq,
right: Box::new(Expr::Literal(ScalarValue::UInt64(Some(150)))),
right: Box::new(150u64.lit()),
})]));
let builder = ParquetReaderBuilder::new(
@@ -603,6 +603,7 @@ mod tests {
let batches = &[
new_batch_by_range(&["a", "d"], 0, 1000),
new_batch_by_range(&["b", "f"], 0, 1000),
new_batch_by_range(&["c", "g"], 0, 1000),
new_batch_by_range(&["b", "h"], 100, 200),
new_batch_by_range(&["b", "h"], 200, 300),
new_batch_by_range(&["b", "h"], 300, 1000),

View File

@@ -95,13 +95,14 @@ impl<'a> MetadataLoader<'a> {
let mut footer = [0; 8];
footer.copy_from_slice(&buffer[buffer_len - FOOTER_SIZE..]);
let metadata_len = ParquetMetaDataReader::decode_footer(&footer).map_err(|e| {
let footer_tail = ParquetMetaDataReader::decode_footer_tail(&footer).map_err(|e| {
error::InvalidParquetSnafu {
file: path,
reason: format!("failed to decode footer, {e}"),
}
.build()
})? as u64;
})?;
let metadata_len = footer_tail.metadata_length() as u64;
if file_size - (FOOTER_SIZE as u64) < metadata_len {
return error::InvalidParquetSnafu {

View File

@@ -103,7 +103,7 @@ impl<'a> RowGroupBase<'a> {
selection
.scan_ranges(&offset_index[idx].page_locations)
.iter()
.map(|range| range.start as u64..range.end as u64),
.map(|range| range.start..range.end),
);
page_start_offsets.push(ranges.iter().map(|range| range.start as usize).collect());

View File

@@ -18,7 +18,7 @@ use std::borrow::Borrow;
use std::collections::HashSet;
use std::sync::Arc;
use datafusion::physical_optimizer::pruning::PruningStatistics;
use datafusion_common::pruning::PruningStatistics;
use datafusion_common::{Column, ScalarValue};
use datatypes::arrow::array::{ArrayRef, BooleanArray, UInt64Array};
use parquet::file::metadata::RowGroupMetaData;

View File

@@ -21,7 +21,7 @@ humantime-serde.workspace = true
lazy_static.workspace = true
md5 = "0.7"
moka = { workspace = true, features = ["future"] }
opendal = { version = "0.52", features = [
opendal = { git = "https://github.com/apache/opendal", rev = "0ba8574b6d08d209056704d28a9a114beb3c1022", features = [
"layers-tracing",
"layers-prometheus",
"services-azblob",

View File

@@ -81,16 +81,17 @@ pub async fn new_azblob_object_store(azblob_config: &AzblobConfig) -> Result<Obj
.container(&azblob_config.container)
.endpoint(&azblob_config.endpoint)
.account_name(azblob_config.account_name.expose_secret())
.account_key(azblob_config.account_key.expose_secret())
.http_client(client);
.account_key(azblob_config.account_key.expose_secret());
if let Some(token) = &azblob_config.sas_token {
builder = builder.sas_token(token);
};
Ok(ObjectStore::new(builder)
let operator = ObjectStore::new(builder)
.context(error::InitBackendSnafu)?
.finish())
.finish();
operator.update_http_client(|_| client);
Ok(operator)
}
pub async fn new_gcs_object_store(gcs_config: &GcsConfig) -> Result<ObjectStore> {
@@ -100,7 +101,7 @@ pub async fn new_gcs_object_store(gcs_config: &GcsConfig) -> Result<ObjectStore>
gcs_config.bucket, &root
);
let client = build_http_client(&gcs_config.http_client);
let client = build_http_client(&gcs_config.http_client)?;
let builder = Gcs::default()
.root(&root)
@@ -108,12 +109,13 @@ pub async fn new_gcs_object_store(gcs_config: &GcsConfig) -> Result<ObjectStore>
.scope(&gcs_config.scope)
.credential_path(gcs_config.credential_path.expose_secret())
.credential(gcs_config.credential.expose_secret())
.endpoint(&gcs_config.endpoint)
.http_client(client?);
.endpoint(&gcs_config.endpoint);
Ok(ObjectStore::new(builder)
let operator = ObjectStore::new(builder)
.context(error::InitBackendSnafu)?
.finish())
.finish();
operator.update_http_client(|_| client);
Ok(operator)
}
pub async fn new_oss_object_store(oss_config: &OssConfig) -> Result<ObjectStore> {
@@ -130,12 +132,13 @@ pub async fn new_oss_object_store(oss_config: &OssConfig) -> Result<ObjectStore>
.bucket(&oss_config.bucket)
.endpoint(&oss_config.endpoint)
.access_key_id(oss_config.access_key_id.expose_secret())
.access_key_secret(oss_config.access_key_secret.expose_secret())
.http_client(client);
.access_key_secret(oss_config.access_key_secret.expose_secret());
Ok(ObjectStore::new(builder)
let operator = ObjectStore::new(builder)
.context(error::InitBackendSnafu)?
.finish())
.finish();
operator.update_http_client(|_| client);
Ok(operator)
}
pub async fn new_s3_object_store(s3_config: &S3Config) -> Result<ObjectStore> {
@@ -152,8 +155,7 @@ pub async fn new_s3_object_store(s3_config: &S3Config) -> Result<ObjectStore> {
.root(&root)
.bucket(&s3_config.bucket)
.access_key_id(s3_config.access_key_id.expose_secret())
.secret_access_key(s3_config.secret_access_key.expose_secret())
.http_client(client);
.secret_access_key(s3_config.secret_access_key.expose_secret());
if s3_config.endpoint.is_some() {
builder = builder.endpoint(s3_config.endpoint.as_ref().unwrap());
@@ -165,7 +167,9 @@ pub async fn new_s3_object_store(s3_config: &S3Config) -> Result<ObjectStore> {
builder = builder.enable_virtual_host_style();
}
Ok(ObjectStore::new(builder)
let operator = ObjectStore::new(builder)
.context(error::InitBackendSnafu)?
.finish())
.finish();
operator.update_http_client(|_| client);
Ok(operator)
}

View File

@@ -40,11 +40,7 @@ mod prometheus {
// remove path label first, waiting for later fix
// TODO(shuiyisong): add dynamic path label trim for opendal
let layer = PrometheusLayer::builder()
.path_label(0)
.register_default()
.unwrap();
let layer = PrometheusLayer::builder().register_default().unwrap();
Mutex::new(layer)
})
.lock()

View File

@@ -16,7 +16,7 @@ use std::fmt::Display;
use std::path;
use std::time::Duration;
use common_telemetry::{debug, error, info, trace, warn};
use common_telemetry::{debug, error, info, warn};
use opendal::layers::{LoggingInterceptor, LoggingLayer, RetryInterceptor, TracingLayer};
use opendal::raw::{AccessorInfo, HttpClient, Operation};
use opendal::{Error, ErrorKind};
@@ -183,24 +183,13 @@ impl LoggingInterceptor for DefaultLoggingInterceptor {
};
}
// Print debug message if operation is oneshot, otherwise in trace.
if operation.is_oneshot() {
debug!(
target: LOGGING_TARGET,
"service={} name={} {}: {operation} {message}",
info.scheme(),
info.name(),
LoggingContext(context),
);
} else {
trace!(
target: LOGGING_TARGET,
"service={} name={} {}: {operation} {message}",
info.scheme(),
info.name(),
LoggingContext(context),
);
};
debug!(
target: LOGGING_TARGET,
"service={} name={} {}: {operation} {message}",
info.scheme(),
info.name(),
LoggingContext(context),
);
}
}

View File

@@ -45,7 +45,7 @@ use query::sql::{
use session::context::QueryContextRef;
use session::table_name::table_idents_to_full_name;
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::{ColumnOption, ObjectName};
use sql::ast::{ColumnOption, ObjectName, ObjectNamePartExt};
use sql::statements::alter::{
AlterDatabase, AlterDatabaseOperation, AlterTable, AlterTableOperation,
};
@@ -819,7 +819,7 @@ fn sanitize_flow_name(mut flow_name: ObjectName) -> Result<String> {
}
);
// safety: we've checked flow_name.0 has exactly one element.
Ok(flow_name.0.swap_remove(0).value)
Ok(flow_name.0.swap_remove(0).to_string_unquoted())
}
#[cfg(test)]

View File

@@ -5,7 +5,7 @@ use api::v1::{
};
use session::context::QueryContextRef;
use snafu::ensure;
use sql::ast::ObjectName;
use sql::ast::{ObjectName, ObjectNamePartExt};
use sql::statements::create::trigger::{ChannelType, CreateTrigger};
use crate::error::Result;
@@ -67,7 +67,7 @@ fn sanitize_trigger_name(mut trigger_name: ObjectName) -> Result<String> {
}
);
// safety: we've checked trigger_name.0 has exactly one element.
Ok(trigger_name.0.swap_remove(0).value)
Ok(trigger_name.0.swap_remove(0).to_string_unquoted())
}
#[cfg(test)]
@@ -81,15 +81,15 @@ mod tests {
#[test]
fn test_sanitize_trigger_name() {
let name = ObjectName(vec![sql::ast::Ident::new("my_trigger")]);
let name = vec![sql::ast::Ident::new("my_trigger")].into();
let sanitized = sanitize_trigger_name(name).unwrap();
assert_eq!(sanitized, "my_trigger");
let name = ObjectName(vec![sql::ast::Ident::with_quote('`', "my_trigger")]);
let name = vec![sql::ast::Ident::with_quote('`', "my_trigger")].into();
let sanitized = sanitize_trigger_name(name).unwrap();
assert_eq!(sanitized, "my_trigger");
let name = ObjectName(vec![sql::ast::Ident::with_quote('\'', "trigger")]);
let name = vec![sql::ast::Ident::with_quote('\'', "trigger")].into();
let sanitized = sanitize_trigger_name(name).unwrap();
assert_eq!(sanitized, "trigger");
}

View File

@@ -22,6 +22,7 @@ use datatypes::schema::{ColumnSchema, SchemaRef};
use partition::manager::PartitionRuleManager;
use session::context::{QueryContext, QueryContextRef};
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::ObjectNamePartExt;
use sql::statements::insert::Insert;
use sqlparser::ast::{ObjectName, Value as SqlValue};
use table::metadata::TableInfoRef;
@@ -172,17 +173,17 @@ impl<'a> StatementToRegion<'a> {
[table] => Ok((
self.ctx.current_catalog().to_owned(),
self.ctx.current_schema(),
table.value.clone(),
table.to_string_unquoted(),
)),
[schema, table] => Ok((
self.ctx.current_catalog().to_owned(),
schema.value.clone(),
table.value.clone(),
schema.to_string_unquoted(),
table.to_string_unquoted(),
)),
[catalog, schema, table] => Ok((
catalog.value.clone(),
schema.value.clone(),
table.value.clone(),
catalog.to_string_unquoted(),
schema.to_string_unquoted(),
table.to_string_unquoted(),
)),
_ => InvalidSqlSnafu {
err_msg: format!(

View File

@@ -55,6 +55,7 @@ use session::context::{Channel, QueryContextRef};
use session::table_name::table_idents_to_full_name;
use set::{set_query_timeout, set_read_preference};
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::ObjectNamePartExt;
use sql::statements::copy::{
CopyDatabase, CopyDatabaseArgument, CopyQueryToArgument, CopyTable, CopyTableArgument,
};
@@ -736,9 +737,9 @@ fn idents_to_full_database_name(
match &obj_name.0[..] {
[database] => Ok((
query_ctx.current_catalog().to_owned(),
database.value.clone(),
database.to_string_unquoted(),
)),
[catalog, database] => Ok((catalog.value.clone(), database.value.clone())),
[catalog, database] => Ok((catalog.to_string_unquoted(), database.to_string_unquoted())),
_ => InvalidSqlSnafu {
err_msg: format!(
"expect database name to be <catalog>.<database>, <database>, found: {obj_name}",

View File

@@ -69,7 +69,7 @@ impl StatementExecutor {
}
.fail();
};
Ok(value)
Ok(&value.value)
})
.collect::<Result<Vec<_>>>()?;

View File

@@ -20,7 +20,6 @@ use std::sync::Arc;
use client::{Output, OutputData, OutputMeta};
use common_base::readable_size::ReadableSize;
use common_datasource::file_format::csv::CsvFormat;
use common_datasource::file_format::json::JsonFormat;
use common_datasource::file_format::orc::{infer_orc_schema, new_orc_stream_reader, ReaderAdapter};
use common_datasource::file_format::{FileFormat, Format};
use common_datasource::lister::{Lister, Source};
@@ -33,12 +32,11 @@ use common_telemetry::{debug, tracing};
use datafusion::datasource::listing::PartitionedFile;
use datafusion::datasource::object_store::ObjectStoreUrl;
use datafusion::datasource::physical_plan::{
CsvConfig, CsvOpener, FileOpener, FileScanConfig, FileStream, JsonOpener,
CsvSource, FileGroup, FileScanConfigBuilder, FileSource, FileStream, JsonSource,
};
use datafusion::parquet::arrow::arrow_reader::ArrowReaderMetadata;
use datafusion::parquet::arrow::ParquetRecordBatchStreamBuilder;
use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion_common::{Constraints, Statistics};
use datafusion_expr::Expr;
use datatypes::arrow::compute::can_cast_types;
use datatypes::arrow::datatypes::{DataType as ArrowDataType, Schema, SchemaRef};
@@ -69,7 +67,6 @@ enum FileMetadata {
},
Json {
schema: SchemaRef,
format: JsonFormat,
path: String,
},
Csv {
@@ -150,7 +147,6 @@ impl StatementExecutor {
.await
.context(error::InferSchemaSnafu { path: &path })?,
),
format,
path,
}),
Format::Parquet(_) => {
@@ -199,30 +195,29 @@ impl StatementExecutor {
}
}
async fn build_file_stream<F: FileOpener + Send + 'static>(
async fn build_file_stream(
&self,
opener: F,
store: &ObjectStore,
filename: &str,
file_schema: SchemaRef,
file_source: Arc<dyn FileSource>,
projection: Option<Vec<usize>>,
) -> Result<DfSendableRecordBatchStream> {
let statistics = Statistics::new_unknown(file_schema.as_ref());
let stream = FileStream::new(
&FileScanConfig {
object_store_url: ObjectStoreUrl::parse("empty://").unwrap(), // won't be used
file_schema,
file_groups: vec![vec![PartitionedFile::new(filename.to_string(), 10)]],
statistics,
projection: None,
limit: None,
table_partition_cols: vec![],
output_ordering: vec![],
constraints: Constraints::empty(),
},
0,
opener,
&ExecutionPlanMetricsSet::new(),
let config = FileScanConfigBuilder::new(
ObjectStoreUrl::local_filesystem(),
file_schema,
file_source.clone(),
)
.context(error::BuildFileStreamSnafu)?;
.with_file_group(FileGroup::new(vec![PartitionedFile::new(filename, 0)]))
.with_projection(projection)
.build();
let store = Arc::new(object_store_opendal::OpendalStore::new(store.clone()));
let file_opener = file_source
.with_projection(&config)
.create_file_opener(store, &config, 0);
let stream = FileStream::new(&config, 0, file_opener, &ExecutionPlanMetricsSet::new())
.context(error::BuildFileStreamSnafu)?;
Ok(Box::pin(stream))
}
@@ -246,29 +241,18 @@ impl StatementExecutor {
.project(&projection)
.context(error::ProjectSchemaSnafu)?,
);
let csv_config = Arc::new(CsvConfig::new(
DEFAULT_BATCH_SIZE,
schema.clone(),
Some(projection.clone()),
format.has_header,
format.delimiter,
b'"',
None,
Arc::new(object_store_opendal::OpendalStore::new(
object_store.clone(),
)),
None,
));
let projected_file_schema = Arc::new(
schema
.project(&projection)
.context(error::ProjectSchemaSnafu)?,
);
let csv_source = CsvSource::new(format.has_header, format.delimiter, b'"')
.with_schema(schema.clone())
.with_batch_size(DEFAULT_BATCH_SIZE);
let stream = self
.build_file_stream(
CsvOpener::new(csv_config, format.compression_type.into()),
object_store,
path,
projected_file_schema,
schema.clone(),
csv_source,
Some(projection),
)
.await?;
@@ -280,32 +264,24 @@ impl StatementExecutor {
.context(error::PhysicalExprSnafu)?,
))
}
FileMetadata::Json {
format,
path,
schema,
} => {
let projected_file_schema = Arc::new(
schema
.project(&projection)
.context(error::ProjectSchemaSnafu)?,
);
FileMetadata::Json { path, schema } => {
let output_schema = Arc::new(
compat_schema
.project(&projection)
.context(error::ProjectSchemaSnafu)?,
);
let store = object_store_opendal::OpendalStore::new(object_store.clone());
let json_source = JsonSource::new()
.with_schema(schema.clone())
.with_batch_size(DEFAULT_BATCH_SIZE);
let stream = self
.build_file_stream(
JsonOpener::new(
DEFAULT_BATCH_SIZE,
projected_file_schema.clone(),
format.compression_type.into(),
Arc::new(store),
),
object_store,
path,
projected_file_schema,
schema.clone(),
json_source,
Some(projection),
)
.await?;

View File

@@ -1817,27 +1817,27 @@ fn convert_one_expr(
// col, val
(Expr::Identifier(ident), Expr::Value(value)) => {
let (column_name, data_type) = convert_identifier(ident, column_name_and_type)?;
let value = convert_value(value, data_type, timezone, None)?;
let value = convert_value(&value.value, data_type, timezone, None)?;
(Operand::Column(column_name), op, Operand::Value(value))
}
(Expr::Identifier(ident), Expr::UnaryOp { op: unary_op, expr })
if let Expr::Value(v) = &**expr =>
{
let (column_name, data_type) = convert_identifier(ident, column_name_and_type)?;
let value = convert_value(v, data_type, timezone, Some(*unary_op))?;
let value = convert_value(&v.value, data_type, timezone, Some(*unary_op))?;
(Operand::Column(column_name), op, Operand::Value(value))
}
// val, col
(Expr::Value(value), Expr::Identifier(ident)) => {
let (column_name, data_type) = convert_identifier(ident, column_name_and_type)?;
let value = convert_value(value, data_type, timezone, None)?;
let value = convert_value(&value.value, data_type, timezone, None)?;
(Operand::Value(value), op, Operand::Column(column_name))
}
(Expr::UnaryOp { op: unary_op, expr }, Expr::Identifier(ident))
if let Expr::Value(v) = &**expr =>
{
let (column_name, data_type) = convert_identifier(ident, column_name_and_type)?;
let value = convert_value(v, data_type, timezone, Some(*unary_op))?;
let value = convert_value(&v.value, data_type, timezone, Some(*unary_op))?;
(Operand::Value(value), op, Operand::Column(column_name))
}
(Expr::BinaryOp { .. }, Expr::BinaryOp { .. }) => {

View File

@@ -25,6 +25,7 @@ use session::ReadPreference;
use snafu::{ensure, OptionExt, ResultExt};
use sql::ast::{Expr, Ident, Value};
use sql::statements::set_variables::SetVariables;
use sqlparser::ast::ValueWithSpan;
use crate::error::{InvalidConfigValueSnafu, InvalidSqlSnafu, NotSupportedSnafu, Result};
@@ -43,8 +44,14 @@ pub fn set_read_preference(exprs: Vec<Expr>, ctx: QueryContextRef) -> Result<()>
})?;
match read_preference_expr {
Expr::Value(Value::SingleQuotedString(expr))
| Expr::Value(Value::DoubleQuotedString(expr)) => {
Expr::Value(ValueWithSpan {
value: Value::SingleQuotedString(expr),
..
})
| Expr::Value(ValueWithSpan {
value: Value::DoubleQuotedString(expr),
..
}) => {
match ReadPreference::from_str(expr.as_str().to_lowercase().as_str()) {
Ok(read_preference) => ctx.set_read_preference(read_preference),
Err(_) => {
@@ -74,7 +81,14 @@ pub fn set_timezone(exprs: Vec<Expr>, ctx: QueryContextRef) -> Result<()> {
feat: "No timezone find in set variable statement",
})?;
match tz_expr {
Expr::Value(Value::SingleQuotedString(tz)) | Expr::Value(Value::DoubleQuotedString(tz)) => {
Expr::Value(ValueWithSpan {
value: Value::SingleQuotedString(tz),
..
})
| Expr::Value(ValueWithSpan {
value: Value::DoubleQuotedString(tz),
..
}) => {
match Timezone::from_tz_string(tz.as_str()) {
Ok(timezone) => ctx.set_timezone(timezone),
Err(_) => {
@@ -110,7 +124,7 @@ pub fn set_bytea_output(exprs: Vec<Expr>, ctx: QueryContextRef) -> Result<()> {
.fail();
};
ctx.configuration_parameter().set_postgres_bytea_output(
PGByteaOutputValue::try_from(value.clone()).context(InvalidConfigValueSnafu)?,
PGByteaOutputValue::try_from(value.value.clone()).context(InvalidConfigValueSnafu)?,
);
Ok(())
}
@@ -120,8 +134,14 @@ pub fn set_search_path(exprs: Vec<Expr>, ctx: QueryContextRef) -> Result<()> {
feat: "No search path find in set variable statement",
})?;
match search_expr {
Expr::Value(Value::SingleQuotedString(search_path))
| Expr::Value(Value::DoubleQuotedString(search_path)) => {
Expr::Value(ValueWithSpan {
value: Value::SingleQuotedString(search_path),
..
})
| Expr::Value(ValueWithSpan {
value: Value::DoubleQuotedString(search_path),
..
}) => {
ctx.set_current_schema(search_path);
Ok(())
}
@@ -147,7 +167,10 @@ pub fn validate_client_encoding(set: SetVariables) -> Result<()> {
.fail();
};
let encoding = match encoding {
Expr::Value(Value::SingleQuotedString(x))
Expr::Value(ValueWithSpan {
value: Value::SingleQuotedString(x),
..
})
| Expr::Identifier(Ident {
value: x,
quote_style: _,
@@ -210,19 +233,20 @@ fn try_parse_datestyle(expr: &Expr) -> Result<(Option<PGDateTimeStyle>, Option<P
quote_style: _,
span: _,
})
| Expr::Value(Value::SingleQuotedString(s))
| Expr::Value(Value::DoubleQuotedString(s)) => {
s.split(',')
.map(|s| s.trim())
.try_fold((None, None), |(style, order), s| match try_parse_str(s)? {
ParsedDateStyle::Order(o) => {
Ok((style, merge_datestyle_value(order, Some(o))?))
}
ParsedDateStyle::Style(s) => {
Ok((merge_datestyle_value(style, Some(s))?, order))
}
})
}
| Expr::Value(ValueWithSpan {
value: Value::SingleQuotedString(s),
..
})
| Expr::Value(ValueWithSpan {
value: Value::DoubleQuotedString(s),
..
}) => s
.split(',')
.map(|s| s.trim())
.try_fold((None, None), |(style, order), s| match try_parse_str(s)? {
ParsedDateStyle::Order(o) => Ok((style, merge_datestyle_value(order, Some(o))?)),
ParsedDateStyle::Style(s) => Ok((merge_datestyle_value(style, Some(s))?, order)),
}),
_ => NotSupportedSnafu {
feat: "Not supported expression for datestyle",
}
@@ -237,7 +261,10 @@ pub fn set_allow_query_fallback(exprs: Vec<Expr>, ctx: QueryContextRef) -> Resul
feat: "No allow query fallback value find in set variable statement",
})?;
match allow_fallback_expr {
Expr::Value(Value::Boolean(allow)) => {
Expr::Value(ValueWithSpan {
value: Value::Boolean(allow),
span: _,
}) => {
ctx.configuration_parameter()
.set_allow_query_fallback(*allow);
Ok(())
@@ -279,7 +306,10 @@ pub fn set_query_timeout(exprs: Vec<Expr>, ctx: QueryContextRef) -> Result<()> {
feat: "No timeout value find in set query timeout statement",
})?;
match timeout_expr {
Expr::Value(Value::Number(timeout, _)) => {
Expr::Value(ValueWithSpan {
value: Value::Number(timeout, _),
..
}) => {
match timeout.parse::<u64>() {
Ok(timeout) => ctx.set_query_timeout(Duration::from_millis(timeout)),
Err(_) => {
@@ -292,8 +322,14 @@ pub fn set_query_timeout(exprs: Vec<Expr>, ctx: QueryContextRef) -> Result<()> {
Ok(())
}
// postgres support time units i.e. SET STATEMENT_TIMEOUT = '50ms';
Expr::Value(Value::SingleQuotedString(timeout))
| Expr::Value(Value::DoubleQuotedString(timeout)) => {
Expr::Value(ValueWithSpan {
value: Value::SingleQuotedString(timeout),
..
})
| Expr::Value(ValueWithSpan {
value: Value::DoubleQuotedString(timeout),
..
}) => {
if ctx.channel() != Postgres {
return NotSupportedSnafu {
feat: format!("Invalid timeout expr {} in set variable statement", timeout),

View File

@@ -20,6 +20,7 @@ use partition::manager::PartitionInfo;
use session::context::QueryContextRef;
use session::table_name::table_idents_to_full_name;
use snafu::{OptionExt, ResultExt};
use sql::ast::ObjectNamePartExt;
use sql::statements::create::Partitions;
use sql::statements::show::{
ShowColumns, ShowCreateFlow, ShowCreateView, ShowDatabases, ShowFlows, ShowIndex, ShowKind,
@@ -245,8 +246,8 @@ impl StatementExecutor {
) -> Result<Output> {
let obj_name = &show.flow_name;
let (catalog_name, flow_name) = match &obj_name.0[..] {
[table] => (query_ctx.current_catalog().to_string(), table.value.clone()),
[catalog, table] => (catalog.value.clone(), table.value.clone()),
[table] => (query_ctx.current_catalog().to_string(), table.to_string_unquoted()),
[catalog, table] => (catalog.to_string_unquoted(), table.to_string_unquoted()),
_ => {
return InvalidSqlSnafu {
err_msg: format!(

View File

@@ -218,13 +218,13 @@ impl PartitionExpr {
// Otherwise it will be rejected by the parser.
let lhs = match &*self.lhs {
Operand::Column(c) => ParserExpr::Identifier(Ident::new(c.clone())),
Operand::Value(v) => ParserExpr::Value(value_to_sql_value(v).unwrap()),
Operand::Value(v) => ParserExpr::Value(value_to_sql_value(v).unwrap().into()),
Operand::Expr(e) => e.to_parser_expr(),
};
let rhs = match &*self.rhs {
Operand::Column(c) => ParserExpr::Identifier(Ident::new(c.clone())),
Operand::Value(v) => ParserExpr::Value(value_to_sql_value(v).unwrap()),
Operand::Value(v) => ParserExpr::Value(value_to_sql_value(v).unwrap().into()),
Operand::Expr(e) => e.to_parser_expr(),
};

View File

@@ -23,8 +23,10 @@ use common_query::OutputData;
use common_recordbatch::util as record_util;
use common_telemetry::{debug, info};
use common_time::timestamp::{TimeUnit, Timestamp};
use datafusion_common::{TableReference, ToDFSchema};
use datafusion_expr::{col, DmlStatement, LogicalPlan};
use datafusion::datasource::DefaultTableSource;
use datafusion::logical_expr::col;
use datafusion_common::TableReference;
use datafusion_expr::{DmlStatement, LogicalPlan};
use datatypes::prelude::ScalarVector;
use datatypes::timestamp::TimestampNanosecond;
use datatypes::vectors::{StringVector, TimestampNanosecondVector, Vector};
@@ -36,6 +38,7 @@ use query::QueryEngineRef;
use session::context::{QueryContextBuilder, QueryContextRef};
use snafu::{ensure, OptionExt, ResultExt};
use table::metadata::TableInfo;
use table::table::adapter::DfTableProviderAdapter;
use table::TableRef;
use crate::error::{
@@ -424,20 +427,13 @@ impl PipelineTable {
table_info.name.clone(),
);
let df_schema = Arc::new(
table_info
.meta
.schema
.arrow_schema()
.clone()
.to_dfschema()
.context(BuildDfLogicalPlanSnafu)?,
);
let table_provider = Arc::new(DfTableProviderAdapter::new(self.table.clone()));
let table_source = Arc::new(DefaultTableSource::new(table_provider));
// create dml stmt
let stmt = DmlStatement::new(
table_name,
df_schema,
table_source,
datafusion_expr::WriteOp::Delete,
Arc::new(dataframe.into_parts().1),
);

View File

@@ -23,7 +23,9 @@ use datafusion::arrow::array::Array;
use datafusion::common::{DFSchemaRef, Result as DataFusionResult};
use datafusion::execution::context::TaskContext;
use datafusion::logical_expr::{Expr, LogicalPlan, UserDefinedLogicalNodeCore};
use datafusion::physical_expr::{EquivalenceProperties, LexRequirement, PhysicalSortRequirement};
use datafusion::physical_expr::{
EquivalenceProperties, LexRequirement, OrderingRequirements, PhysicalSortRequirement,
};
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::expressions::Column as ColumnExpr;
use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
@@ -278,8 +280,8 @@ impl ExecutionPlan for AbsentExec {
vec![Distribution::SinglePartition]
}
fn required_input_ordering(&self) -> Vec<Option<LexRequirement>> {
vec![Some(LexRequirement::new(vec![PhysicalSortRequirement {
fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>> {
let requirement = LexRequirement::from([PhysicalSortRequirement {
expr: Arc::new(
ColumnExpr::new_with_schema(&self.time_index_column, &self.input.schema()).unwrap(),
),
@@ -287,7 +289,8 @@ impl ExecutionPlan for AbsentExec {
descending: false,
nulls_first: false,
}),
}]))]
}]);
vec![Some(OrderingRequirements::new(requirement))]
}
fn maintains_input_order(&self) -> Vec<bool> {
@@ -358,7 +361,9 @@ impl ExecutionPlan for AbsentExec {
impl DisplayAs for AbsentExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(
f,
"PromAbsentExec: start={}, end={}, step={}",
@@ -509,7 +514,8 @@ mod tests {
use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit};
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::catalog::memory::DataSourceExec;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::prelude::SessionContext;
use datatypes::arrow::array::{Float64Array, TimestampMillisecondArray};
@@ -532,7 +538,9 @@ mod tests {
let batch =
RecordBatch::try_new(schema.clone(), vec![timestamp_array, value_array]).unwrap();
let memory_exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap();
let memory_exec = DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![batch]], schema, None).unwrap(),
));
let output_schema = Arc::new(Schema::new(vec![
Field::new(
@@ -599,7 +607,9 @@ mod tests {
]));
// Empty input
let memory_exec = MemoryExec::try_new(&[vec![]], schema, None).unwrap();
let memory_exec = DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![]], schema, None).unwrap(),
));
let output_schema = Arc::new(Schema::new(vec![
Field::new(

View File

@@ -26,6 +26,7 @@ use datafusion::common::stats::Precision;
use datafusion::common::{
DFSchema, DFSchemaRef, Result as DataFusionResult, Statistics, TableReference,
};
use datafusion::datasource::{provider_as_source, MemTable};
use datafusion::error::DataFusionError;
use datafusion::execution::context::{SessionState, TaskContext};
use datafusion::logical_expr::{ExprSchemable, LogicalPlan, UserDefinedLogicalNodeCore};
@@ -38,6 +39,7 @@ use datafusion::physical_plan::{
};
use datafusion::physical_planner::PhysicalPlanner;
use datafusion::prelude::{col, lit, Expr};
use datafusion_expr::LogicalPlanBuilder;
use datatypes::arrow::array::TimestampMillisecondArray;
use datatypes::arrow::datatypes::SchemaRef;
use datatypes::arrow::record_batch::RecordBatch;
@@ -60,6 +62,12 @@ pub struct EmptyMetric {
time_index_schema: DFSchemaRef,
/// Schema of the output record batch
result_schema: DFSchemaRef,
// This dummy input's sole purpose is to provide a schema for use in DataFusion's
// `SimplifyExpressions`. Otherwise it may report a "no field name ..." error.
// The error is caused by an optimization that tries to rewrite "A = A", during
// which will find the field in plan's schema. However, the schema is empty if the
// plan does not have an input.
dummy_input: LogicalPlan,
}
impl EmptyMetric {
@@ -83,6 +91,11 @@ impl EmptyMetric {
}
let schema = Arc::new(DFSchema::new_with_metadata(fields, HashMap::new())?);
let table = MemTable::try_new(Arc::new(schema.as_arrow().clone()), vec![vec![]])?;
let source = provider_as_source(Arc::new(table));
let dummy_input =
LogicalPlanBuilder::scan("dummy", source, None).and_then(|x| x.build())?;
Ok(Self {
start,
end,
@@ -90,6 +103,7 @@ impl EmptyMetric {
time_index_schema: Arc::new(ts_only_schema),
result_schema: schema,
expr: field_expr,
dummy_input,
})
}
@@ -135,7 +149,7 @@ impl UserDefinedLogicalNodeCore for EmptyMetric {
}
fn inputs(&self) -> Vec<&LogicalPlan> {
vec![]
vec![&self.dummy_input]
}
fn schema(&self) -> &DFSchemaRef {
@@ -170,6 +184,7 @@ impl UserDefinedLogicalNodeCore for EmptyMetric {
expr: exprs.into_iter().next(),
time_index_schema: self.time_index_schema.clone(),
result_schema: self.result_schema.clone(),
dummy_input: self.dummy_input.clone(),
})
}
}
@@ -258,7 +273,11 @@ impl ExecutionPlan for EmptyMetricExec {
Some(self.metric.clone_inner())
}
fn statistics(&self) -> DataFusionResult<Statistics> {
fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
if partition.is_some() {
return Ok(Statistics::new_unknown(self.schema().as_ref()));
}
let estimated_row_num = if self.end > self.start {
(self.end - self.start) as f64 / self.interval as f64
} else {
@@ -281,7 +300,9 @@ impl ExecutionPlan for EmptyMetricExec {
impl DisplayAs for EmptyMetricExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => write!(
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => write!(
f,
"EmptyMetric: range=[{}..{}], interval=[{}]",
self.start, self.end, self.interval,
@@ -328,7 +349,7 @@ impl Stream for EmptyMetricStream {
let num_rows = time_array.len();
let input_record_batch =
RecordBatch::try_new(self.time_index_schema.clone(), vec![time_array.clone()])
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
let mut result_arrays: Vec<ArrayRef> = vec![time_array];
// evaluate the field expr and get the result
@@ -342,7 +363,7 @@ impl Stream for EmptyMetricStream {
// assemble the output record batch
let batch = RecordBatch::try_new(self.result_schema.clone(), result_arrays)
.map_err(|e| DataFusionError::ArrowError(e, None));
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None));
Poll::Ready(Some(batch))
} else {

View File

@@ -29,7 +29,9 @@ use datafusion::common::{ColumnStatistics, DFSchema, DFSchemaRef, Statistics};
use datafusion::error::{DataFusionError, Result as DataFusionResult};
use datafusion::execution::TaskContext;
use datafusion::logical_expr::{LogicalPlan, UserDefinedLogicalNodeCore};
use datafusion::physical_expr::{EquivalenceProperties, LexRequirement, PhysicalSortRequirement};
use datafusion::physical_expr::{
EquivalenceProperties, LexRequirement, OrderingRequirements, PhysicalSortRequirement,
};
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::expressions::{CastExpr as PhyCast, Column as PhyColumn};
use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
@@ -150,10 +152,10 @@ impl HistogramFold {
let check_column = |col| {
if !input_schema.has_column_with_unqualified_name(col) {
Err(DataFusionError::SchemaError(
datafusion::common::SchemaError::FieldNotFound {
Box::new(datafusion::common::SchemaError::FieldNotFound {
field: Box::new(Column::new(None::<String>, col)),
valid_fields: input_schema.columns(),
},
}),
Box::new(None),
))
} else {
@@ -266,7 +268,7 @@ impl ExecutionPlan for HistogramFoldExec {
&self.properties
}
fn required_input_ordering(&self) -> Vec<Option<LexRequirement>> {
fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>> {
let mut cols = self
.tag_col_exprs()
.into_iter()
@@ -299,7 +301,10 @@ impl ExecutionPlan for HistogramFoldExec {
}),
});
vec![Some(LexRequirement::new(cols))]
// Safety: `cols` is not empty
let requirement = LexRequirement::new(cols).unwrap();
vec![Some(OrderingRequirements::Hard(vec![requirement]))]
}
fn required_input_distribution(&self) -> Vec<Distribution> {
@@ -370,7 +375,7 @@ impl ExecutionPlan for HistogramFoldExec {
Some(self.metric.clone_inner())
}
fn statistics(&self) -> DataFusionResult<Statistics> {
fn partition_statistics(&self, _: Option<usize>) -> DataFusionResult<Statistics> {
Ok(Statistics {
num_rows: Precision::Absent,
total_byte_size: Precision::Absent,
@@ -414,7 +419,9 @@ impl HistogramFoldExec {
impl DisplayAs for HistogramFoldExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(
f,
"HistogramFoldExec: le=@{}, field=@{}, quantile={}",
@@ -629,7 +636,7 @@ impl HistogramFoldStream {
self.output_buffered_rows = 0;
RecordBatch::try_new(self.output_schema.clone(), columns)
.map(Some)
.map_err(|e| DataFusionError::ArrowError(e, None))
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))
}
/// Find the first `+Inf` which indicates the end of the bucket group
@@ -729,13 +736,14 @@ mod test {
use datafusion::arrow::array::Float64Array;
use datafusion::arrow::datatypes::{Field, Schema};
use datafusion::common::ToDFSchema;
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::prelude::SessionContext;
use datatypes::arrow_array::StringArray;
use super::*;
fn prepare_test_data() -> MemoryExec {
fn prepare_test_data() -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new("host", DataType::Utf8, true),
Field::new("le", DataType::Utf8, true),
@@ -788,7 +796,9 @@ mod test {
)
.unwrap();
MemoryExec::try_new(&[vec![data_1, data_2, data_3]], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![data_1, data_2, data_3]], schema, None).unwrap(),
))
}
#[tokio::test]

View File

@@ -283,8 +283,8 @@ impl ExecutionPlan for InstantManipulateExec {
Some(self.metric.clone_inner())
}
fn statistics(&self) -> DataFusionResult<Statistics> {
let input_stats = self.input.statistics()?;
fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
let input_stats = self.input.partition_statistics(partition)?;
let estimated_row_num = (self.end - self.start) as f64 / self.interval as f64;
let estimated_total_bytes = input_stats
@@ -315,7 +315,9 @@ impl ExecutionPlan for InstantManipulateExec {
impl DisplayAs for InstantManipulateExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(
f,
"PromInstantManipulateExec: range=[{}..{}], lookback=[{}], interval=[{}], time index=[{}]",
@@ -500,7 +502,7 @@ impl InstantManipulateStream {
arrays[self.time_index] = Arc::new(TimestampMillisecondArray::from(aligned_ts));
let result = RecordBatch::try_new(record_batch.schema(), arrays)
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
Ok(result)
}
}

View File

@@ -254,8 +254,8 @@ impl ExecutionPlan for SeriesNormalizeExec {
Some(self.metric.clone_inner())
}
fn statistics(&self) -> DataFusionResult<Statistics> {
self.input.statistics()
fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
self.input.partition_statistics(partition)
}
fn name(&self) -> &str {
@@ -266,7 +266,9 @@ impl ExecutionPlan for SeriesNormalizeExec {
impl DisplayAs for SeriesNormalizeExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(
f,
"PromSeriesNormalizeExec: offset=[{}], time index=[{}], filter NaN: [{}]",
@@ -332,7 +334,7 @@ impl SeriesNormalizeStream {
}
let result = compute::filter_record_batch(&result_batch, &BooleanArray::from(filter))
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
Ok(result)
}
}
@@ -371,7 +373,8 @@ mod test {
use datafusion::arrow::datatypes::{
ArrowPrimitiveType, DataType, Field, Schema, TimestampMillisecondType,
};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::prelude::SessionContext;
use datatypes::arrow::array::TimestampMillisecondArray;
use datatypes::arrow_array::StringArray;
@@ -380,7 +383,7 @@ mod test {
const TIME_INDEX_COLUMN: &str = "timestamp";
fn prepare_test_data() -> MemoryExec {
fn prepare_test_data() -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
Field::new("value", DataType::Float64, true),
@@ -397,7 +400,9 @@ mod test {
)
.unwrap();
MemoryExec::try_new(&[vec![data]], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![data]], schema, None).unwrap(),
))
}
#[tokio::test]

View File

@@ -430,8 +430,8 @@ impl ExecutionPlan for RangeManipulateExec {
Some(self.metric.clone_inner())
}
fn statistics(&self) -> DataFusionResult<Statistics> {
let input_stats = self.input.statistics()?;
fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
let input_stats = self.input.partition_statistics(partition)?;
let estimated_row_num = (self.end - self.start) as f64 / self.interval as f64;
let estimated_total_bytes = input_stats
@@ -459,7 +459,9 @@ impl ExecutionPlan for RangeManipulateExec {
impl DisplayAs for RangeManipulateExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(
f,
"PromRangeManipulateExec: req range=[{}..{}], interval=[{}], eval range=[{}], time index=[{}]",
@@ -569,7 +571,7 @@ impl RangeManipulateStream {
RecordBatch::try_new(self.output_schema.clone(), new_columns)
.map(Some)
.map_err(|e| DataFusionError::ArrowError(e, None))
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))
}
fn build_aligned_ts_array(start: i64, end: i64, interval: i64) -> ArrayRef {
@@ -665,9 +667,10 @@ mod test {
ArrowPrimitiveType, DataType, Field, Int64Type, Schema, TimestampMillisecondType,
};
use datafusion::common::ToDFSchema;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::physical_expr::Partitioning;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::prelude::SessionContext;
use datatypes::arrow::array::TimestampMillisecondArray;
@@ -675,7 +678,7 @@ mod test {
const TIME_INDEX_COLUMN: &str = "timestamp";
fn prepare_test_data() -> MemoryExec {
fn prepare_test_data() -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
Field::new("value_1", DataType::Float64, true),
@@ -700,7 +703,9 @@ mod test {
)
.unwrap();
MemoryExec::try_new(&[vec![data]], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![data]], schema, None).unwrap(),
))
}
async fn do_normalize_test(

View File

@@ -123,10 +123,10 @@ impl ScalarCalculate {
let input_schema = exec_input.schema();
let ts_index = input_schema
.index_of(&self.time_index)
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
let val_index = input_schema
.index_of(&self.field_column)
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
let schema = Arc::new(Schema::new(fields));
let properties = exec_input.properties();
let properties = PlanProperties::new(
@@ -366,8 +366,8 @@ impl ExecutionPlan for ScalarCalculateExec {
Some(self.metric.clone_inner())
}
fn statistics(&self) -> DataFusionResult<Statistics> {
let input_stats = self.input.statistics()?;
fn partition_statistics(&self, partition: Option<usize>) -> DataFusionResult<Statistics> {
let input_stats = self.input.partition_statistics(partition)?;
let estimated_row_num = (self.end - self.start) as f64 / self.interval as f64;
let estimated_total_bytes = input_stats
@@ -395,7 +395,9 @@ impl ExecutionPlan for ScalarCalculateExec {
impl DisplayAs for ScalarCalculateExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(f, "ScalarCalculateExec: tags={:?}", self.tag_columns)
}
}
@@ -535,22 +537,25 @@ impl Stream for ScalarCalculateStream {
#[cfg(test)]
mod test {
use datafusion::arrow::datatypes::{DataType, Field, Schema};
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::prelude::SessionContext;
use datatypes::arrow::array::{Float64Array, TimestampMillisecondArray};
use datatypes::arrow::datatypes::TimeUnit;
use super::*;
fn prepare_test_data(series: Vec<RecordBatch>) -> MemoryExec {
fn prepare_test_data(series: Vec<RecordBatch>) -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new("ts", DataType::Timestamp(TimeUnit::Millisecond, None), true),
Field::new("tag1", DataType::Utf8, true),
Field::new("tag2", DataType::Utf8, true),
Field::new("val", DataType::Float64, true),
]));
MemoryExec::try_new(&[series], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[series], schema, None).unwrap(),
))
}
async fn run_test(series: Vec<RecordBatch>, expected: &str) {

View File

@@ -24,7 +24,7 @@ use datafusion::common::{DFSchema, DFSchemaRef};
use datafusion::error::Result as DataFusionResult;
use datafusion::execution::context::TaskContext;
use datafusion::logical_expr::{EmptyRelation, Expr, LogicalPlan, UserDefinedLogicalNodeCore};
use datafusion::physical_expr::{LexRequirement, PhysicalSortRequirement};
use datafusion::physical_expr::{LexRequirement, OrderingRequirements, PhysicalSortRequirement};
use datafusion::physical_plan::expressions::Column as ColumnExpr;
use datafusion::physical_plan::metrics::{
BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricValue, MetricsSet,
@@ -175,7 +175,7 @@ impl ExecutionPlan for SeriesDivideExec {
)]
}
fn required_input_ordering(&self) -> Vec<Option<LexRequirement>> {
fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>> {
let input_schema = self.input.schema();
let mut exprs: Vec<PhysicalSortRequirement> = self
.tag_columns
@@ -199,7 +199,11 @@ impl ExecutionPlan for SeriesDivideExec {
nulls_first: true,
}),
});
vec![Some(LexRequirement::new(exprs))]
// Safety: `exprs` is not empty
let requirement = LexRequirement::new(exprs).unwrap();
vec![Some(OrderingRequirements::Hard(vec![requirement]))]
}
fn maintains_input_order(&self) -> Vec<bool> {
@@ -273,7 +277,9 @@ impl ExecutionPlan for SeriesDivideExec {
impl DisplayAs for SeriesDivideExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(f, "PromSeriesDivideExec: tags={:?}", self.tag_columns)
}
}
@@ -486,12 +492,13 @@ impl SeriesDivideStream {
#[cfg(test)]
mod test {
use datafusion::arrow::datatypes::{DataType, Field, Schema};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::prelude::SessionContext;
use super::*;
fn prepare_test_data() -> MemoryExec {
fn prepare_test_data() -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new("host", DataType::Utf8, true),
Field::new("path", DataType::Utf8, true),
@@ -547,7 +554,9 @@ mod test {
)
.unwrap();
MemoryExec::try_new(&[vec![data_1, data_2, data_3]], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![data_1, data_2, data_3]], schema, None).unwrap(),
))
}
#[tokio::test]
@@ -792,8 +801,8 @@ mod test {
.unwrap();
// Create MemoryExec with these batches, keeping same combinations adjacent
let memory_exec = Arc::new(
MemoryExec::try_new(
let memory_exec = DataSourceExec::from_data_source(
MemorySourceConfig::try_new(
&[vec![batch1, batch2, batch3, batch4, batch5, batch6]],
schema.clone(),
None,

View File

@@ -21,13 +21,14 @@ use datafusion::arrow::array::Float64Array;
use datafusion::arrow::datatypes::{
ArrowPrimitiveType, DataType, Field, Schema, TimestampMillisecondType,
};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datatypes::arrow::array::TimestampMillisecondArray;
use datatypes::arrow_array::StringArray;
pub(crate) const TIME_INDEX_COLUMN: &str = "timestamp";
pub(crate) fn prepare_test_data() -> MemoryExec {
pub(crate) fn prepare_test_data() -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
Field::new("value", DataType::Float64, true),
@@ -46,10 +47,12 @@ pub(crate) fn prepare_test_data() -> MemoryExec {
)
.unwrap();
MemoryExec::try_new(&[vec![data]], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![data]], schema, None).unwrap(),
))
}
pub(crate) fn prepare_test_data_with_nan() -> MemoryExec {
pub(crate) fn prepare_test_data_with_nan() -> DataSourceExec {
let schema = Arc::new(Schema::new(vec![
Field::new(TIME_INDEX_COLUMN, TimestampMillisecondType::DATA_TYPE, true),
Field::new("value", DataType::Float64, true),
@@ -60,5 +63,7 @@ pub(crate) fn prepare_test_data_with_nan() -> MemoryExec {
let field_column = Arc::new(Float64Array::from(vec![0.0, f64::NAN, 6.0, f64::NAN, 12.0])) as _;
let data = RecordBatch::try_new(schema.clone(), vec![timestamp_column, field_column]).unwrap();
MemoryExec::try_new(&[vec![data]], schema, None).unwrap()
DataSourceExec::new(Arc::new(
MemorySourceConfig::try_new(&[vec![data]], schema, None).unwrap(),
))
}

View File

@@ -292,7 +292,9 @@ impl ExecutionPlan for UnionDistinctOnExec {
impl DisplayAs for UnionDistinctOnExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(
f,
"UnionDistinctOnExec: on col=[{:?}], ts_col=[{}]",
@@ -507,7 +509,7 @@ fn interleave_batches(
// assemble new record batch
RecordBatch::try_new(schema, interleaved_arrays)
.map_err(|e| DataFusionError::ArrowError(e, None))
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))
}
/// Utility function to take rows from a record batch. Based on [take](datafusion::arrow::compute::take)
@@ -525,10 +527,10 @@ fn take_batch(batch: &RecordBatch, indices: &[usize]) -> DataFusionResult<Record
.iter()
.map(|array| compute::take(array, &indices_array, None))
.collect::<std::result::Result<Vec<_>, _>>()
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
let result =
RecordBatch::try_new(schema, arrays).map_err(|e| DataFusionError::ArrowError(e, None))?;
let result = RecordBatch::try_new(schema, arrays)
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
Ok(result)
}

View File

@@ -97,7 +97,7 @@ impl Round {
(a / nearest).round() * nearest
}
})
.map_err(|err: ArrowError| DataFusionError::ArrowError(err, None))?;
.map_err(|err: ArrowError| DataFusionError::ArrowError(Box::new(err), None))?;
Ok(ColumnarValue::Array(Arc::new(result) as _))
}
@@ -107,8 +107,10 @@ impl Round {
#[cfg(test)]
mod tests {
use datafusion_common::config::ConfigOptions;
use datafusion_expr::ScalarFunctionArgs;
use datatypes::arrow::array::Float64Array;
use datatypes::arrow::datatypes::Field;
use super::*;
@@ -118,10 +120,17 @@ mod tests {
ColumnarValue::Array(Arc::new(Float64Array::from(value))),
ColumnarValue::Scalar(ScalarValue::Float64(Some(nearest))),
];
let arg_fields = vec![
Arc::new(Field::new("a", input[0].data_type(), false)),
Arc::new(Field::new("b", input[1].data_type(), false)),
];
let return_field = Arc::new(Field::new("c", DataType::Float64, false));
let args = ScalarFunctionArgs {
args: input,
arg_fields,
number_rows: 1,
return_type: &DataType::Float64,
return_field,
config_options: Arc::new(ConfigOptions::default()),
};
let result = round_udf.invoke_with_args(args).unwrap();
let result_array = extract_array(&result).unwrap();

View File

@@ -17,9 +17,10 @@ use std::sync::Arc;
use datafusion::arrow::array::Float64Array;
use datafusion::logical_expr::ScalarUDF;
use datafusion::physical_plan::ColumnarValue;
use datafusion_common::config::ConfigOptions;
use datafusion_common::ScalarValue;
use datafusion_expr::ScalarFunctionArgs;
use datatypes::arrow::datatypes::DataType;
use datatypes::arrow::datatypes::{DataType, Field};
use crate::functions::extract_array;
use crate::range_array::RangeArray;
@@ -40,10 +41,17 @@ pub fn simple_range_udf_runner(
.into_iter()
.chain(other_args.into_iter().map(ColumnarValue::Scalar))
.collect::<Vec<_>>();
let arg_fields = vec![
Arc::new(Field::new("a", input[0].data_type(), false)),
Arc::new(Field::new("b", input[1].data_type(), false)),
];
let return_field = Arc::new(Field::new("c", DataType::Float64, false));
let args = ScalarFunctionArgs {
args: input,
arg_fields,
number_rows: num_rows,
return_type: &DataType::Float64,
return_field,
config_options: Arc::new(ConfigOptions::default()),
};
let value = range_fn.invoke_with_args(args).unwrap();
let eval_result: Vec<Option<f64>> = extract_array(&value)

View File

@@ -89,7 +89,9 @@ impl DistAnalyzeExec {
impl DisplayAs for DistAnalyzeExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
DisplayFormatType::Default
| DisplayFormatType::Verbose
| DisplayFormatType::TreeRender => {
write!(f, "DistAnalyzeExec",)
}
}

View File

@@ -206,6 +206,7 @@ impl DistPlannerAnalyzer {
Ok(Subquery {
subquery: Arc::new(rewrote_subquery),
outer_ref_columns: subquery.outer_ref_columns,
spans: Default::default(),
})
}
}

View File

@@ -646,7 +646,8 @@ fn expand_part_col_aggr_part_col_aggr() {
.unwrap();
let expected_original = [
"Aggregate: groupBy=[[t.pk1, t.pk2, max(t.number)]], aggr=[[min(max(t.number))]]", // notice here `max(t.number)` is added to groupBy due to aggr exprs depend on this column
// See DataFusion #14860 for change details.
"Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(max(t.number))]]",
" Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]",
" TableScan: t",
]
@@ -657,9 +658,9 @@ fn expand_part_col_aggr_part_col_aggr() {
let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap();
let expected = [
"Projection: t.pk1, t.pk2, max(t.number), min(max(t.number))",
"Projection: t.pk1, t.pk2, min(max(t.number))",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[t.pk1, t.pk2, max(t.number)]], aggr=[[min(max(t.number))]]",
"Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[min(max(t.number))]]",
" Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]",
" TableScan: t",
"]]",

View File

@@ -78,7 +78,7 @@ pub fn step_aggr_to_upper_aggr(
pub fn is_all_aggr_exprs_steppable(aggr_exprs: &[Expr]) -> bool {
aggr_exprs.iter().all(|expr| {
if let Some(aggr_func) = get_aggr_func(expr) {
if aggr_func.distinct {
if aggr_func.params.distinct {
// Distinct aggregate functions are not steppable(yet).
return false;
}
@@ -259,10 +259,11 @@ impl Categorizer {
}
pub fn check_expr(expr: &Expr) -> Commutativity {
#[allow(deprecated)]
match expr {
Expr::Column(_)
| Expr::ScalarVariable(_, _)
| Expr::Literal(_)
| Expr::Literal(_, _)
| Expr::BinaryExpr(_)
| Expr::Not(_)
| Expr::IsNotNull(_)

View File

@@ -40,9 +40,7 @@ use datafusion::physical_plan::{
use datafusion_common::{Column as ColumnExpr, Result};
use datafusion_expr::{Expr, Extension, LogicalPlan, UserDefinedLogicalNodeCore};
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_expr::{
Distribution, EquivalenceProperties, LexOrdering, PhysicalSortExpr,
};
use datafusion_physical_expr::{Distribution, EquivalenceProperties, PhysicalSortExpr};
use datatypes::schema::{Schema, SchemaRef};
use futures_util::StreamExt;
use greptime_proto::v1::region::RegionRequestHeader;
@@ -209,10 +207,7 @@ impl MergeScanExec {
))
})
.collect::<Result<Vec<_>>>()?;
EquivalenceProperties::new_with_orderings(
arrow_schema.clone(),
&[LexOrdering::new(lex_ordering)],
)
EquivalenceProperties::new_with_orderings(arrow_schema.clone(), vec![lex_ordering])
} else {
EquivalenceProperties::new(arrow_schema.clone())
};

View File

@@ -398,7 +398,7 @@ impl DataFusionExprConverter {
};
Ok(Operand::Column(column_name))
}
Expr::Literal(scalar_value) => {
Expr::Literal(scalar_value, _) => {
let value = Value::try_from(scalar_value.clone()).unwrap();
Ok(Operand::Value(value))
}

View File

@@ -17,6 +17,7 @@
#![feature(try_blocks)]
#![feature(stmt_expr_attributes)]
#![feature(iterator_try_collect)]
#![feature(box_patterns)]
mod analyze;
pub mod dataframe;

View File

@@ -1546,7 +1546,7 @@ mod tests {
.unwrap();
// Verify the nested structure is properly created
let expected_expr_debug = "BinaryExpr(BinaryExpr { left: BinaryExpr(BinaryExpr { left: Column(Column { relation: None, name: \"age\" }), op: Plus, right: Literal(Int32(5)) }), op: Gt, right: Literal(Int32(30)) })";
let expected_expr_debug = r#"BinaryExpr(BinaryExpr { left: BinaryExpr(BinaryExpr { left: Column(Column { relation: None, name: "age" }), op: Plus, right: Literal(Int32(5), None) }), op: Gt, right: Literal(Int32(30), None) })"#;
assert_eq!(format!("{:?}", expr), expected_expr_debug);
}
}

View File

@@ -13,6 +13,7 @@
// limitations under the License.
use std::fmt;
use std::fmt::Formatter;
use std::hash::{Hash, Hasher};
use std::sync::Arc;
@@ -129,6 +130,10 @@ impl PhysicalExpr for PreCompiledMatchesTermExpr {
probes: self.probes.clone(),
}))
}
fn fmt_sql(&self, f: &mut Formatter<'_>) -> fmt::Result {
write!(f, "{}", self)
}
}
/// Optimizer rule that pre-compiles constant term in `matches_term` function.
@@ -236,13 +241,14 @@ mod tests {
use common_function::scalars::matches_term::MatchesTermFunction;
use common_function::scalars::udf::create_udf;
use common_function::state::FunctionState;
use datafusion::datasource::memory::MemorySourceConfig;
use datafusion::datasource::source::DataSourceExec;
use datafusion::physical_optimizer::PhysicalOptimizerRule;
use datafusion::physical_plan::filter::FilterExec;
use datafusion::physical_plan::get_plan_string;
use datafusion::physical_plan::memory::MemoryExec;
use datafusion_common::{Column, DFSchema, ScalarValue};
use datafusion_common::{Column, DFSchema};
use datafusion_expr::expr::ScalarFunction;
use datafusion_expr::{Expr, ScalarUDF};
use datafusion_expr::{Expr, Literal, ScalarUDF};
use datafusion_physical_expr::{create_physical_expr, ScalarFunctionExpr};
use datatypes::prelude::ConcreteDataType;
use datatypes::schema::ColumnSchema;
@@ -335,18 +341,16 @@ mod tests {
let predicate = create_physical_expr(
&Expr::ScalarFunction(ScalarFunction::new_udf(
matches_term_udf(),
vec![
Expr::Column(Column::from_name("text")),
Expr::Literal(ScalarValue::Utf8(Some("hello".to_string()))),
],
vec![Expr::Column(Column::from_name("text")), "hello".lit()],
)),
&DFSchema::try_from(batch.schema().clone()).unwrap(),
&Default::default(),
)
.unwrap();
let input =
Arc::new(MemoryExec::try_new(&[vec![batch.clone()]], batch.schema(), None).unwrap());
let input = DataSourceExec::from_data_source(
MemorySourceConfig::try_new(&[vec![batch.clone()]], batch.schema(), None).unwrap(),
);
let filter = FilterExec::try_new(predicate, input).unwrap();
// Apply the optimizer
@@ -385,8 +389,9 @@ mod tests {
)
.unwrap();
let input =
Arc::new(MemoryExec::try_new(&[vec![batch.clone()]], batch.schema(), None).unwrap());
let input = DataSourceExec::from_data_source(
MemorySourceConfig::try_new(&[vec![batch.clone()]], batch.schema(), None).unwrap(),
);
let filter = FilterExec::try_new(predicate, input).unwrap();
let optimizer = MatchesConstantTermOptimizer;

View File

@@ -16,7 +16,7 @@ use datafusion::datasource::DefaultTableSource;
use datafusion_common::tree_node::{
Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeVisitor,
};
use datafusion_common::{Column, Result as DataFusionResult};
use datafusion_common::{Column, Result as DataFusionResult, ScalarValue};
use datafusion_expr::expr::{AggregateFunction, WindowFunction};
use datafusion_expr::utils::COUNT_STAR_EXPANSION;
use datafusion_expr::{col, lit, Expr, LogicalPlan, WindowFunctionDefinition};
@@ -62,13 +62,13 @@ impl CountWildcardToTimeIndexRule {
Expr::WindowFunction(mut window_function)
if Self::is_count_star_window_aggregate(&window_function) =>
{
window_function.args.clone_from(&new_arg);
window_function.params.args.clone_from(&new_arg);
Ok(Transformed::yes(Expr::WindowFunction(window_function)))
}
Expr::AggregateFunction(mut aggregate_function)
if Self::is_count_star_aggregate(&aggregate_function) =>
{
aggregate_function.args.clone_from(&new_arg);
aggregate_function.params.args.clone_from(&new_arg);
Ok(Transformed::yes(Expr::AggregateFunction(
aggregate_function,
)))
@@ -105,24 +105,30 @@ impl CountWildcardToTimeIndexRule {
/// Utility functions from the original rule.
impl CountWildcardToTimeIndexRule {
fn is_wildcard(expr: &Expr) -> bool {
matches!(expr, Expr::Wildcard { .. })
#[expect(deprecated)]
fn args_at_most_wildcard_or_literal_one(args: &[Expr]) -> bool {
match args {
[] => true,
[Expr::Literal(ScalarValue::Int64(Some(v)), _)] => *v == 1,
[Expr::Wildcard { .. }] => true,
_ => false,
}
}
fn is_count_star_aggregate(aggregate_function: &AggregateFunction) -> bool {
let args = &aggregate_function.params.args;
matches!(aggregate_function,
AggregateFunction {
func,
args,
..
} if func.name() == "count" && (args.len() == 1 && Self::is_wildcard(&args[0]) || args.is_empty()))
} if func.name() == "count" && Self::args_at_most_wildcard_or_literal_one(args))
}
fn is_count_star_window_aggregate(window_function: &WindowFunction) -> bool {
let args = &window_function.args;
let args = &window_function.params.args;
matches!(window_function.fun,
WindowFunctionDefinition::AggregateUDF(ref udaf)
if udaf.name() == "count" && (args.len() == 1 && Self::is_wildcard(&args[0]) || args.is_empty()))
if udaf.name() == "count" && Self::args_at_most_wildcard_or_literal_one(args))
}
}
@@ -184,8 +190,8 @@ impl TimeIndexFinder {
mod test {
use std::sync::Arc;
use datafusion::functions_aggregate::count::count;
use datafusion_expr::{wildcard, LogicalPlanBuilder};
use datafusion::functions_aggregate::count::count_all;
use datafusion_expr::LogicalPlanBuilder;
use table::table::numbers::NumbersTable;
use super::*;
@@ -199,7 +205,7 @@ mod test {
let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![])
.unwrap()
.aggregate(Vec::<Expr>::new(), vec![count(wildcard())])
.aggregate(Vec::<Expr>::new(), vec![count_all()])
.unwrap()
.alias(r#""FgHiJ""#)
.unwrap()

View File

@@ -57,7 +57,7 @@ impl ParallelizeScan {
.transform_down(|plan| {
if let Some(sort_exec) = plan.as_any().downcast_ref::<SortExec>() {
// save the first order expr
first_order_expr = sort_exec.expr().first().cloned();
first_order_expr = Some(sort_exec.expr().first()).cloned();
} else if let Some(region_scan_exec) =
plan.as_any().downcast_ref::<RegionScanExec>()
{

View File

@@ -217,13 +217,16 @@ impl TreeNodeVisitor<'_> for ScanHintVisitor {
is_all_last_value = false;
break;
};
if func.func.name() != "last_value" || func.filter.is_some() || func.distinct {
if func.func.name() != "last_value"
|| func.params.filter.is_some()
|| func.params.distinct
{
is_all_last_value = false;
break;
}
// check order by requirement
if let Some(order_by) = &func.order_by
&& let Some(first_order_by) = order_by.first()
let order_by = &func.params.order_by;
if let Some(first_order_by) = order_by.first()
&& order_by.len() == 1
{
if let Some(existing_order_by) = &order_by_expr {
@@ -298,7 +301,7 @@ mod test {
use std::sync::Arc;
use datafusion::functions_aggregate::first_last::last_value_udaf;
use datafusion_expr::expr::AggregateFunction;
use datafusion_expr::expr::{AggregateFunction, AggregateFunctionParams};
use datafusion_expr::{col, LogicalPlanBuilder};
use datafusion_optimizer::OptimizerContext;
use store_api::storage::RegionId;
@@ -320,7 +323,6 @@ mod test {
.unwrap();
let context = OptimizerContext::default();
assert!(ScanHintRule.supports_rewrite());
ScanHintRule.rewrite(plan, &context).unwrap();
// should read the first (with `.sort(true, false)`) sort option
@@ -347,15 +349,17 @@ mod test {
vec![col("k0")],
vec![Expr::AggregateFunction(AggregateFunction {
func: last_value_udaf(),
args: vec![col("v0")],
distinct: false,
filter: None,
order_by: Some(vec![Sort {
expr: col("ts"),
asc: true,
nulls_first: true,
}]),
null_treatment: None,
params: AggregateFunctionParams {
args: vec![col("v0")],
distinct: false,
filter: None,
order_by: vec![Sort {
expr: col("ts"),
asc: true,
nulls_first: true,
}],
null_treatment: None,
},
})],
)
.unwrap()
@@ -363,7 +367,6 @@ mod test {
.unwrap();
let context = OptimizerContext::default();
assert!(ScanHintRule.supports_rewrite());
ScanHintRule.rewrite(plan, &context).unwrap();
let scan_req = provider.scan_request();

View File

@@ -90,9 +90,9 @@ impl TreeNodeRewriter for StringNormalizationConverter {
Expr::Cast(Cast { expr, data_type }) => {
let expr = match data_type {
DataType::Timestamp(_, _) => match *expr {
Expr::Literal(value) => match value {
Expr::Literal(value, _) => match value {
ScalarValue::Utf8(Some(s)) => trim_utf_expr(s),
_ => Expr::Literal(value),
_ => Expr::Literal(value, None),
},
expr => expr,
},
@@ -112,7 +112,7 @@ impl TreeNodeRewriter for StringNormalizationConverter {
fn trim_utf_expr(s: String) -> Expr {
let parts: Vec<_> = s.split_whitespace().collect();
let trimmed = parts.join(" ");
Expr::Literal(ScalarValue::Utf8(Some(trimmed)))
Expr::Literal(ScalarValue::Utf8(Some(trimmed)), None)
}
#[cfg(test)]
@@ -195,7 +195,7 @@ mod tests {
fn prepare_test_plan_builder() -> LogicalPlanBuilder {
let schema = Schema::new(vec![Field::new("f", DataType::Float64, false)]);
let table = MemTable::try_new(SchemaRef::from(schema), vec![]).unwrap();
let table = MemTable::try_new(SchemaRef::from(schema), vec![vec![]]).unwrap();
LogicalPlanBuilder::scan("t", provider_as_source(Arc::new(table)), None).unwrap()
}
}

View File

@@ -125,7 +125,7 @@ mod tests {
Field::new("a", DataType::Utf8, false),
Field::new("b", DataType::Utf8, false),
]);
let table = MemTable::try_new(SchemaRef::from(schema), vec![]).unwrap();
let table = MemTable::try_new(SchemaRef::from(schema), vec![vec![]]).unwrap();
LogicalPlanBuilder::scan("t", provider_as_source(Arc::new(table)), None).unwrap()
}

View File

@@ -87,7 +87,6 @@ impl ExtensionAnalyzerRule for TypeConversionRule {
| LogicalPlan::Extension { .. }
| LogicalPlan::Sort { .. }
| LogicalPlan::Union { .. }
| LogicalPlan::Join { .. }
| LogicalPlan::Values { .. }
| LogicalPlan::Analyze { .. } => {
let mut converter = TypeConverter {
@@ -116,7 +115,8 @@ impl ExtensionAnalyzerRule for TypeConversionRule {
| LogicalPlan::Statement(_)
| LogicalPlan::Ddl(_)
| LogicalPlan::Copy(_)
| LogicalPlan::RecursiveQuery(_) => Ok(Transformed::no(plan)),
| LogicalPlan::RecursiveQuery(_)
| LogicalPlan::Join { .. } => Ok(Transformed::no(plan)),
})
.map(|x| x.data)
}
@@ -154,7 +154,7 @@ impl TypeConverter {
(target_type, value) => {
let value_arr = value.to_array()?;
let arr = compute::cast(&value_arr, target_type)
.map_err(|e| DataFusionError::ArrowError(e, None))?;
.map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
ScalarValue::try_from_array(
&arr,
@@ -180,23 +180,23 @@ impl TypeConverter {
}
match (left, right) {
(Expr::Column(col), Expr::Literal(value)) => {
(Expr::Column(col), Expr::Literal(value, _)) => {
let casted_right = self.cast_scalar_value(value, target_type)?;
if casted_right.is_null() {
return Err(DataFusionError::Plan(format!(
"column:{col:?}. Casting value:{value:?} to {target_type:?} is invalid",
)));
}
Ok((left.clone(), Expr::Literal(casted_right)))
Ok((left.clone(), Expr::Literal(casted_right, None)))
}
(Expr::Literal(value), Expr::Column(col)) => {
(Expr::Literal(value, _), Expr::Column(col)) => {
let casted_left = self.cast_scalar_value(value, target_type)?;
if casted_left.is_null() {
return Err(DataFusionError::Plan(format!(
"column:{col:?}. Casting value:{value:?} to {target_type:?} is invalid",
)));
}
Ok((Expr::Literal(casted_left), right.clone()))
Ok((Expr::Literal(casted_left, None), right.clone()))
}
_ => Ok((left.clone(), right.clone())),
}
@@ -255,7 +255,7 @@ impl TreeNodeRewriter for TypeConverter {
negated,
})
}
Expr::Literal(value) => match value {
Expr::Literal(value, _) => match value {
ScalarValue::TimestampSecond(Some(i), _) => {
timestamp_to_timestamp_ms_expr(i, TimeUnit::Second)
}
@@ -268,7 +268,7 @@ impl TreeNodeRewriter for TypeConverter {
ScalarValue::TimestampNanosecond(Some(i), _) => {
timestamp_to_timestamp_ms_expr(i, TimeUnit::Nanosecond)
}
_ => Expr::Literal(value),
_ => Expr::Literal(value, None),
},
expr => expr,
};
@@ -284,7 +284,10 @@ fn timestamp_to_timestamp_ms_expr(val: i64, unit: TimeUnit) -> Expr {
TimeUnit::Nanosecond => val / 1_000 / 1_000,
};
Expr::Literal(ScalarValue::TimestampMillisecond(Some(timestamp), None))
Expr::Literal(
ScalarValue::TimestampMillisecond(Some(timestamp), None),
None,
)
}
fn string_to_timestamp_ms(string: &str, timezone: Option<&Timezone>) -> Result<ScalarValue> {
@@ -308,7 +311,7 @@ mod tests {
use datafusion_common::arrow::datatypes::Field;
use datafusion_common::{Column, DFSchema};
use datafusion_expr::LogicalPlanBuilder;
use datafusion_expr::{Literal, LogicalPlanBuilder};
use datafusion_sql::TableReference;
use session::context::QueryContext;
@@ -348,36 +351,36 @@ mod tests {
fn test_timestamp_to_timestamp_ms_expr() {
assert_eq!(
timestamp_to_timestamp_ms_expr(123, TimeUnit::Second),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(123000), None))
ScalarValue::TimestampMillisecond(Some(123000), None).lit()
);
assert_eq!(
timestamp_to_timestamp_ms_expr(123, TimeUnit::Millisecond),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(123), None))
ScalarValue::TimestampMillisecond(Some(123), None).lit()
);
assert_eq!(
timestamp_to_timestamp_ms_expr(123, TimeUnit::Microsecond),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(0), None))
ScalarValue::TimestampMillisecond(Some(0), None).lit()
);
assert_eq!(
timestamp_to_timestamp_ms_expr(1230, TimeUnit::Microsecond),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(1), None))
ScalarValue::TimestampMillisecond(Some(1), None).lit()
);
assert_eq!(
timestamp_to_timestamp_ms_expr(123000, TimeUnit::Microsecond),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(123), None))
ScalarValue::TimestampMillisecond(Some(123), None).lit()
);
assert_eq!(
timestamp_to_timestamp_ms_expr(1230, TimeUnit::Nanosecond),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(0), None))
ScalarValue::TimestampMillisecond(Some(0), None).lit()
);
assert_eq!(
timestamp_to_timestamp_ms_expr(123_000_000, TimeUnit::Nanosecond),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(123), None))
ScalarValue::TimestampMillisecond(Some(123), None).lit()
);
}
@@ -405,16 +408,13 @@ mod tests {
};
assert_eq!(
Expr::Column(Column::from_name("ts")).gt(Expr::Literal(ScalarValue::TimestampSecond(
Expr::Column(Column::from_name("ts")).gt(ScalarValue::TimestampSecond(
Some(1599514949),
None
))),
)
.lit()),
converter
.f_up(
Expr::Column(Column::from_name("ts")).gt(Expr::Literal(ScalarValue::Utf8(
Some("2020-09-08T05:42:29+08:00".to_string()),
)))
)
.f_up(Expr::Column(Column::from_name("ts")).gt("2020-09-08T05:42:29+08:00".lit()))
.unwrap()
.data
);
@@ -439,13 +439,9 @@ mod tests {
};
assert_eq!(
Expr::Column(Column::from_name(col_name))
.eq(Expr::Literal(ScalarValue::Boolean(Some(true)))),
Expr::Column(Column::from_name(col_name)).eq(true.lit()),
converter
.f_up(
Expr::Column(Column::from_name(col_name))
.eq(Expr::Literal(ScalarValue::Utf8(Some("true".to_string()))))
)
.f_up(Expr::Column(Column::from_name(col_name)).eq("true".lit()))
.unwrap()
.data
);
@@ -453,40 +449,36 @@ mod tests {
#[test]
fn test_retrieve_type_from_aggr_plan() {
let plan =
LogicalPlanBuilder::values(vec![vec![
Expr::Literal(ScalarValue::Int64(Some(1))),
Expr::Literal(ScalarValue::Float64(Some(1.0))),
Expr::Literal(ScalarValue::TimestampMillisecond(Some(1), None)),
]])
.unwrap()
.filter(Expr::Column(Column::from_name("column3")).gt(Expr::Literal(
ScalarValue::Utf8(Some("1970-01-01 00:00:00+08:00".to_string())),
)))
.unwrap()
.filter(
Expr::Literal(ScalarValue::Utf8(Some(
"1970-01-01 00:00:00+08:00".to_string(),
)))
let plan = LogicalPlanBuilder::values(vec![vec![
ScalarValue::Int64(Some(1)).lit(),
ScalarValue::Float64(Some(1.0)).lit(),
ScalarValue::TimestampMillisecond(Some(1), None).lit(),
]])
.unwrap()
.filter(Expr::Column(Column::from_name("column3")).gt("1970-01-01 00:00:00+08:00".lit()))
.unwrap()
.filter(
"1970-01-01 00:00:00+08:00"
.lit()
.lt_eq(Expr::Column(Column::from_name("column3"))),
)
.unwrap()
.aggregate(
Vec::<Expr>::new(),
vec![Expr::AggregateFunction(
datafusion_expr::expr::AggregateFunction::new_udf(
datafusion::functions_aggregate::count::count_udaf(),
vec![Expr::Column(Column::from_name("column1"))],
false,
None,
None,
None,
),
)],
)
.unwrap()
.build()
.unwrap();
)
.unwrap()
.aggregate(
Vec::<Expr>::new(),
vec![Expr::AggregateFunction(
datafusion_expr::expr::AggregateFunction::new_udf(
datafusion::functions_aggregate::count::count_udaf(),
vec![Expr::Column(Column::from_name("column1"))],
false,
None,
vec![],
None,
),
)],
)
.unwrap()
.build()
.unwrap();
let context = QueryEngineContext::mock();
let transformed_plan = TypeConversionRule
@@ -505,21 +497,18 @@ mod tests {
fn test_reverse_non_ts_type() {
let context = QueryEngineContext::mock();
let plan =
LogicalPlanBuilder::values(vec![vec![Expr::Literal(ScalarValue::Float64(Some(1.0)))]])
.unwrap()
.filter(
Expr::Column(Column::from_name("column1"))
.gt_eq(Expr::Literal(ScalarValue::Utf8(Some("1.2345".to_string())))),
)
.unwrap()
.filter(
Expr::Literal(ScalarValue::Utf8(Some("1.2345".to_string())))
.lt(Expr::Column(Column::from_name("column1"))),
)
.unwrap()
.build()
.unwrap();
let plan = LogicalPlanBuilder::values(vec![vec![1.0f64.lit()]])
.unwrap()
.filter(Expr::Column(Column::from_name("column1")).gt_eq("1.2345".lit()))
.unwrap()
.filter(
"1.2345"
.lit()
.lt(Expr::Column(Column::from_name("column1"))),
)
.unwrap()
.build()
.unwrap();
let transformed_plan = TypeConversionRule
.analyze(plan, &context, &ConfigOptions::default())
.unwrap();

View File

@@ -18,6 +18,7 @@ use std::sync::Arc;
use datafusion::physical_optimizer::PhysicalOptimizerRule;
use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec;
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
use datafusion::physical_plan::coop::CooperativeExec;
use datafusion::physical_plan::filter::FilterExec;
use datafusion::physical_plan::projection::ProjectionExec;
use datafusion::physical_plan::repartition::RepartitionExec;
@@ -27,7 +28,6 @@ use datafusion::physical_plan::ExecutionPlan;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::Result as DataFusionResult;
use datafusion_physical_expr::expressions::Column as PhysicalColumn;
use datafusion_physical_expr::LexOrdering;
use store_api::region_engine::PartitionRange;
use table::table::scan::RegionScanExec;
@@ -89,11 +89,11 @@ impl WindowedSortPhysicalRule {
};
let input_schema = sort_input.schema();
if let Some(first_sort_expr) = sort_exec.expr().first()
&& let Some(column_expr) = first_sort_expr
.expr
.as_any()
.downcast_ref::<PhysicalColumn>()
let first_sort_expr = sort_exec.expr().first();
if let Some(column_expr) = first_sort_expr
.expr
.as_any()
.downcast_ref::<PhysicalColumn>()
&& scanner_info
.time_index
.contains(input_schema.field(column_expr.index()).name())
@@ -101,7 +101,6 @@ impl WindowedSortPhysicalRule {
} else {
return Ok(Transformed::no(plan));
}
let first_sort_expr = sort_exec.expr().first().unwrap().clone();
// PartSortExec is unnecessary if:
// - there is no tag column, and
@@ -120,7 +119,7 @@ impl WindowedSortPhysicalRule {
};
let windowed_sort_exec = WindowedSortExec::try_new(
first_sort_expr,
first_sort_expr.clone(),
sort_exec.fetch(),
scanner_info.partition_ranges,
new_input,
@@ -128,7 +127,7 @@ impl WindowedSortPhysicalRule {
if !preserve_partitioning {
let order_preserving_merge = SortPreservingMergeExec::new(
LexOrdering::new(sort_exec.expr().to_vec()),
sort_exec.expr().clone(),
Arc::new(windowed_sort_exec),
);
return Ok(Transformed {
@@ -168,6 +167,10 @@ fn fetch_partition_range(input: Arc<dyn ExecutionPlan>) -> DataFusionResult<Opti
let mut is_batch_coalesced = false;
input.transform_up(|plan| {
if plan.as_any().is::<CooperativeExec>() {
return Ok(Transformed::no(plan));
}
// Unappliable case, reset the state.
if plan.as_any().is::<RepartitionExec>()
|| plan.as_any().is::<CoalescePartitionsExec>()

View File

@@ -35,7 +35,7 @@ use datafusion::physical_plan::{
DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, TopK,
};
use datafusion_common::{internal_err, DataFusionError};
use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr};
use datafusion_physical_expr::PhysicalSortExpr;
use futures::{Stream, StreamExt};
use itertools::Itertools;
use snafu::location;
@@ -243,11 +243,13 @@ impl PartSortStream {
TopK::try_new(
partition,
sort.schema().clone(),
LexOrdering::new(vec![sort.expression.clone()]),
vec![],
[sort.expression.clone()].into(),
limit,
context.session_config().batch_size(),
context.runtime_env(),
&sort.metrics,
None,
)?,
0,
)
@@ -429,14 +431,14 @@ impl PartSortStream {
let sort_column =
concat(&sort_columns.iter().map(|a| a.as_ref()).collect_vec()).map_err(|e| {
DataFusionError::ArrowError(
e,
Box::new(e),
Some(format!("Fail to concat sort columns at {}", location!())),
)
})?;
let indices = sort_to_indices(&sort_column, opt, self.limit).map_err(|e| {
DataFusionError::ArrowError(
e,
Box::new(e),
Some(format!("Fail to sort to indices at {}", location!())),
)
})?;
@@ -468,7 +470,7 @@ impl PartSortStream {
let full_input = concat_batches(&self.schema, &buffer).map_err(|e| {
DataFusionError::ArrowError(
e,
Box::new(e),
Some(format!(
"Fail to concat input batches when sorting at {}",
location!()
@@ -478,7 +480,7 @@ impl PartSortStream {
let sorted = take_record_batch(&full_input, &indices).map_err(|e| {
DataFusionError::ArrowError(
e,
Box::new(e),
Some(format!(
"Fail to take result record batch when sorting at {}",
location!()
@@ -498,11 +500,13 @@ impl PartSortStream {
let new_top_buffer = TopK::try_new(
self.partition,
self.schema().clone(),
LexOrdering::new(vec![self.expression.clone()]),
vec![],
[self.expression.clone()].into(),
self.limit.unwrap(),
self.context.session_config().batch_size(),
self.context.runtime_env(),
&self.root_metrics,
None,
)?;
let PartSortBuffer::Top(top_k, _) =
std::mem::replace(&mut self.buffer, PartSortBuffer::Top(new_top_buffer, 0))
@@ -532,7 +536,7 @@ impl PartSortStream {
let concat_batch = concat_batches(&self.schema, &results).map_err(|e| {
DataFusionError::ArrowError(
e,
Box::new(e),
Some(format!(
"Fail to concat top k result record batch when sorting at {}",
location!()

View File

@@ -17,7 +17,7 @@ use std::collections::HashSet;
use datafusion::datasource::DefaultTableSource;
use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter};
use datafusion_common::TableReference;
use datafusion_expr::{BinaryExpr, Expr, Join, LogicalPlan, Operator};
use datafusion_expr::{Expr, LogicalPlan};
use session::context::QueryContextRef;
pub use table::metadata::TableType;
use table::table::adapter::DfTableProviderAdapter;
@@ -132,24 +132,7 @@ pub trait ExtractExpr {
impl ExtractExpr for LogicalPlan {
fn expressions_consider_join(&self) -> Vec<Expr> {
match self {
LogicalPlan::Join(Join { on, filter, .. }) => {
// The first part of expr is equi-exprs,
// and the struct of each equi-expr is like `left-expr = right-expr`.
// We only normalize the filter_expr (non equality predicate from ON clause).
on.iter()
.map(|(left, right)| {
Expr::BinaryExpr(BinaryExpr {
left: Box::new(left.clone()),
op: Operator::Eq,
right: Box::new(right.clone()),
})
})
.chain(filter.clone())
.collect()
}
_ => self.expressions(),
}
self.expressions()
}
}

View File

@@ -97,7 +97,15 @@ impl DfLogicalPlanner {
is_tql_cte = true;
}
let df_stmt = stmt.as_ref().try_into().context(SqlSnafu)?;
let mut df_stmt = stmt.as_ref().try_into().context(SqlSnafu)?;
// TODO(LFC): Remove this when Datafusion supports **both** the syntax and implementation of "explain with format".
if let datafusion::sql::parser::Statement::Statement(
box datafusion::sql::sqlparser::ast::Statement::Explain { format, .. },
) = &mut df_stmt
{
format.take();
}
let table_provider = DfTableSourceProvider::new(
self.engine_state.catalog_manager().clone(),
@@ -124,11 +132,8 @@ impl DfLogicalPlanner {
let config_options = self.session_state.config().options();
let parser_options = &config_options.sql_parser;
let parser_options = ParserOptions {
enable_ident_normalization: parser_options.enable_ident_normalization,
parse_float_as_decimal: parser_options.parse_float_as_decimal,
support_varchar_with_length: parser_options.support_varchar_with_length,
enable_options_value_normalization: parser_options.enable_options_value_normalization,
collect_spans: parser_options.collect_spans,
map_string_types_to_utf8view: false,
..parser_options.into()
};
let sql_to_rel = SqlToRel::new_with_options(&context_provider, parser_options);
@@ -183,12 +188,10 @@ impl DfLogicalPlanner {
let config_options = self.session_state.config().options();
let parser_options = &config_options.sql_parser;
let parser_options = ParserOptions {
let parser_options: ParserOptions = ParserOptions {
map_string_types_to_utf8view: false,
enable_ident_normalization: normalize_ident,
parse_float_as_decimal: parser_options.parse_float_as_decimal,
support_varchar_with_length: parser_options.support_varchar_with_length,
enable_options_value_normalization: parser_options.enable_options_value_normalization,
collect_spans: parser_options.collect_spans,
..parser_options.into()
};
let sql_to_rel = SqlToRel::new_with_options(&context_provider, parser_options);

View File

@@ -29,8 +29,8 @@ use crate::promql::error::{
fn build_time_filter(time_index_expr: Expr, start: Timestamp, end: Timestamp) -> Expr {
time_index_expr
.clone()
.gt_eq(Expr::Literal(timestamp_to_scalar_value(start)))
.and(time_index_expr.lt_eq(Expr::Literal(timestamp_to_scalar_value(end))))
.gt_eq(Expr::Literal(timestamp_to_scalar_value(start), None))
.and(time_index_expr.lt_eq(Expr::Literal(timestamp_to_scalar_value(end), None)))
}
fn timestamp_to_scalar_value(timestamp: Timestamp) -> ScalarValue {

View File

@@ -44,9 +44,10 @@ use datafusion::prelude as df_prelude;
use datafusion::prelude::{Column, Expr as DfExpr, JoinType};
use datafusion::scalar::ScalarValue;
use datafusion::sql::TableReference;
use datafusion_common::DFSchema;
use datafusion_common::{DFSchema, NullEquality};
use datafusion_expr::expr::WindowFunctionParams;
use datafusion_expr::utils::conjunction;
use datafusion_expr::{col, lit, ExprSchemable, SortExpr};
use datafusion_expr::{col, lit, ExprSchemable, Literal, SortExpr};
use datatypes::arrow::datatypes::{DataType as ArrowDataType, TimeUnit as ArrowTimeUnit};
use datatypes::data_type::ConcreteDataType;
use itertools::Itertools;
@@ -1299,11 +1300,11 @@ impl PromPlanner {
.field_with_unqualified_name(&matcher.name)
.is_err()
{
DfExpr::Literal(ScalarValue::Utf8(Some(String::new()))).alias(matcher.name)
DfExpr::Literal(ScalarValue::Utf8(Some(String::new())), None).alias(matcher.name)
} else {
DfExpr::Column(Column::from_name(matcher.name))
};
let lit = DfExpr::Literal(ScalarValue::Utf8(Some(matcher.value)));
let lit = DfExpr::Literal(ScalarValue::Utf8(Some(matcher.value)), None);
let expr = match matcher.op {
MatchOp::Equal => col.eq(lit),
MatchOp::NotEqual => col.not_eq(lit),
@@ -1315,17 +1316,13 @@ impl PromPlanner {
DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(col),
op: Operator::RegexMatch,
right: Box::new(DfExpr::Literal(ScalarValue::Utf8(Some(
re.as_str().to_string(),
)))),
right: Box::new(re.as_str().lit()),
})
}
MatchOp::NotRe(re) => DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(col),
op: Operator::RegexNotMatch,
right: Box::new(DfExpr::Literal(ScalarValue::Utf8(Some(
re.as_str().to_string(),
)))),
right: Box::new(re.as_str().lit()),
}),
};
exprs.push(expr);
@@ -1367,16 +1364,20 @@ impl PromPlanner {
if (end - start) / interval > MAX_SCATTER_POINTS || interval <= INTERVAL_1H {
let single_time_range = time_index_expr
.clone()
.gt_eq(DfExpr::Literal(ScalarValue::TimestampMillisecond(
Some(self.ctx.start + offset_duration - self.ctx.lookback_delta - range),
.gt_eq(DfExpr::Literal(
ScalarValue::TimestampMillisecond(
Some(self.ctx.start + offset_duration - self.ctx.lookback_delta - range),
None,
),
None,
)))
.and(
time_index_expr.lt_eq(DfExpr::Literal(ScalarValue::TimestampMillisecond(
))
.and(time_index_expr.lt_eq(DfExpr::Literal(
ScalarValue::TimestampMillisecond(
Some(self.ctx.end + offset_duration + self.ctx.lookback_delta),
None,
))),
);
),
None,
)));
return Ok(Some(single_time_range));
}
@@ -1386,15 +1387,19 @@ impl PromPlanner {
filters.push(
time_index_expr
.clone()
.gt_eq(DfExpr::Literal(ScalarValue::TimestampMillisecond(
Some(timestamp + offset_duration - lookback_delta - range),
.gt_eq(DfExpr::Literal(
ScalarValue::TimestampMillisecond(
Some(timestamp + offset_duration - lookback_delta - range),
None,
),
None,
)))
))
.and(time_index_expr.clone().lt_eq(DfExpr::Literal(
ScalarValue::TimestampMillisecond(
Some(timestamp + offset_duration + lookback_delta),
None,
),
None,
))),
)
}
@@ -1457,6 +1462,7 @@ impl PromPlanner {
table: table_ref.to_quoted_string(),
})?
.clone(),
metadata: None,
})))
.collect::<Vec<_>>();
scan_plan = LogicalPlanBuilder::from(scan_plan)
@@ -1547,7 +1553,7 @@ impl PromPlanner {
self.ctx.interval,
SPECIAL_TIME_FUNCTION.to_string(),
DEFAULT_FIELD_COLUMN.to_string(),
Some(DfExpr::Literal(ScalarValue::Float64(Some(0.0)))),
Some(lit(0.0f64)),
)
.context(DataFusionPlanningSnafu)?,
),
@@ -1700,13 +1706,14 @@ impl PromPlanner {
// 'days',
// (date_trunc('month', <TIME INDEX>::date) + interval '1 month - 1 day')
// );
let day_lit_expr = DfExpr::Literal(ScalarValue::Utf8(Some("day".to_string())));
let month_lit_expr = DfExpr::Literal(ScalarValue::Utf8(Some("month".to_string())));
let day_lit_expr = "day".lit();
let month_lit_expr = "month".lit();
let interval_1month_lit_expr =
DfExpr::Literal(ScalarValue::IntervalYearMonth(Some(1)));
let interval_1day_lit_expr = DfExpr::Literal(ScalarValue::IntervalDayTime(Some(
IntervalDayTime::new(1, 0),
)));
DfExpr::Literal(ScalarValue::IntervalYearMonth(Some(1)), None);
let interval_1day_lit_expr = DfExpr::Literal(
ScalarValue::IntervalDayTime(Some(IntervalDayTime::new(1, 0))),
None,
);
let the_1month_minus_1day_expr = DfExpr::BinaryExpr(BinaryExpr {
left: Box::new(interval_1month_lit_expr),
op: Operator::Minus,
@@ -1794,7 +1801,7 @@ impl PromPlanner {
}
"round" => {
if other_input_exprs.is_empty() {
other_input_exprs.push_front(DfExpr::Literal(ScalarValue::Float64(Some(0.0))));
other_input_exprs.push_front(0.0f64.lit());
}
ScalarFunc::DataFusionUdf(Arc::new(Round::scalar_udf()))
}
@@ -1949,7 +1956,7 @@ impl PromPlanner {
) -> Result<Option<(DfExpr, String)>> {
// label_replace(vector, dst_label, replacement, src_label, regex)
let dst_label = match other_input_exprs.pop_front() {
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)))) => d,
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)), _)) => d,
other => UnexpectedPlanExprSnafu {
desc: format!("expected dst_label string literal, but found {:?}", other),
}
@@ -1959,14 +1966,14 @@ impl PromPlanner {
// Validate the destination label name
Self::validate_label_name(&dst_label)?;
let replacement = match other_input_exprs.pop_front() {
Some(DfExpr::Literal(ScalarValue::Utf8(Some(r)))) => r,
Some(DfExpr::Literal(ScalarValue::Utf8(Some(r)), _)) => r,
other => UnexpectedPlanExprSnafu {
desc: format!("expected replacement string literal, but found {:?}", other),
}
.fail()?,
};
let src_label = match other_input_exprs.pop_front() {
Some(DfExpr::Literal(ScalarValue::Utf8(Some(s)))) => s,
Some(DfExpr::Literal(ScalarValue::Utf8(Some(s)), None)) => s,
other => UnexpectedPlanExprSnafu {
desc: format!("expected src_label string literal, but found {:?}", other),
}
@@ -1974,7 +1981,7 @@ impl PromPlanner {
};
let regex = match other_input_exprs.pop_front() {
Some(DfExpr::Literal(ScalarValue::Utf8(Some(r)))) => r,
Some(DfExpr::Literal(ScalarValue::Utf8(Some(r)), None)) => r,
other => UnexpectedPlanExprSnafu {
desc: format!("expected regex string literal, but found {:?}", other),
}
@@ -2004,7 +2011,7 @@ impl PromPlanner {
// the replacement is not empty, always adds dst_label with replacement value.
return Ok(Some((
// alias literal `replacement` as dst_label
DfExpr::Literal(ScalarValue::Utf8(Some(replacement))).alias(&dst_label),
lit(replacement).alias(&dst_label),
dst_label,
)));
}
@@ -2025,12 +2032,12 @@ impl PromPlanner {
// regexp_replace(src_label, regex, replacement)
let args = vec![
if src_label.is_empty() {
DfExpr::Literal(ScalarValue::Utf8(Some(String::new())))
DfExpr::Literal(ScalarValue::Utf8(Some(String::new())), None)
} else {
DfExpr::Column(Column::from_name(src_label))
},
DfExpr::Literal(ScalarValue::Utf8(Some(regex))),
DfExpr::Literal(ScalarValue::Utf8(Some(replacement))),
DfExpr::Literal(ScalarValue::Utf8(Some(regex)), None),
DfExpr::Literal(ScalarValue::Utf8(Some(replacement)), None),
];
Ok(Some((
@@ -2052,14 +2059,14 @@ impl PromPlanner {
// label_join(vector, dst_label, separator, src_label_1, src_label_2, ...)
let dst_label = match other_input_exprs.pop_front() {
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)))) => d,
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)), _)) => d,
other => UnexpectedPlanExprSnafu {
desc: format!("expected dst_label string literal, but found {:?}", other),
}
.fail()?,
};
let separator = match other_input_exprs.pop_front() {
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)))) => d,
Some(DfExpr::Literal(ScalarValue::Utf8(Some(d)), _)) => d,
other => UnexpectedPlanExprSnafu {
desc: format!("expected separator string literal, but found {:?}", other),
}
@@ -2080,15 +2087,15 @@ impl PromPlanner {
.map(|expr| {
// Cast source label into column or null literal
match expr {
DfExpr::Literal(ScalarValue::Utf8(Some(label))) => {
DfExpr::Literal(ScalarValue::Utf8(Some(label)), None) => {
if label.is_empty() {
Ok(DfExpr::Literal(ScalarValue::Null))
Ok(DfExpr::Literal(ScalarValue::Null, None))
} else if available_columns.contains(label.as_str()) {
// Label exists in the table schema
Ok(DfExpr::Column(Column::from_name(label)))
} else {
// Label doesn't exist, treat as empty string (null)
Ok(DfExpr::Literal(ScalarValue::Null))
Ok(DfExpr::Literal(ScalarValue::Null, None))
}
}
other => UnexpectedPlanExprSnafu {
@@ -2116,7 +2123,7 @@ impl PromPlanner {
// concat_ws(separator, src_label_1, src_label_2, ...) as dst_label
let mut args = Vec::with_capacity(1 + src_labels.len());
args.push(DfExpr::Literal(ScalarValue::Utf8(Some(separator))));
args.push(DfExpr::Literal(ScalarValue::Utf8(Some(separator)), None));
args.extend(src_labels);
Ok((
@@ -2187,7 +2194,7 @@ impl PromPlanner {
tags.iter()
.map(|col| match col {
DfExpr::Literal(ScalarValue::Utf8(Some(label))) => {
DfExpr::Literal(ScalarValue::Utf8(Some(label)), _) => {
Ok(DfExpr::Column(Column::from_name(label)).sort(asc, false))
}
other => UnexpectedPlanExprSnafu {
@@ -2400,14 +2407,17 @@ impl PromPlanner {
// Try to ensure the relative stability of the output results.
sort_exprs.extend(tag_sort_exprs.clone());
DfExpr::WindowFunction(WindowFunction {
DfExpr::WindowFunction(Box::new(WindowFunction {
fun: WindowFunctionDefinition::WindowUDF(Arc::new(RowNumber::new().into())),
args: vec![],
partition_by: group_exprs.clone(),
order_by: sort_exprs,
window_frame: WindowFrame::new(Some(true)),
null_treatment: None,
})
params: WindowFunctionParams {
args: vec![],
partition_by: group_exprs.clone(),
order_by: sort_exprs,
window_frame: WindowFrame::new(Some(true)),
null_treatment: None,
distinct: false,
},
}))
})
.collect();
@@ -2601,7 +2611,7 @@ impl PromPlanner {
.with_context(|| ValueNotFoundSnafu {
table: self.ctx.table_name.clone().unwrap_or_default(),
})?;
let first_value_expr = first_value(first_field_expr, None);
let first_value_expr = first_value(first_field_expr, vec![]);
let ordered_aggregated_input = LogicalPlanBuilder::from(input)
.aggregate(
@@ -2647,14 +2657,8 @@ impl PromPlanner {
/// `None` if the input is not a literal expression.
fn try_build_literal_expr(expr: &PromExpr) -> Option<DfExpr> {
match expr {
PromExpr::NumberLiteral(NumberLiteral { val }) => {
let scalar_value = ScalarValue::Float64(Some(*val));
Some(DfExpr::Literal(scalar_value))
}
PromExpr::StringLiteral(StringLiteral { val }) => {
let scalar_value = ScalarValue::Utf8(Some(val.to_string()));
Some(DfExpr::Literal(scalar_value))
}
PromExpr::NumberLiteral(NumberLiteral { val }) => Some(val.lit()),
PromExpr::StringLiteral(StringLiteral { val }) => Some(val.lit()),
PromExpr::VectorSelector(_)
| PromExpr::MatrixSelector(_)
| PromExpr::Extension(_)
@@ -2850,7 +2854,7 @@ impl PromPlanner {
.collect::<Vec<_>>(),
),
None,
true,
NullEquality::NullEqualsNull,
)
.context(DataFusionPlanningSnafu)?
.build()
@@ -2986,7 +2990,7 @@ impl PromPlanner {
JoinType::LeftSemi,
(join_keys.clone(), join_keys),
None,
true,
NullEquality::NullEqualsNull,
)
.context(DataFusionPlanningSnafu)?
.build()
@@ -2999,7 +3003,7 @@ impl PromPlanner {
JoinType::LeftAnti,
(join_keys.clone(), join_keys),
None,
true,
NullEquality::NullEqualsNull,
)
.context(DataFusionPlanningSnafu)?
.build()
@@ -3105,7 +3109,7 @@ impl PromPlanner {
// step 1: align schema using project, fill non-exist columns with null
let left_proj_exprs = all_columns.iter().map(|col| {
if tags_not_in_left.contains(col) {
DfExpr::Literal(ScalarValue::Utf8(None)).alias(col.to_string())
DfExpr::Literal(ScalarValue::Utf8(None), None).alias(col.to_string())
} else {
DfExpr::Column(Column::new(None::<String>, col))
}
@@ -3137,7 +3141,7 @@ impl PromPlanner {
right_field_col,
))
} else if tags_not_in_right.contains(col) {
DfExpr::Literal(ScalarValue::Utf8(None)).alias(col.to_string())
DfExpr::Literal(ScalarValue::Utf8(None), None).alias(col.to_string())
} else {
DfExpr::Column(Column::new(None::<String>, col))
}
@@ -3286,7 +3290,6 @@ impl PromPlanner {
/// Generate an expr like `date_part("hour", <TIME_INDEX>)`. Caller should ensure the
/// time index column in context is set
fn date_part_on_time_index(&self, date_part: &str) -> Result<DfExpr> {
let lit_expr = DfExpr::Literal(ScalarValue::Utf8(Some(date_part.to_string())));
let input_expr = datafusion::logical_expr::col(
self.ctx
.time_index_column
@@ -3298,7 +3301,7 @@ impl PromPlanner {
);
let fn_expr = DfExpr::ScalarFunction(ScalarFunction {
func: datafusion_functions::datetime::date_part(),
args: vec![lit_expr, input_expr],
args: vec![date_part.lit(), input_expr],
});
Ok(fn_expr)
}
@@ -3900,7 +3903,7 @@ mod test {
assert_eq!(plan.display_indent_schema().to_string(), expected);
}
async fn indie_query_plan_compare(query: &str, expected: String) {
async fn indie_query_plan_compare<T: AsRef<str>>(query: &str, expected: T) {
let prom_expr = parser::parse(query).unwrap();
let eval_stmt = EvalStmt {
expr: prom_expr,
@@ -3929,7 +3932,7 @@ mod test {
.await
.unwrap();
assert_eq!(plan.display_indent_schema().to_string(), expected);
assert_eq!(plan.display_indent_schema().to_string(), expected.as_ref());
}
#[tokio::test]
@@ -3950,8 +3953,8 @@ mod test {
#[tokio::test]
async fn binary_op_literal_literal() {
let query = r#"1 + 1"#;
let expected = String::from("EmptyMetric: range=[0..100000000], interval=[5000] [time:Timestamp(Millisecond, None), value:Float64;N]");
let expected = r#"EmptyMetric: range=[0..100000000], interval=[5000] [time:Timestamp(Millisecond, None), value:Float64;N]
TableScan: dummy [time:Timestamp(Millisecond, None), value:Float64;N]"#;
indie_query_plan_compare(query, expected).await;
}
@@ -4950,21 +4953,22 @@ Filter: up.field_0 IS NOT NULL [timestamp:Timestamp(Millisecond, None), field_0:
PromPlanner::stmt_to_plan(table_provider, &eval_stmt, &build_query_engine_state())
.await
.unwrap();
let expected = "UnionDistinctOn: on col=[[\"job\"]], ts_col=[greptime_timestamp] [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8, sum(metric_exists.greptime_value):Float64;N]\
\n SubqueryAlias: metric_exists [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8, sum(metric_exists.greptime_value):Float64;N]\
\n Projection: metric_exists.greptime_timestamp, metric_exists.job, sum(metric_exists.greptime_value) [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8, sum(metric_exists.greptime_value):Float64;N]\
\n Sort: metric_exists.job ASC NULLS LAST, metric_exists.greptime_timestamp ASC NULLS LAST [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), sum(metric_exists.greptime_value):Float64;N]\
\n Aggregate: groupBy=[[metric_exists.job, metric_exists.greptime_timestamp]], aggr=[[sum(metric_exists.greptime_value)]] [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), sum(metric_exists.greptime_value):Float64;N]\
\n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[greptime_timestamp] [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]\
\n PromSeriesDivide: tags=[\"job\"] [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]\
\n Sort: metric_exists.job ASC NULLS FIRST, metric_exists.greptime_timestamp ASC NULLS FIRST [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]\
\n Filter: metric_exists.greptime_timestamp >= TimestampMillisecond(-1000, None) AND metric_exists.greptime_timestamp <= TimestampMillisecond(100001000, None) [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]\
\n TableScan: metric_exists [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]\
\n SubqueryAlias: [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8;N, sum(.value):Float64;N]\
\n Projection: .time AS greptime_timestamp, Utf8(NULL) AS job, sum(.value) [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8;N, sum(.value):Float64;N]\
\n Sort: .time ASC NULLS LAST [time:Timestamp(Millisecond, None), sum(.value):Float64;N]\
\n Aggregate: groupBy=[[.time]], aggr=[[sum(.value)]] [time:Timestamp(Millisecond, None), sum(.value):Float64;N]\
\n EmptyMetric: range=[0..-1], interval=[5000] [time:Timestamp(Millisecond, None), value:Float64;N]";
let expected = r#"UnionDistinctOn: on col=[["job"]], ts_col=[greptime_timestamp] [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8, sum(metric_exists.greptime_value):Float64;N]
SubqueryAlias: metric_exists [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8, sum(metric_exists.greptime_value):Float64;N]
Projection: metric_exists.greptime_timestamp, metric_exists.job, sum(metric_exists.greptime_value) [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8, sum(metric_exists.greptime_value):Float64;N]
Sort: metric_exists.job ASC NULLS LAST, metric_exists.greptime_timestamp ASC NULLS LAST [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), sum(metric_exists.greptime_value):Float64;N]
Aggregate: groupBy=[[metric_exists.job, metric_exists.greptime_timestamp]], aggr=[[sum(metric_exists.greptime_value)]] [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), sum(metric_exists.greptime_value):Float64;N]
PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[greptime_timestamp] [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]
PromSeriesDivide: tags=["job"] [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]
Sort: metric_exists.job ASC NULLS FIRST, metric_exists.greptime_timestamp ASC NULLS FIRST [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]
Filter: metric_exists.greptime_timestamp >= TimestampMillisecond(-1000, None) AND metric_exists.greptime_timestamp <= TimestampMillisecond(100001000, None) [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]
TableScan: metric_exists [job:Utf8, greptime_timestamp:Timestamp(Millisecond, None), greptime_value:Float64;N]
SubqueryAlias: [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8;N, sum(.value):Float64;N]
Projection: .time AS greptime_timestamp, Utf8(NULL) AS job, sum(.value) [greptime_timestamp:Timestamp(Millisecond, None), job:Utf8;N, sum(.value):Float64;N]
Sort: .time ASC NULLS LAST [time:Timestamp(Millisecond, None), sum(.value):Float64;N]
Aggregate: groupBy=[[.time]], aggr=[[sum(.value)]] [time:Timestamp(Millisecond, None), sum(.value):Float64;N]
EmptyMetric: range=[0..-1], interval=[5000] [time:Timestamp(Millisecond, None), value:Float64;N]
TableScan: dummy [time:Timestamp(Millisecond, None), value:Float64;N]"#;
assert_eq!(plan.display_indent_schema().to_string(), expected);
}

Some files were not shown because too many files have changed in this diff Show More