diff --git a/Cargo.toml b/Cargo.toml index e1e3aca771539..0a7184ad2d99d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -169,3 +169,4 @@ large_futures = "warn" [workspace.lints.rust] unexpected_cfgs = { level = "warn", check-cfg = ["cfg(tarpaulin)"] } +unused_qualifications = "deny" diff --git a/benchmarks/README.md b/benchmarks/README.md index afaf28bb75769..a9aa1afb97a1c 100644 --- a/benchmarks/README.md +++ b/benchmarks/README.md @@ -330,6 +330,16 @@ steps. The tests sort the entire dataset using several different sort orders. +## IMDB + +Run Join Order Benchmark (JOB) on IMDB dataset. + +The Internet Movie Database (IMDB) dataset contains real-world movie data. Unlike synthetic datasets like TPCH, which assume uniform data distribution and uncorrelated columns, the IMDB dataset includes skewed data and correlated columns (which are common for real dataset), making it more suitable for testing query optimizers, particularly for cardinality estimation. + +This benchmark is derived from [Join Order Benchmark](https://github.com/gregrahn/join-order-benchmark). + +See paper [How Good Are Query Optimizers, Really](http://www.vldb.org/pvldb/vol9/p204-leis.pdf) for more details. + ## TPCH Run the tpch benchmark. @@ -342,6 +352,34 @@ This benchmarks is derived from the [TPC-H][1] version [2]: https://github.com/databricks/tpch-dbgen.git, [2.17.1]: https://www.tpc.org/tpc_documents_current_versions/pdf/tpc-h_v2.17.1.pdf +## External Aggregation + +Run the benchmark for aggregations with limited memory. + +When the memory limit is exceeded, the aggregation intermediate results will be spilled to disk, and finally read back with sort-merge. + +External aggregation benchmarks run several aggregation queries with different memory limits, on TPCH `lineitem` table. Queries can be found in [`external_aggr.rs`](src/bin/external_aggr.rs). + +This benchmark is inspired by [DuckDB's external aggregation paper](https://hannes.muehleisen.org/publications/icde2024-out-of-core-kuiper-boncz-muehleisen.pdf), specifically Section VI. + +### External Aggregation Example Runs +1. Run all queries with predefined memory limits: +```bash +# Under 'benchmarks/' directory +cargo run --release --bin external_aggr -- benchmark -n 4 --iterations 3 -p '....../data/tpch_sf1' -o '/tmp/aggr.json' +``` + +2. Run a query with specific memory limit: +```bash +cargo run --release --bin external_aggr -- benchmark -n 4 --iterations 3 -p '....../data/tpch_sf1' -o '/tmp/aggr.json' --query 1 --memory-limit 30M +``` + +3. Run all queries with `bench.sh` script: +```bash +./bench.sh data external_aggr +./bench.sh run external_aggr +``` + # Older Benchmarks diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 70faa9ef2b737..47c5d1261605b 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -78,6 +78,7 @@ sort: Benchmark of sorting speed clickbench_1: ClickBench queries against a single parquet file clickbench_partitioned: ClickBench queries against a partitioned (100 files) parquet clickbench_extended: ClickBench \"inspired\" queries against a single parquet (DataFusion specific) +external_aggr: External aggregation benchmark ********** * Supported Configuration (Environment Variables) @@ -170,6 +171,10 @@ main() { imdb) data_imdb ;; + external_aggr) + # same data as for tpch + data_tpch "1" + ;; *) echo "Error: unknown benchmark '$BENCHMARK' for data generation" usage @@ -212,6 +217,7 @@ main() { run_clickbench_partitioned run_clickbench_extended run_imdb + run_external_aggr ;; tpch) run_tpch "1" @@ -243,6 +249,9 @@ main() { imdb) run_imdb ;; + external_aggr) + run_external_aggr + ;; *) echo "Error: unknown benchmark '$BENCHMARK' for run" usage @@ -357,7 +366,7 @@ run_parquet() { RESULTS_FILE="${RESULTS_DIR}/parquet.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running parquet filter benchmark..." - $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } # Runs the sort benchmark @@ -365,7 +374,7 @@ run_sort() { RESULTS_FILE="${RESULTS_DIR}/sort.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running sort benchmark..." - $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } @@ -524,7 +533,21 @@ run_imdb() { $CARGO_COMMAND --bin imdb -- benchmark datafusion --iterations 5 --path "${IMDB_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format parquet -o "${RESULTS_FILE}" } - +# Runs the external aggregation benchmark +run_external_aggr() { + # Use TPC-H SF1 dataset + TPCH_DIR="${DATA_DIR}/tpch_sf1" + RESULTS_FILE="${RESULTS_DIR}/external_aggr.json" + echo "RESULTS_FILE: ${RESULTS_FILE}" + echo "Running external aggregation benchmark..." + + # Only parquet is supported. + # Since per-operator memory limit is calculated as (total-memory-limit / + # number-of-partitions), and by default `--partitions` is set to number of + # CPU cores, we set a constant number of partitions to prevent this + # benchmark to fail on some machines. + $CARGO_COMMAND --bin external_aggr -- benchmark --partitions 4 --iterations 5 --path "${TPCH_DIR}" -o "${RESULTS_FILE}" +} compare_benchmarks() { diff --git a/benchmarks/src/bin/external_aggr.rs b/benchmarks/src/bin/external_aggr.rs new file mode 100644 index 0000000000000..1bc74e22ccfae --- /dev/null +++ b/benchmarks/src/bin/external_aggr.rs @@ -0,0 +1,390 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! external_aggr binary entrypoint + +use std::collections::HashMap; +use std::path::PathBuf; +use std::sync::Arc; +use std::sync::OnceLock; +use structopt::StructOpt; + +use arrow::record_batch::RecordBatch; +use arrow::util::pretty; +use datafusion::datasource::file_format::parquet::ParquetFormat; +use datafusion::datasource::listing::{ + ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, +}; +use datafusion::datasource::{MemTable, TableProvider}; +use datafusion::error::Result; +use datafusion::execution::memory_pool::FairSpillPool; +use datafusion::execution::memory_pool::{human_readable_size, units}; +use datafusion::execution::runtime_env::RuntimeConfig; +use datafusion::physical_plan::display::DisplayableExecutionPlan; +use datafusion::physical_plan::{collect, displayable}; +use datafusion::prelude::*; +use datafusion_benchmarks::util::{BenchmarkRun, CommonOpt}; +use datafusion_common::instant::Instant; +use datafusion_common::{exec_datafusion_err, exec_err, DEFAULT_PARQUET_EXTENSION}; + +#[derive(Debug, StructOpt)] +#[structopt( + name = "datafusion-external-aggregation", + about = "DataFusion external aggregation benchmark" +)] +enum ExternalAggrOpt { + Benchmark(ExternalAggrConfig), +} + +#[derive(Debug, StructOpt)] +struct ExternalAggrConfig { + /// Query number. If not specified, runs all queries + #[structopt(short, long)] + query: Option, + + /// Memory limit (e.g. '100M', '1.5G'). If not specified, run all pre-defined memory limits for given query. + #[structopt(long)] + memory_limit: Option, + + /// Common options + #[structopt(flatten)] + common: CommonOpt, + + /// Path to data files (lineitem). Only parquet format is supported + #[structopt(parse(from_os_str), required = true, short = "p", long = "path")] + path: PathBuf, + + /// Load the data into a MemTable before executing the query + #[structopt(short = "m", long = "mem-table")] + mem_table: bool, + + /// Path to JSON benchmark result to be compare using `compare.py` + #[structopt(parse(from_os_str), short = "o", long = "output")] + output_path: Option, +} + +struct QueryResult { + elapsed: std::time::Duration, + row_count: usize, +} + +/// Query Memory Limits +/// Map query id to predefined memory limits +/// +/// Q1 requires 36MiB for aggregation +/// Memory limits to run: 64MiB, 32MiB, 16MiB +/// Q2 requires 250MiB for aggregation +/// Memory limits to run: 512MiB, 256MiB, 128MiB, 64MiB, 32MiB +static QUERY_MEMORY_LIMITS: OnceLock>> = OnceLock::new(); + +impl ExternalAggrConfig { + const AGGR_TABLES: [&'static str; 1] = ["lineitem"]; + const AGGR_QUERIES: [&'static str; 2] = [ + // Q1: Output size is ~25% of lineitem table + r#" + SELECT count(*) + FROM ( + SELECT DISTINCT l_orderkey + FROM lineitem + ) + "#, + // Q2: Output size is ~99% of lineitem table + r#" + SELECT count(*) + FROM ( + SELECT DISTINCT l_orderkey, l_suppkey + FROM lineitem + ) + "#, + ]; + + fn init_query_memory_limits() -> &'static HashMap> { + use units::*; + QUERY_MEMORY_LIMITS.get_or_init(|| { + let mut map = HashMap::new(); + map.insert(1, vec![64 * MB, 32 * MB, 16 * MB]); + map.insert(2, vec![512 * MB, 256 * MB, 128 * MB, 64 * MB, 32 * MB]); + map + }) + } + + /// If `--query` and `--memory-limit` is not speicified, run all queries + /// with pre-configured memory limits + /// If only `--query` is specified, run the query with all memory limits + /// for this query + /// If both `--query` and `--memory-limit` are specified, run the query + /// with the specified memory limit + pub async fn run(&self) -> Result<()> { + let mut benchmark_run = BenchmarkRun::new(); + + let memory_limit = match &self.memory_limit { + Some(limit) => Some(Self::parse_memory_limit(limit)?), + None => None, + }; + + let query_range = match self.query { + Some(query_id) => query_id..=query_id, + None => 1..=Self::AGGR_QUERIES.len(), + }; + + // Each element is (query_id, memory_limit) + // e.g. [(1, 64_000), (1, 32_000)...] means first run Q1 with 64KiB + // memory limit, next run Q1 with 32KiB memory limit, etc. + let mut query_executions = vec![]; + // Setup `query_executions` + for query_id in query_range { + if query_id > Self::AGGR_QUERIES.len() { + return exec_err!( + "Invalid '--query'(query number) {} for external aggregation benchmark.", + query_id + ); + } + + match memory_limit { + Some(limit) => { + query_executions.push((query_id, limit)); + } + None => { + let memory_limits_table = Self::init_query_memory_limits(); + let memory_limits = memory_limits_table.get(&query_id).unwrap(); + for limit in memory_limits { + query_executions.push((query_id, *limit)); + } + } + } + } + + for (query_id, mem_limit) in query_executions { + benchmark_run.start_new_case(&format!( + "{query_id}({})", + human_readable_size(mem_limit as usize) + )); + + let query_results = self.benchmark_query(query_id, mem_limit).await?; + for iter in query_results { + benchmark_run.write_iter(iter.elapsed, iter.row_count); + } + } + + benchmark_run.maybe_write_json(self.output_path.as_ref())?; + + Ok(()) + } + + /// Benchmark query `query_id` in `AGGR_QUERIES` + async fn benchmark_query( + &self, + query_id: usize, + mem_limit: u64, + ) -> Result> { + let query_name = + format!("Q{query_id}({})", human_readable_size(mem_limit as usize)); + let mut config = self.common.config(); + config + .options_mut() + .execution + .parquet + .schema_force_view_types = self.common.force_view_types; + let runtime_config = RuntimeConfig::new() + .with_memory_pool(Arc::new(FairSpillPool::new(mem_limit as usize))) + .build_arc()?; + let ctx = SessionContext::new_with_config_rt(config, runtime_config); + + // register tables + self.register_tables(&ctx).await?; + + let mut millis = vec![]; + // run benchmark + let mut query_results = vec![]; + for i in 0..self.iterations() { + let start = Instant::now(); + + let query_idx = query_id - 1; // 1-indexed -> 0-indexed + let sql = Self::AGGR_QUERIES[query_idx]; + + let result = self.execute_query(&ctx, sql).await?; + + let elapsed = start.elapsed(); //.as_secs_f64() * 1000.0; + let ms = elapsed.as_secs_f64() * 1000.0; + millis.push(ms); + + let row_count = result.iter().map(|b| b.num_rows()).sum(); + println!( + "{query_name} iteration {i} took {ms:.1} ms and returned {row_count} rows" + ); + query_results.push(QueryResult { elapsed, row_count }); + } + + let avg = millis.iter().sum::() / millis.len() as f64; + println!("{query_name} avg time: {avg:.2} ms"); + + Ok(query_results) + } + + async fn register_tables(&self, ctx: &SessionContext) -> Result<()> { + for table in Self::AGGR_TABLES { + let table_provider = { self.get_table(ctx, table).await? }; + + if self.mem_table { + println!("Loading table '{table}' into memory"); + let start = Instant::now(); + let memtable = + MemTable::load(table_provider, Some(self.partitions()), &ctx.state()) + .await?; + println!( + "Loaded table '{}' into memory in {} ms", + table, + start.elapsed().as_millis() + ); + ctx.register_table(table, Arc::new(memtable))?; + } else { + ctx.register_table(table, table_provider)?; + } + } + Ok(()) + } + + async fn execute_query( + &self, + ctx: &SessionContext, + sql: &str, + ) -> Result> { + let debug = self.common.debug; + let plan = ctx.sql(sql).await?; + let (state, plan) = plan.into_parts(); + + if debug { + println!("=== Logical plan ===\n{plan}\n"); + } + + let plan = state.optimize(&plan)?; + if debug { + println!("=== Optimized logical plan ===\n{plan}\n"); + } + let physical_plan = state.create_physical_plan(&plan).await?; + if debug { + println!( + "=== Physical plan ===\n{}\n", + displayable(physical_plan.as_ref()).indent(true) + ); + } + let result = collect(physical_plan.clone(), state.task_ctx()).await?; + if debug { + println!( + "=== Physical plan with metrics ===\n{}\n", + DisplayableExecutionPlan::with_metrics(physical_plan.as_ref()) + .indent(true) + ); + if !result.is_empty() { + // do not call print_batches if there are no batches as the result is confusing + // and makes it look like there is a batch with no columns + pretty::print_batches(&result)?; + } + } + Ok(result) + } + + async fn get_table( + &self, + ctx: &SessionContext, + table: &str, + ) -> Result> { + let path = self.path.to_str().unwrap(); + + // Obtain a snapshot of the SessionState + let state = ctx.state(); + let path = format!("{path}/{table}"); + let format = Arc::new( + ParquetFormat::default() + .with_options(ctx.state().table_options().parquet.clone()), + ); + let extension = DEFAULT_PARQUET_EXTENSION; + + let options = ListingOptions::new(format) + .with_file_extension(extension) + .with_collect_stat(state.config().collect_statistics()); + + let table_path = ListingTableUrl::parse(path)?; + let config = ListingTableConfig::new(table_path).with_listing_options(options); + let config = config.infer_schema(&state).await?; + + Ok(Arc::new(ListingTable::try_new(config)?)) + } + + fn iterations(&self) -> usize { + self.common.iterations + } + + fn partitions(&self) -> usize { + self.common.partitions.unwrap_or(num_cpus::get()) + } + + /// Parse memory limit from string to number of bytes + /// e.g. '1.5G', '100M' -> 1572864 + fn parse_memory_limit(limit: &str) -> Result { + let (number, unit) = limit.split_at(limit.len() - 1); + let number: f64 = number.parse().map_err(|_| { + exec_datafusion_err!("Failed to parse number from memory limit '{}'", limit) + })?; + + match unit { + "K" => Ok((number * 1024.0) as u64), + "M" => Ok((number * 1024.0 * 1024.0) as u64), + "G" => Ok((number * 1024.0 * 1024.0 * 1024.0) as u64), + _ => exec_err!("Unsupported unit '{}' in memory limit '{}'", unit, limit), + } + } +} + +#[tokio::main] +pub async fn main() -> Result<()> { + env_logger::init(); + + match ExternalAggrOpt::from_args() { + ExternalAggrOpt::Benchmark(opt) => opt.run().await?, + } + + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_parse_memory_limit_all() { + // Test valid inputs + assert_eq!( + ExternalAggrConfig::parse_memory_limit("100K").unwrap(), + 102400 + ); + assert_eq!( + ExternalAggrConfig::parse_memory_limit("1.5M").unwrap(), + 1572864 + ); + assert_eq!( + ExternalAggrConfig::parse_memory_limit("2G").unwrap(), + 2147483648 + ); + + // Test invalid unit + assert!(ExternalAggrConfig::parse_memory_limit("500X").is_err()); + + // Test invalid number + assert!(ExternalAggrConfig::parse_memory_limit("abcM").is_err()); + } +} diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 1259f90d64496..414596bdc6787 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -193,7 +193,7 @@ impl Accumulator for GeometricMean { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } @@ -394,8 +394,8 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { } fn size(&self) -> usize { - self.counts.capacity() * std::mem::size_of::() - + self.prods.capacity() * std::mem::size_of::() + self.counts.capacity() * size_of::() + + self.prods.capacity() * size_of::() } } diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 0f7748b133650..7440e592962b3 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -110,7 +110,7 @@ struct CustomDataSourceInner { } impl Debug for CustomDataSource { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { f.write_str("custom_db") } } @@ -220,7 +220,7 @@ impl CustomExec { } impl DisplayAs for CustomExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> std::fmt::Result { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { write!(f, "CustomExec") } } diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index b85127d42f71e..95168597ebaaf 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -74,10 +74,7 @@ impl FileFormat for TSVFileFormat { "tsv".to_string() } - fn get_ext_with_compression( - &self, - c: &FileCompressionType, - ) -> datafusion::error::Result { + fn get_ext_with_compression(&self, c: &FileCompressionType) -> Result { if c == &FileCompressionType::UNCOMPRESSED { Ok("tsv".to_string()) } else { diff --git a/datafusion-examples/examples/flight/flight_server.rs b/datafusion-examples/examples/flight/flight_server.rs index f9d1b8029f04b..cc5f43746ddfb 100644 --- a/datafusion-examples/examples/flight/flight_server.rs +++ b/datafusion-examples/examples/flight/flight_server.rs @@ -105,7 +105,7 @@ impl FlightService for FlightServiceImpl { } // add an initial FlightData message that sends schema - let options = datafusion::arrow::ipc::writer::IpcWriteOptions::default(); + let options = arrow::ipc::writer::IpcWriteOptions::default(); let schema_flight_data = SchemaAsIpc::new(&schema, &options); let mut flights = vec![FlightData::from(schema_flight_data)]; diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index f57b3bf604048..b42f25437d772 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -121,7 +121,7 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { &self.name } - fn signature(&self) -> &datafusion_expr::Signature { + fn signature(&self) -> &Signature { &self.signature } diff --git a/datafusion-examples/examples/simple_udaf.rs b/datafusion-examples/examples/simple_udaf.rs index 140fc0d3572da..ef97bf9763b0f 100644 --- a/datafusion-examples/examples/simple_udaf.rs +++ b/datafusion-examples/examples/simple_udaf.rs @@ -131,7 +131,7 @@ impl Accumulator for GeometricMean { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } diff --git a/datafusion-examples/examples/simplify_udaf_expression.rs b/datafusion-examples/examples/simplify_udaf_expression.rs index aedc511c62fef..52a27317e3c3d 100644 --- a/datafusion-examples/examples/simplify_udaf_expression.rs +++ b/datafusion-examples/examples/simplify_udaf_expression.rs @@ -70,7 +70,7 @@ impl AggregateUDFImpl for BetterAvgUdaf { unimplemented!("should not be invoked") } - fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { unimplemented!("should not be invoked") } @@ -90,8 +90,7 @@ impl AggregateUDFImpl for BetterAvgUdaf { fn simplify(&self) -> Option { // as an example for this functionality we replace UDF function // with build-in aggregate function to illustrate the use - let simplify = |aggregate_function: datafusion_expr::expr::AggregateFunction, - _: &dyn SimplifyInfo| { + let simplify = |aggregate_function: AggregateFunction, _: &dyn SimplifyInfo| { Ok(Expr::AggregateFunction(AggregateFunction::new_udf( avg_udaf(), // yes it is the same Avg, `BetterAvgUdaf` was just a diff --git a/datafusion-examples/examples/simplify_udwf_expression.rs b/datafusion-examples/examples/simplify_udwf_expression.rs index d95f1147bc376..117063df4e0d8 100644 --- a/datafusion-examples/examples/simplify_udwf_expression.rs +++ b/datafusion-examples/examples/simplify_udwf_expression.rs @@ -70,8 +70,7 @@ impl WindowUDFImpl for SimplifySmoothItUdf { /// this function will simplify `SimplifySmoothItUdf` to `SmoothItUdf`. fn simplify(&self) -> Option { - let simplify = |window_function: datafusion_expr::expr::WindowFunction, - _: &dyn SimplifyInfo| { + let simplify = |window_function: WindowFunction, _: &dyn SimplifyInfo| { Ok(Expr::WindowFunction(WindowFunction { fun: datafusion_expr::WindowFunctionDefinition::AggregateUDF(avg_udaf()), args: window_function.args, diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 33e5184d2cace..15290204fbace 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -876,7 +876,7 @@ pub trait ConfigExtension: ExtensionOptions { } /// An object-safe API for storing arbitrary configuration -pub trait ExtensionOptions: Send + Sync + std::fmt::Debug + 'static { +pub trait ExtensionOptions: Send + Sync + fmt::Debug + 'static { /// Return `self` as [`Any`] /// /// This is needed until trait upcasting is stabilised diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index fbdae1c50a83e..d502e7836da3a 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -97,7 +97,7 @@ pub enum JoinConstraint { } impl Display for JoinSide { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { match self { JoinSide::Left => write!(f, "left"), JoinSide::Right => write!(f, "right"), diff --git a/datafusion/common/src/parsers.rs b/datafusion/common/src/parsers.rs index e23edb4e2adb7..c73c8a55f18c5 100644 --- a/datafusion/common/src/parsers.rs +++ b/datafusion/common/src/parsers.rs @@ -18,7 +18,6 @@ //! Interval parsing logic use std::fmt::Display; -use std::result; use std::str::FromStr; use sqlparser::parser::ParserError; @@ -41,7 +40,7 @@ pub enum CompressionTypeVariant { impl FromStr for CompressionTypeVariant { type Err = ParserError; - fn from_str(s: &str) -> result::Result { + fn from_str(s: &str) -> Result { let s = s.to_uppercase(); match s.as_str() { "GZIP" | "GZ" => Ok(Self::GZIP), diff --git a/datafusion/common/src/pyarrow.rs b/datafusion/common/src/pyarrow.rs index 87254a499fb11..bdcf831c7884b 100644 --- a/datafusion/common/src/pyarrow.rs +++ b/datafusion/common/src/pyarrow.rs @@ -34,7 +34,7 @@ impl From for PyErr { } impl FromPyArrow for ScalarValue { - fn from_pyarrow_bound(value: &pyo3::Bound<'_, pyo3::PyAny>) -> PyResult { + fn from_pyarrow_bound(value: &Bound<'_, PyAny>) -> PyResult { let py = value.py(); let typ = value.getattr("type")?; let val = value.call_method0("as_py")?; diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 43f22265f5f64..7a1eaa2ad65b0 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -28,6 +28,7 @@ use std::fmt; use std::hash::Hash; use std::hash::Hasher; use std::iter::repeat; +use std::mem::{size_of, size_of_val}; use std::str::FromStr; use std::sync::Arc; @@ -58,6 +59,7 @@ use arrow::{ use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, ScalarBuffer}; use arrow_schema::{UnionFields, UnionMode}; +use crate::format::DEFAULT_CAST_OPTIONS; use half::f16; pub use struct_builder::ScalarStructBuilder; @@ -690,8 +692,8 @@ hash_float_value!((f64, u64), (f32, u32)); // # Panics // // Panics if there is an error when creating hash values for rows -impl std::hash::Hash for ScalarValue { - fn hash(&self, state: &mut H) { +impl Hash for ScalarValue { + fn hash(&self, state: &mut H) { use ScalarValue::*; match self { Decimal128(v, p, s) => { @@ -767,7 +769,7 @@ impl std::hash::Hash for ScalarValue { } } -fn hash_nested_array(arr: ArrayRef, state: &mut H) { +fn hash_nested_array(arr: ArrayRef, state: &mut H) { let arrays = vec![arr.to_owned()]; let hashes_buffer = &mut vec![0; arr.len()]; let random_state = ahash::RandomState::with_seeds(0, 0, 0, 0); @@ -801,7 +803,7 @@ fn dict_from_scalar( let values_array = value.to_array_of_size(1)?; // Create a key array with `size` elements, each of 0 - let key_array: PrimitiveArray = std::iter::repeat(if value.is_null() { + let key_array: PrimitiveArray = repeat(if value.is_null() { None } else { Some(K::default_value()) @@ -2042,7 +2044,7 @@ impl ScalarValue { scale: i8, size: usize, ) -> Result { - Ok(std::iter::repeat(value) + Ok(repeat(value) .take(size) .collect::() .with_precision_and_scale(precision, scale)?) @@ -2511,7 +2513,7 @@ impl ScalarValue { } fn list_to_array_of_size(arr: &dyn Array, size: usize) -> Result { - let arrays = std::iter::repeat(arr).take(size).collect::>(); + let arrays = repeat(arr).take(size).collect::>(); let ret = match !arrays.is_empty() { true => arrow::compute::concat(arrays.as_slice())?, false => arr.slice(0, 0), @@ -2809,22 +2811,30 @@ impl ScalarValue { /// Try to parse `value` into a ScalarValue of type `target_type` pub fn try_from_string(value: String, target_type: &DataType) -> Result { - let value = ScalarValue::from(value); - let cast_options = CastOptions { - safe: false, - format_options: Default::default(), - }; - let cast_arr = cast_with_options(&value.to_array()?, target_type, &cast_options)?; - ScalarValue::try_from_array(&cast_arr, 0) + ScalarValue::from(value).cast_to(target_type) } /// Try to cast this value to a ScalarValue of type `data_type` - pub fn cast_to(&self, data_type: &DataType) -> Result { - let cast_options = CastOptions { - safe: false, - format_options: Default::default(), + pub fn cast_to(&self, target_type: &DataType) -> Result { + self.cast_to_with_options(target_type, &DEFAULT_CAST_OPTIONS) + } + + /// Try to cast this value to a ScalarValue of type `data_type` with [`CastOptions`] + pub fn cast_to_with_options( + &self, + target_type: &DataType, + cast_options: &CastOptions<'static>, + ) -> Result { + let scalar_array = match (self, target_type) { + ( + ScalarValue::Float64(Some(float_ts)), + DataType::Timestamp(TimeUnit::Nanosecond, None), + ) => ScalarValue::Int64(Some((float_ts * 1_000_000_000_f64).trunc() as i64)) + .to_array()?, + _ => self.to_array()?, }; - let cast_arr = cast_with_options(&self.to_array()?, data_type, &cast_options)?; + + let cast_arr = cast_with_options(&scalar_array, target_type, cast_options)?; ScalarValue::try_from_array(&cast_arr, 0) } @@ -3074,7 +3084,7 @@ impl ScalarValue { /// Estimate size if bytes including `Self`. For values with internal containers such as `String` /// includes the allocated size (`capacity`) rather than the current length (`len`) pub fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) + match self { ScalarValue::Null | ScalarValue::Boolean(_) @@ -3128,12 +3138,12 @@ impl ScalarValue { ScalarValue::Map(arr) => arr.get_array_memory_size(), ScalarValue::Union(vals, fields, _mode) => { vals.as_ref() - .map(|(_id, sv)| sv.size() - std::mem::size_of_val(sv)) + .map(|(_id, sv)| sv.size() - size_of_val(sv)) .unwrap_or_default() // `fields` is boxed, so it is NOT already included in `self` - + std::mem::size_of_val(fields) - + (std::mem::size_of::() * fields.len()) - + fields.iter().map(|(_idx, field)| field.size() - std::mem::size_of_val(field)).sum::() + + size_of_val(fields) + + (size_of::() * fields.len()) + + fields.iter().map(|(_idx, field)| field.size() - size_of_val(field)).sum::() } ScalarValue::Dictionary(dt, sv) => { // `dt` and `sv` are boxed, so they are NOT already included in `self` @@ -3146,11 +3156,11 @@ impl ScalarValue { /// /// Includes the size of the [`Vec`] container itself. pub fn size_of_vec(vec: &Vec) -> usize { - std::mem::size_of_val(vec) - + (std::mem::size_of::() * vec.capacity()) + size_of_val(vec) + + (size_of::() * vec.capacity()) + vec .iter() - .map(|sv| sv.size() - std::mem::size_of_val(sv)) + .map(|sv| sv.size() - size_of_val(sv)) .sum::() } @@ -3158,11 +3168,11 @@ impl ScalarValue { /// /// Includes the size of the [`VecDeque`] container itself. pub fn size_of_vec_deque(vec_deque: &VecDeque) -> usize { - std::mem::size_of_val(vec_deque) - + (std::mem::size_of::() * vec_deque.capacity()) + size_of_val(vec_deque) + + (size_of::() * vec_deque.capacity()) + vec_deque .iter() - .map(|sv| sv.size() - std::mem::size_of_val(sv)) + .map(|sv| sv.size() - size_of_val(sv)) .sum::() } @@ -3170,11 +3180,11 @@ impl ScalarValue { /// /// Includes the size of the [`HashSet`] container itself. pub fn size_of_hashset(set: &HashSet) -> usize { - std::mem::size_of_val(set) - + (std::mem::size_of::() * set.capacity()) + size_of_val(set) + + (size_of::() * set.capacity()) + set .iter() - .map(|sv| sv.size() - std::mem::size_of_val(sv)) + .map(|sv| sv.size() - size_of_val(sv)) .sum::() } } @@ -4436,7 +4446,7 @@ mod tests { let right_array = right.to_array().expect("Failed to convert to array"); let arrow_left_array = left_array.as_primitive::(); let arrow_right_array = right_array.as_primitive::(); - let arrow_result = kernels::numeric::add(arrow_left_array, arrow_right_array); + let arrow_result = add(arrow_left_array, arrow_right_array); assert_eq!(scalar_result.is_ok(), arrow_result.is_ok()); } @@ -5051,13 +5061,13 @@ mod tests { // thus the size of the enum appears to as well // The value may also change depending on rust version - assert_eq!(std::mem::size_of::(), 64); + assert_eq!(size_of::(), 64); } #[test] fn memory_size() { let sv = ScalarValue::Binary(Some(Vec::with_capacity(10))); - assert_eq!(sv.size(), std::mem::size_of::() + 10,); + assert_eq!(sv.size(), size_of::() + 10,); let sv_size = sv.size(); let mut v = Vec::with_capacity(10); @@ -5066,9 +5076,7 @@ mod tests { assert_eq!(v.capacity(), 10); assert_eq!( ScalarValue::size_of_vec(&v), - std::mem::size_of::>() - + (9 * std::mem::size_of::()) - + sv_size, + size_of::>() + (9 * size_of::()) + sv_size, ); let mut s = HashSet::with_capacity(0); @@ -5078,8 +5086,8 @@ mod tests { let s_capacity = s.capacity(); assert_eq!( ScalarValue::size_of_hashset(&s), - std::mem::size_of::>() - + ((s_capacity - 1) * std::mem::size_of::()) + size_of::>() + + ((s_capacity - 1) * size_of::()) + sv_size, ); } diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index d8e62b3045f93..e669c674f78a2 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -190,7 +190,7 @@ impl Precision { } } -impl Debug for Precision { +impl Debug for Precision { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { Precision::Exact(inner) => write!(f, "Exact({:?})", inner), @@ -200,7 +200,7 @@ impl Debug for Precision } } -impl Display for Precision { +impl Display for Precision { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { Precision::Exact(inner) => write!(f, "Exact({:?})", inner), @@ -341,7 +341,7 @@ fn check_num_rows(value: Option, is_exact: bool) -> Precision { } impl Display for Statistics { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { // string of column statistics let column_stats = self .column_statistics diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index 422fcb5eb3e0b..d3b8c84512583 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -347,7 +347,7 @@ macro_rules! record_batch { let batch = arrow_array::RecordBatch::try_new( schema, vec![$( - create_array!($type, $values), + $crate::create_array!($type, $values), )*] ); diff --git a/datafusion/common/src/utils/memory.rs b/datafusion/common/src/utils/memory.rs index 2c34b61bd0930..d5ce59e3421b9 100644 --- a/datafusion/common/src/utils/memory.rs +++ b/datafusion/common/src/utils/memory.rs @@ -18,6 +18,7 @@ //! This module provides a function to estimate the memory size of a HashTable prior to alloaction use crate::{DataFusionError, Result}; +use std::mem::size_of; /// Estimates the memory size required for a hash table prior to allocation. /// @@ -87,7 +88,7 @@ pub fn estimate_memory_size(num_elements: usize, fixed_size: usize) -> Result // + size of entry * number of buckets // + 1 byte for each bucket // + fixed size of collection (HashSet/HashTable) - std::mem::size_of::() + size_of::() .checked_mul(estimated_buckets)? .checked_add(estimated_buckets)? .checked_add(fixed_size) @@ -108,7 +109,7 @@ mod tests { #[test] fn test_estimate_memory() { // size (bytes): 48 - let fixed_size = std::mem::size_of::>(); + let fixed_size = size_of::>(); // estimated buckets: 16 = (8 * 8 / 7).next_power_of_two() let num_elements = 8; @@ -126,7 +127,7 @@ mod tests { #[test] fn test_estimate_memory_overflow() { let num_elements = usize::MAX; - let fixed_size = std::mem::size_of::>(); + let fixed_size = size_of::>(); let estimated = estimate_memory_size::(num_elements, fixed_size); assert!(estimated.is_err()); diff --git a/datafusion/common/src/utils/proxy.rs b/datafusion/common/src/utils/proxy.rs index d68b5e354384a..5d14a15171295 100644 --- a/datafusion/common/src/utils/proxy.rs +++ b/datafusion/common/src/utils/proxy.rs @@ -18,6 +18,7 @@ //! [`VecAllocExt`] and [`RawTableAllocExt`] to help tracking of memory allocations use hashbrown::raw::{Bucket, RawTable}; +use std::mem::size_of; /// Extension trait for [`Vec`] to account for allocations. pub trait VecAllocExt { @@ -93,7 +94,7 @@ impl VecAllocExt for Vec { let new_capacity = self.capacity(); if new_capacity > prev_capacty { // capacity changed, so we allocated more - let bump_size = (new_capacity - prev_capacty) * std::mem::size_of::(); + let bump_size = (new_capacity - prev_capacty) * size_of::(); // Note multiplication should never overflow because `push` would // have panic'd first, but the checked_add could potentially // overflow since accounting could be tracking additional values, and @@ -102,7 +103,7 @@ impl VecAllocExt for Vec { } } fn allocated_size(&self) -> usize { - std::mem::size_of::() * self.capacity() + size_of::() * self.capacity() } } @@ -157,7 +158,7 @@ impl RawTableAllocExt for RawTable { // need to request more memory let bump_elements = self.capacity().max(16); - let bump_size = bump_elements * std::mem::size_of::(); + let bump_size = bump_elements * size_of::(); *accounting = (*accounting).checked_add(bump_size).expect("overflow"); self.reserve(bump_elements, hasher); diff --git a/datafusion/core/benches/parquet_query_sql.rs b/datafusion/core/benches/parquet_query_sql.rs index bc4298786002e..f82a126c56520 100644 --- a/datafusion/core/benches/parquet_query_sql.rs +++ b/datafusion/core/benches/parquet_query_sql.rs @@ -249,7 +249,7 @@ fn criterion_benchmark(c: &mut Criterion) { } // Temporary file must outlive the benchmarks, it is deleted when dropped - std::mem::drop(temp_file); + drop(temp_file); } criterion_group!(benches, criterion_benchmark); diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index 6f9cf02873d15..140e266a02720 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -15,22 +15,31 @@ // specific language governing permissions and limitations // under the License. +extern crate arrow; #[macro_use] extern crate criterion; -extern crate arrow; extern crate datafusion; mod data_utils; + use crate::criterion::Criterion; use arrow::datatypes::{DataType, Field, Fields, Schema}; use datafusion::datasource::MemTable; use datafusion::execution::context::SessionContext; +use itertools::Itertools; +use std::fs::File; +use std::io::{BufRead, BufReader}; +use std::path::PathBuf; use std::sync::Arc; use test_utils::tpcds::tpcds_schemas; use test_utils::tpch::tpch_schemas; use test_utils::TableDef; use tokio::runtime::Runtime; +const BENCHMARKS_PATH_1: &str = "../../benchmarks/"; +const BENCHMARKS_PATH_2: &str = "./benchmarks/"; +const CLICKBENCH_DATA_PATH: &str = "data/hits_partitioned/"; + /// Create a logical plan from the specified sql fn logical_plan(ctx: &SessionContext, sql: &str) { let rt = Runtime::new().unwrap(); @@ -91,7 +100,37 @@ fn register_defs(ctx: SessionContext, defs: Vec) -> SessionContext { ctx } +fn register_clickbench_hits_table() -> SessionContext { + let ctx = SessionContext::new(); + let rt = Runtime::new().unwrap(); + + // use an external table for clickbench benchmarks + let path = + if PathBuf::from(format!("{BENCHMARKS_PATH_1}{CLICKBENCH_DATA_PATH}")).exists() { + format!("{BENCHMARKS_PATH_1}{CLICKBENCH_DATA_PATH}") + } else { + format!("{BENCHMARKS_PATH_2}{CLICKBENCH_DATA_PATH}") + }; + + let sql = format!("CREATE EXTERNAL TABLE hits STORED AS PARQUET LOCATION '{path}'"); + + rt.block_on(ctx.sql(&sql)).unwrap(); + + let count = + rt.block_on(async { ctx.table("hits").await.unwrap().count().await.unwrap() }); + assert!(count > 0); + ctx +} + fn criterion_benchmark(c: &mut Criterion) { + // verify that we can load the clickbench data prior to running the benchmark + if !PathBuf::from(format!("{BENCHMARKS_PATH_1}{CLICKBENCH_DATA_PATH}")).exists() + && !PathBuf::from(format!("{BENCHMARKS_PATH_2}{CLICKBENCH_DATA_PATH}")).exists() + { + panic!("benchmarks/data/hits_partitioned/ could not be loaded. Please run \ + 'benchmarks/bench.sh data clickbench_partitioned' prior to running this benchmark") + } + let ctx = create_context(); // Test simplest @@ -235,9 +274,15 @@ fn criterion_benchmark(c: &mut Criterion) { "q16", "q17", "q18", "q19", "q20", "q21", "q22", ]; + let benchmarks_path = if PathBuf::from(BENCHMARKS_PATH_1).exists() { + BENCHMARKS_PATH_1 + } else { + BENCHMARKS_PATH_2 + }; + for q in tpch_queries { let sql = - std::fs::read_to_string(format!("../../benchmarks/queries/{q}.sql")).unwrap(); + std::fs::read_to_string(format!("{benchmarks_path}queries/{q}.sql")).unwrap(); c.bench_function(&format!("physical_plan_tpch_{}", q), |b| { b.iter(|| physical_plan(&tpch_ctx, &sql)) }); @@ -246,7 +291,7 @@ fn criterion_benchmark(c: &mut Criterion) { let all_tpch_sql_queries = tpch_queries .iter() .map(|q| { - std::fs::read_to_string(format!("../../benchmarks/queries/{q}.sql")).unwrap() + std::fs::read_to_string(format!("{benchmarks_path}queries/{q}.sql")).unwrap() }) .collect::>(); @@ -258,20 +303,25 @@ fn criterion_benchmark(c: &mut Criterion) { }) }); - c.bench_function("logical_plan_tpch_all", |b| { - b.iter(|| { - for sql in &all_tpch_sql_queries { - logical_plan(&tpch_ctx, sql) - } - }) - }); + // c.bench_function("logical_plan_tpch_all", |b| { + // b.iter(|| { + // for sql in &all_tpch_sql_queries { + // logical_plan(&tpch_ctx, sql) + // } + // }) + // }); // --- TPC-DS --- let tpcds_ctx = register_defs(SessionContext::new(), tpcds_schemas()); + let tests_path = if PathBuf::from("./tests/").exists() { + "./tests/" + } else { + "datafusion/core/tests/" + }; let raw_tpcds_sql_queries = (1..100) - .map(|q| std::fs::read_to_string(format!("./tests/tpc-ds/{q}.sql")).unwrap()) + .map(|q| std::fs::read_to_string(format!("{tests_path}tpc-ds/{q}.sql")).unwrap()) .collect::>(); // some queries have multiple statements @@ -288,10 +338,53 @@ fn criterion_benchmark(c: &mut Criterion) { }) }); - c.bench_function("logical_plan_tpcds_all", |b| { + // c.bench_function("logical_plan_tpcds_all", |b| { + // b.iter(|| { + // for sql in &all_tpcds_sql_queries { + // logical_plan(&tpcds_ctx, sql) + // } + // }) + // }); + + // -- clickbench -- + + let queries_file = + File::open(format!("{benchmarks_path}queries/clickbench/queries.sql")).unwrap(); + let extended_file = + File::open(format!("{benchmarks_path}queries/clickbench/extended.sql")).unwrap(); + + let clickbench_queries: Vec = BufReader::new(queries_file) + .lines() + .chain(BufReader::new(extended_file).lines()) + .map(|l| l.expect("Could not parse line")) + .collect_vec(); + + let clickbench_ctx = register_clickbench_hits_table(); + + // for (i, sql) in clickbench_queries.iter().enumerate() { + // c.bench_function(&format!("logical_plan_clickbench_q{}", i + 1), |b| { + // b.iter(|| logical_plan(&clickbench_ctx, sql)) + // }); + // } + + for (i, sql) in clickbench_queries.iter().enumerate() { + c.bench_function(&format!("physical_plan_clickbench_q{}", i + 1), |b| { + b.iter(|| physical_plan(&clickbench_ctx, sql)) + }); + } + + // c.bench_function("logical_plan_clickbench_all", |b| { + // b.iter(|| { + // for sql in &clickbench_queries { + // logical_plan(&clickbench_ctx, sql) + // } + // }) + // }); + + c.bench_function("physical_plan_clickbench_all", |b| { b.iter(|| { - for sql in &all_tpcds_sql_queries { - logical_plan(&tpcds_ctx, sql) + for sql in &clickbench_queries { + physical_plan(&clickbench_ctx, sql) } }) }); diff --git a/datafusion/core/src/bin/print_functions_docs.rs b/datafusion/core/src/bin/print_functions_docs.rs index d87c3cefe6663..3aedcbc2aa63e 100644 --- a/datafusion/core/src/bin/print_functions_docs.rs +++ b/datafusion/core/src/bin/print_functions_docs.rs @@ -16,6 +16,7 @@ // under the License. use datafusion::execution::SessionStateDefaults; +use datafusion_common::{not_impl_err, Result}; use datafusion_expr::{ aggregate_doc_sections, scalar_doc_sections, window_doc_sections, AggregateUDF, DocSection, Documentation, ScalarUDF, WindowUDF, @@ -30,7 +31,7 @@ use std::fmt::Write as _; /// Usage: `cargo run --bin print_functions_docs -- ` /// /// Called from `dev/update_function_docs.sh` -fn main() { +fn main() -> Result<()> { let args: Vec = args().collect(); if args.len() != 2 { @@ -48,12 +49,13 @@ fn main() { _ => { panic!("Unknown function type: {}", function_type) } - }; + }?; println!("{docs}"); + Ok(()) } -fn print_aggregate_docs() -> String { +fn print_aggregate_docs() -> Result { let mut providers: Vec> = vec![]; for f in SessionStateDefaults::default_aggregate_functions() { @@ -63,7 +65,7 @@ fn print_aggregate_docs() -> String { print_docs(providers, aggregate_doc_sections::doc_sections()) } -fn print_scalar_docs() -> String { +fn print_scalar_docs() -> Result { let mut providers: Vec> = vec![]; for f in SessionStateDefaults::default_scalar_functions() { @@ -73,7 +75,7 @@ fn print_scalar_docs() -> String { print_docs(providers, scalar_doc_sections::doc_sections()) } -fn print_window_docs() -> String { +fn print_window_docs() -> Result { let mut providers: Vec> = vec![]; for f in SessionStateDefaults::default_window_functions() { @@ -86,7 +88,7 @@ fn print_window_docs() -> String { fn print_docs( providers: Vec>, doc_sections: Vec, -) -> String { +) -> Result { let mut docs = "".to_string(); // Ensure that all providers have documentation @@ -193,6 +195,13 @@ fn print_docs( ); } + if let Some(alt_syntax) = &documentation.alternative_syntax { + let _ = writeln!(docs, "#### Alternative Syntax\n"); + for syntax in alt_syntax { + let _ = writeln!(docs, "```sql\n{}\n```", syntax); + } + } + // next, aliases if !f.get_aliases().is_empty() { let _ = writeln!(docs, "#### Aliases"); @@ -217,12 +226,13 @@ fn print_docs( // eventually make this an error: https://github.com/apache/datafusion/issues/12872 if !providers_with_no_docs.is_empty() { eprintln!("INFO: The following functions do not have documentation:"); - for f in providers_with_no_docs { + for f in &providers_with_no_docs { eprintln!(" - {f}"); } + not_impl_err!("Some functions do not have documentation. Please implement `documentation` for: {providers_with_no_docs:?}") + } else { + Ok(docs) } - - docs } /// Trait for accessing name / aliases / documentation for differnet functions diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index d1d49bfaa6930..e5d352a63c7a3 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1941,12 +1941,12 @@ mod tests { use crate::physical_plan::{ColumnarValue, Partitioning, PhysicalExpr}; use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name}; - use arrow::array::{self, Int32Array}; + use arrow::array::Int32Array; use datafusion_common::{assert_batches_eq, Constraint, Constraints, ScalarValue}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::expr::WindowFunction; use datafusion_expr::{ - cast, create_udf, expr, lit, BuiltInWindowFunction, ExprFunctionExt, + cast, create_udf, lit, BuiltInWindowFunction, ExprFunctionExt, ScalarFunctionImplementation, Volatility, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; @@ -1979,8 +1979,8 @@ mod tests { let batch = RecordBatch::try_new( dual_schema.clone(), vec![ - Arc::new(array::Int32Array::from(vec![1])), - Arc::new(array::StringArray::from(vec!["a"])), + Arc::new(Int32Array::from(vec![1])), + Arc::new(StringArray::from(vec!["a"])), ], ) .unwrap(); @@ -2176,7 +2176,7 @@ mod tests { async fn select_with_window_exprs() -> Result<()> { // build plan using Table API let t = test_table().await?; - let first_row = Expr::WindowFunction(expr::WindowFunction::new( + let first_row = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::BuiltInWindowFunction( BuiltInWindowFunction::FirstValue, ), @@ -3570,11 +3570,10 @@ mod tests { #[tokio::test] async fn with_column_renamed_case_sensitive() -> Result<()> { - let config = - SessionConfig::from_string_hash_map(&std::collections::HashMap::from([( - "datafusion.sql_parser.enable_ident_normalization".to_owned(), - "false".to_owned(), - )]))?; + let config = SessionConfig::from_string_hash_map(&HashMap::from([( + "datafusion.sql_parser.enable_ident_normalization".to_owned(), + "false".to_owned(), + )]))?; let ctx = SessionContext::new_with_config(config); let name = "aggregate_test_100"; register_aggregate_csv(&ctx, name).await?; @@ -3646,7 +3645,7 @@ mod tests { #[tokio::test] async fn row_writer_resize_test() -> Result<()> { - let schema = Arc::new(Schema::new(vec![arrow::datatypes::Field::new( + let schema = Arc::new(Schema::new(vec![Field::new( "column_1", DataType::Utf8, false, @@ -3655,7 +3654,7 @@ mod tests { let data = RecordBatch::try_new( schema, vec![ - Arc::new(arrow::array::StringArray::from(vec![ + Arc::new(StringArray::from(vec![ Some("2a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000"), Some("3a0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000800"), ])) diff --git a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs index 98b6702bc3834..9f089c7c0cea8 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs @@ -206,7 +206,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { fn build_primitive_array(&self, rows: RecordSlice, col_name: &str) -> ArrayRef where T: ArrowNumericType + Resolver, - T::Native: num_traits::cast::NumCast, + T::Native: NumCast, { Arc::new( rows.iter() @@ -354,7 +354,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { let builder = builder .as_any_mut() .downcast_mut::>() - .ok_or_else(||ArrowError::SchemaError( + .ok_or_else(||SchemaError( "Cast failed for ListBuilder during nested data parsing".to_string(), ))?; for val in vals { @@ -369,7 +369,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { builder.append(true); } DataType::Dictionary(_, _) => { - let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||ArrowError::SchemaError( + let builder = builder.as_any_mut().downcast_mut::>>().ok_or_else(||SchemaError( "Cast failed for ListBuilder during nested data parsing".to_string(), ))?; for val in vals { @@ -402,7 +402,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { col_name: &str, ) -> ArrowResult where - T::Native: num_traits::cast::NumCast, + T::Native: NumCast, T: ArrowPrimitiveType + ArrowDictionaryKeyType, { let mut builder: StringDictionaryBuilder = @@ -453,12 +453,10 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::UInt64 => { self.build_dictionary_array::(rows, col_name) } - _ => Err(ArrowError::SchemaError( - "unsupported dictionary key type".to_string(), - )), + _ => Err(SchemaError("unsupported dictionary key type".to_string())), } } else { - Err(ArrowError::SchemaError( + Err(SchemaError( "dictionary types other than UTF-8 not yet supported".to_string(), )) } @@ -532,7 +530,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { DataType::UInt32 => self.read_primitive_list_values::(rows), DataType::UInt64 => self.read_primitive_list_values::(rows), DataType::Float16 => { - return Err(ArrowError::SchemaError("Float16 not supported".to_string())) + return Err(SchemaError("Float16 not supported".to_string())) } DataType::Float32 => self.read_primitive_list_values::(rows), DataType::Float64 => self.read_primitive_list_values::(rows), @@ -541,7 +539,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { | DataType::Date64 | DataType::Time32(_) | DataType::Time64(_) => { - return Err(ArrowError::SchemaError( + return Err(SchemaError( "Temporal types are not yet supported, see ARROW-4803".to_string(), )) } @@ -623,7 +621,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { .unwrap() } datatype => { - return Err(ArrowError::SchemaError(format!( + return Err(SchemaError(format!( "Nested list of {datatype:?} not supported" ))); } @@ -737,7 +735,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { &field_path, ), t => { - return Err(ArrowError::SchemaError(format!( + return Err(SchemaError(format!( "TimeUnit {t:?} not supported with Time64" ))) } @@ -751,7 +749,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { &field_path, ), t => { - return Err(ArrowError::SchemaError(format!( + return Err(SchemaError(format!( "TimeUnit {t:?} not supported with Time32" ))) } @@ -854,7 +852,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { make_array(data) } _ => { - return Err(ArrowError::SchemaError(format!( + return Err(SchemaError(format!( "type {:?} not supported", field.data_type() ))) @@ -870,7 +868,7 @@ impl<'a, R: Read> AvroArrowArrayReader<'a, R> { fn read_primitive_list_values(&self, rows: &[&Value]) -> ArrayData where T: ArrowPrimitiveType + ArrowNumericType, - T::Native: num_traits::cast::NumCast, + T::Native: NumCast, { let values = rows .iter() @@ -970,7 +968,7 @@ fn resolve_u8(v: &Value) -> AvroResult { other => Err(AvroError::GetU8(other.into())), }?; if let Value::Int(n) = int { - if n >= 0 && n <= std::convert::From::from(u8::MAX) { + if n >= 0 && n <= From::from(u8::MAX) { return Ok(n as u8); } } @@ -1048,7 +1046,7 @@ fn maybe_resolve_union(value: &Value) -> &Value { impl Resolver for N where N: ArrowNumericType, - N::Native: num_traits::cast::NumCast, + N::Native: NumCast, { fn resolve(value: &Value) -> Option { let value = maybe_resolve_union(value); diff --git a/datafusion/core/src/datasource/avro_to_arrow/mod.rs b/datafusion/core/src/datasource/avro_to_arrow/mod.rs index c59078c89dd00..71184a78c96f5 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/mod.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/mod.rs @@ -39,7 +39,7 @@ use std::io::Read; pub fn read_avro_schema_from_reader(reader: &mut R) -> Result { let avro_reader = apache_avro::Reader::new(reader)?; let schema = avro_reader.writer_schema(); - schema::to_arrow_schema(schema) + to_arrow_schema(schema) } #[cfg(not(feature = "avro"))] diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index f235c3b628a0d..3cb5ae4f85cad 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -78,7 +78,7 @@ impl CsvFormatFactory { } } -impl fmt::Debug for CsvFormatFactory { +impl Debug for CsvFormatFactory { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("CsvFormatFactory") .field("options", &self.options) @@ -968,7 +968,7 @@ mod tests { limit: Option, has_header: bool, ) -> Result> { - let root = format!("{}/csv", crate::test_util::arrow_test_data()); + let root = format!("{}/csv", arrow_test_data()); let format = CsvFormat::default().with_has_header(has_header); scan_format(state, &format, &root, file_name, projection, limit).await } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index c9ed0c0d28059..fd97da52165b9 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -118,7 +118,7 @@ impl GetExt for JsonFormatFactory { } } -impl fmt::Debug for JsonFormatFactory { +impl Debug for JsonFormatFactory { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("JsonFormatFactory") .field("options", &self.options) diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index a313a7a9bcb19..24f1111517d2e 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -79,7 +79,7 @@ pub trait FileFormatFactory: Sync + Send + GetExt + Debug { /// /// [`TableProvider`]: crate::catalog::TableProvider #[async_trait] -pub trait FileFormat: Send + Sync + fmt::Debug { +pub trait FileFormat: Send + Sync + Debug { /// Returns the table provider as [`Any`](std::any::Any) so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; @@ -224,7 +224,7 @@ pub fn format_as_file_type( /// downcasted to a [DefaultFileType]. pub fn file_type_to_format( file_type: &Arc, -) -> datafusion_common::Result> { +) -> Result> { match file_type .as_ref() .as_any() @@ -447,8 +447,8 @@ pub(crate) mod test_util { iterations_detected: Arc>, } - impl std::fmt::Display for VariableStream { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + impl Display for VariableStream { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "VariableStream") } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 2d45c76ce9182..9153e71a5c267 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -165,7 +165,7 @@ impl GetExt for ParquetFormatFactory { } } -impl fmt::Debug for ParquetFormatFactory { +impl Debug for ParquetFormatFactory { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("ParquetFormatFactory") .field("ParquetFormatFactory", &self.options) @@ -1439,7 +1439,7 @@ mod tests { } impl Display for RequestCountingObjectStore { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { write!(f, "RequestCounting({})", self.inner) } } @@ -1707,7 +1707,7 @@ mod tests { let null_utf8 = if force_views { ScalarValue::Utf8View(None) } else { - ScalarValue::Utf8(None) + Utf8(None) }; // Fetch statistics for first file @@ -1720,7 +1720,7 @@ mod tests { let expected_type = if force_views { ScalarValue::Utf8View } else { - ScalarValue::Utf8 + Utf8 }; assert_eq!( c1_stats.max_value, diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 427b28db40301..1746ffef8282b 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -280,9 +280,8 @@ async fn hive_style_partitions_demuxer( Some(part_tx) => part_tx, None => { // Create channel for previously unseen distinct partition key and notify consumer of new file - let (part_tx, part_rx) = tokio::sync::mpsc::channel::( - max_buffered_recordbatches, - ); + let (part_tx, part_rx) = + mpsc::channel::(max_buffered_recordbatches); let file_path = compute_hive_style_file_path( &part_key, &partition_by, diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 701a13477b5bb..581d88d25884a 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -91,7 +91,7 @@ impl TableProviderFactory for ListingTableFactory { .field_with_name(col) .map_err(|e| arrow_datafusion_err!(e)) }) - .collect::>>()? + .collect::>>()? .into_iter() .map(|f| (f.name().to_owned(), f.data_type().to_owned())) .collect(); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 6cd1864deb1d4..5beffc3b0581d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -1216,7 +1216,7 @@ mod tests { let session_ctx = SessionContext::new(); let store = object_store::memory::InMemory::new(); - let data = bytes::Bytes::from("a,b\n1,2\n3,4"); + let data = Bytes::from("a,b\n1,2\n3,4"); let path = object_store::path::Path::from("a.csv"); store.put(&path, data.into()).await.unwrap(); @@ -1247,7 +1247,7 @@ mod tests { let session_ctx = SessionContext::new(); let store = object_store::memory::InMemory::new(); - let data = bytes::Bytes::from("a,b\r1,2\r3,4"); + let data = Bytes::from("a,b\r1,2\r3,4"); let path = object_store::path::Path::from("a.csv"); store.put(&path, data.into()).await.unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 415ea62b3bb36..96c0e452e29e4 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -19,7 +19,8 @@ //! file sources. use std::{ - borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, sync::Arc, vec, + borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, mem::size_of, + sync::Arc, vec, }; use super::{get_projected_output_ordering, statistics::MinMaxStatistics}; @@ -497,7 +498,7 @@ impl ZeroBufferGenerator where T: ArrowNativeType, { - const SIZE: usize = std::mem::size_of::(); + const SIZE: usize = size_of::(); fn get_buffer(&mut self, n_vals: usize) -> Buffer { match &mut self.cache { diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 6e8752ccfbf4b..407a3b74f79f2 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -763,7 +763,7 @@ mod tests { /// create a PartitionedFile for testing fn partitioned_file(path: &str) -> PartitionedFile { let object_meta = ObjectMeta { - location: object_store::path::Path::parse(path).unwrap(), + location: Path::parse(path).unwrap(), last_modified: Utc::now(), size: 42, e_tag: None, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 743dd5896986a..059f86ce110f4 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -2227,7 +2227,7 @@ mod tests { // execute a simple query and write the results to parquet let out_dir = tmp_dir.as_ref().to_str().unwrap().to_string() + "/out"; - std::fs::create_dir(&out_dir).unwrap(); + fs::create_dir(&out_dir).unwrap(); let df = ctx.sql("SELECT c1, c2 FROM test").await?; let schema: Schema = df.schema().into(); // Register a listing table - this will use all files in the directory as data sources diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs index a1d74cb54355e..7406676652f66 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs @@ -779,11 +779,8 @@ mod tests { // INT32: c1 > 5, the c1 is decimal(9,2) // The type of scalar value if decimal(9,2), don't need to do cast - let schema = Arc::new(Schema::new(vec![Field::new( - "c1", - DataType::Decimal128(9, 2), - false, - )])); + let schema = + Arc::new(Schema::new(vec![Field::new("c1", Decimal128(9, 2), false)])); let field = PrimitiveTypeField::new("c1", PhysicalType::INT32) .with_logical_type(LogicalType::Decimal { scale: 2, @@ -849,11 +846,8 @@ mod tests { // The c1 type is decimal(9,0) in the parquet file, and the type of scalar is decimal(5,2). // We should convert all type to the coercion type, which is decimal(11,2) // The decimal of arrow is decimal(5,2), the decimal of parquet is decimal(9,0) - let schema = Arc::new(Schema::new(vec![Field::new( - "c1", - DataType::Decimal128(9, 0), - false, - )])); + let schema = + Arc::new(Schema::new(vec![Field::new("c1", Decimal128(9, 0), false)])); let field = PrimitiveTypeField::new("c1", PhysicalType::INT32) .with_logical_type(LogicalType::Decimal { @@ -863,7 +857,7 @@ mod tests { .with_scale(0) .with_precision(9); let schema_descr = get_test_schema_descr(vec![field]); - let expr = cast(col("c1"), DataType::Decimal128(11, 2)).gt(cast( + let expr = cast(col("c1"), Decimal128(11, 2)).gt(cast( lit(ScalarValue::Decimal128(Some(500), 5, 2)), Decimal128(11, 2), )); @@ -947,7 +941,7 @@ mod tests { // INT64: c1 < 5, the c1 is decimal(18,2) let schema = Arc::new(Schema::new(vec![Field::new( "c1", - DataType::Decimal128(18, 2), + Decimal128(18, 2), false, )])); let field = PrimitiveTypeField::new("c1", PhysicalType::INT64) @@ -1005,7 +999,7 @@ mod tests { // the type of parquet is decimal(18,2) let schema = Arc::new(Schema::new(vec![Field::new( "c1", - DataType::Decimal128(18, 2), + Decimal128(18, 2), false, )])); let field = PrimitiveTypeField::new("c1", PhysicalType::FIXED_LEN_BYTE_ARRAY) @@ -1018,7 +1012,7 @@ mod tests { .with_byte_len(16); let schema_descr = get_test_schema_descr(vec![field]); // cast the type of c1 to decimal(28,3) - let left = cast(col("c1"), DataType::Decimal128(28, 3)); + let left = cast(col("c1"), Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); @@ -1083,7 +1077,7 @@ mod tests { // the type of parquet is decimal(18,2) let schema = Arc::new(Schema::new(vec![Field::new( "c1", - DataType::Decimal128(18, 2), + Decimal128(18, 2), false, )])); let field = PrimitiveTypeField::new("c1", PhysicalType::BYTE_ARRAY) @@ -1096,7 +1090,7 @@ mod tests { .with_byte_len(16); let schema_descr = get_test_schema_descr(vec![field]); // cast the type of c1 to decimal(28,3) - let left = cast(col("c1"), DataType::Decimal128(28, 3)); + let left = cast(col("c1"), Decimal128(28, 3)); let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3))); let expr = logical2physical(&expr, &schema); let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index e1c61ec1a7129..3ca3ba89f4d97 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -278,13 +278,9 @@ impl MinMaxStatistics { fn sort_columns_from_physical_sort_exprs( sort_order: &[PhysicalSortExpr], -) -> Option> { +) -> Option> { sort_order .iter() - .map(|expr| { - expr.expr - .as_any() - .downcast_ref::() - }) + .map(|expr| expr.expr.as_any().downcast_ref::()) .collect::>>() } diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 131b8c354ce7a..5ba597e4b5420 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -32,11 +32,19 @@ use std::sync::Arc; /// /// This interface provides a way to implement custom schema adaptation logic /// for ParquetExec (for example, to fill missing columns with default value -/// other than null) +/// other than null). +/// +/// Most users should use [`DefaultSchemaAdapterFactory`]. See that struct for +/// more details and examples. pub trait SchemaAdapterFactory: Debug + Send + Sync + 'static { - /// Provides `SchemaAdapter`. - // The design of this function is mostly modeled for the needs of DefaultSchemaAdapterFactory, - // read its implementation docs for the reasoning + /// Create a [`SchemaAdapter`] + /// + /// Arguments: + /// + /// * `projected_table_schema`: The schema for the table, projected to + /// include only the fields being output (projected) by the this mapping. + /// + /// * `table_schema`: The entire table schema for the table fn create( &self, projected_table_schema: SchemaRef, @@ -44,53 +52,57 @@ pub trait SchemaAdapterFactory: Debug + Send + Sync + 'static { ) -> Box; } -/// Adapt file-level [`RecordBatch`]es to a table schema, which may have a schema -/// obtained from merging multiple file-level schemas. -/// -/// This is useful for enabling schema evolution in partitioned datasets. -/// -/// This has to be done in two stages. +/// Creates [`SchemaMapper`]s to map file-level [`RecordBatch`]es to a table +/// schema, which may have a schema obtained from merging multiple file-level +/// schemas. /// -/// 1. Before reading the file, we have to map projected column indexes from the -/// table schema to the file schema. +/// This is useful for implementing schema evolution in partitioned datasets. /// -/// 2. After reading a record batch map the read columns back to the expected -/// columns indexes and insert null-valued columns wherever the file schema was -/// missing a column present in the table schema. +/// See [`DefaultSchemaAdapterFactory`] for more details and examples. pub trait SchemaAdapter: Send + Sync { /// Map a column index in the table schema to a column index in a particular /// file schema /// + /// This is used while reading a file to push down projections by mapping + /// projected column indexes from the table schema to the file schema + /// /// Panics if index is not in range for the table schema fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option; - /// Creates a `SchemaMapping` that can be used to cast or map the columns - /// from the file schema to the table schema. + /// Creates a mapping for casting columns from the file schema to the table + /// schema. /// - /// If the provided `file_schema` contains columns of a different type to the expected - /// `table_schema`, the method will attempt to cast the array data from the file schema - /// to the table schema where possible. + /// This is used after reading a record batch. The returned [`SchemaMapper`]: /// - /// Returns a [`SchemaMapper`] that can be applied to the output batch - /// along with an ordered list of columns to project from the file + /// 1. Maps columns to the expected columns indexes + /// 2. Handles missing values (e.g. fills nulls or a default value) for + /// columns in the in the table schema not in the file schema + /// 2. Handles different types: if the column in the file schema has a + /// different type than `table_schema`, the mapper will resolve this + /// difference (e.g. by casting to the appropriate type) + /// + /// Returns: + /// * a [`SchemaMapper`] + /// * an ordered list of columns to project from the file fn map_schema( &self, file_schema: &Schema, ) -> datafusion_common::Result<(Arc, Vec)>; } -/// Maps, by casting or reordering columns from the file schema to the table -/// schema. +/// Maps, columns from a specific file schema to the table schema. +/// +/// See [`DefaultSchemaAdapterFactory`] for more details and examples. pub trait SchemaMapper: Debug + Send + Sync { - /// Adapts a `RecordBatch` to match the `table_schema` using the stored - /// mapping and conversions. + /// Adapts a `RecordBatch` to match the `table_schema` fn map_batch(&self, batch: RecordBatch) -> datafusion_common::Result; /// Adapts a [`RecordBatch`] that does not have all the columns from the /// file schema. /// - /// This method is used when applying a filter to a subset of the columns as - /// part of `DataFusionArrowPredicate` when `filter_pushdown` is enabled. + /// This method is used, for example, when applying a filter to a subset of + /// the columns as part of `DataFusionArrowPredicate` when `filter_pushdown` + /// is enabled. /// /// This method is slower than `map_batch` as it looks up columns by name. fn map_partial_batch( @@ -99,11 +111,106 @@ pub trait SchemaMapper: Debug + Send + Sync { ) -> datafusion_common::Result; } -/// Implementation of [`SchemaAdapterFactory`] that maps columns by name -/// and casts columns to the expected type. +/// Default [`SchemaAdapterFactory`] for mapping schemas. +/// +/// This can be used to adapt file-level record batches to a table schema and +/// implement schema evolution. +/// +/// Given an input file schema and a table schema, this factory returns +/// [`SchemaAdapter`] that return [`SchemaMapper`]s that: +/// +/// 1. Reorder columns +/// 2. Cast columns to the correct type +/// 3. Fill missing columns with nulls +/// +/// # Errors: +/// +/// * If a column in the table schema is non-nullable but is not present in the +/// file schema (i.e. it is missing), the returned mapper tries to fill it with +/// nulls resulting in a schema error. +/// +/// # Illustration of Schema Mapping +/// +/// ```text +/// ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ +/// ┌───────┐ ┌───────┐ │ ┌───────┐ ┌───────┐ ┌───────┐ │ +/// ││ 1.0 │ │ "foo" │ ││ NULL │ │ "foo" │ │ "1.0" │ +/// ├───────┤ ├───────┤ │ Schema mapping ├───────┤ ├───────┤ ├───────┤ │ +/// ││ 2.0 │ │ "bar" │ ││ NULL │ │ "bar" │ │ "2.0" │ +/// └───────┘ └───────┘ │────────────────▶ └───────┘ └───────┘ └───────┘ │ +/// │ │ +/// column "c" column "b"│ column "a" column "b" column "c"│ +/// │ Float64 Utf8 │ Int32 Utf8 Utf8 +/// ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ +/// Input Record Batch Output Record Batch +/// +/// Schema { Schema { +/// "c": Float64, "a": Int32, +/// "b": Utf8, "b": Utf8, +/// } "c": Utf8, +/// } +/// ``` +/// +/// # Example of using the `DefaultSchemaAdapterFactory` to map [`RecordBatch`]s +/// +/// Note `SchemaMapping` also supports mapping partial batches, which is used as +/// part of predicate pushdown. +/// +/// ``` +/// # use std::sync::Arc; +/// # use arrow::datatypes::{DataType, Field, Schema}; +/// # use datafusion::datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; +/// # use datafusion_common::record_batch; +/// // Table has fields "a", "b" and "c" +/// let table_schema = Schema::new(vec![ +/// Field::new("a", DataType::Int32, true), +/// Field::new("b", DataType::Utf8, true), +/// Field::new("c", DataType::Utf8, true), +/// ]); +/// +/// // create an adapter to map the table schema to the file schema +/// let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); +/// +/// // The file schema has fields "c" and "b" but "b" is stored as an 'Float64' +/// // instead of 'Utf8' +/// let file_schema = Schema::new(vec![ +/// Field::new("c", DataType::Utf8, true), +/// Field::new("b", DataType::Float64, true), +/// ]); +/// +/// // Get a mapping from the file schema to the table schema +/// let (mapper, _indices) = adapter.map_schema(&file_schema).unwrap(); +/// +/// let file_batch = record_batch!( +/// ("c", Utf8, vec!["foo", "bar"]), +/// ("b", Float64, vec![1.0, 2.0]) +/// ).unwrap(); +/// +/// let mapped_batch = mapper.map_batch(file_batch).unwrap(); +/// +/// // the mapped batch has the correct schema and the "b" column has been cast to Utf8 +/// let expected_batch = record_batch!( +/// ("a", Int32, vec![None, None]), // missing column filled with nulls +/// ("b", Utf8, vec!["1.0", "2.0"]), // b was cast to string and order was changed +/// ("c", Utf8, vec!["foo", "bar"]) +/// ).unwrap(); +/// assert_eq!(mapped_batch, expected_batch); +/// ``` #[derive(Clone, Debug, Default)] pub struct DefaultSchemaAdapterFactory; +impl DefaultSchemaAdapterFactory { + /// Create a new factory for mapping batches from a file schema to a table + /// schema. + /// + /// This is a convenience for [`DefaultSchemaAdapterFactory::create`] with + /// the same schema for both the projected table schema and the table + /// schema. + pub fn from_schema(table_schema: SchemaRef) -> Box { + Self.create(Arc::clone(&table_schema), table_schema) + } +} + impl SchemaAdapterFactory for DefaultSchemaAdapterFactory { fn create( &self, @@ -117,8 +224,8 @@ impl SchemaAdapterFactory for DefaultSchemaAdapterFactory { } } -/// This SchemaAdapter requires both the table schema and the projected table schema because of the -/// needs of the [`SchemaMapping`] it creates. Read its documentation for more details +/// This SchemaAdapter requires both the table schema and the projected table +/// schema. See [`SchemaMapping`] for more details #[derive(Clone, Debug)] pub(crate) struct DefaultSchemaAdapter { /// The schema for the table, projected to include only the fields being output (projected) by the @@ -142,11 +249,12 @@ impl SchemaAdapter for DefaultSchemaAdapter { Some(file_schema.fields.find(field.name())?.0) } - /// Creates a `SchemaMapping` that can be used to cast or map the columns from the file schema to the table schema. + /// Creates a `SchemaMapping` for casting or mapping the columns from the + /// file schema to the table schema. /// - /// If the provided `file_schema` contains columns of a different type to the expected - /// `table_schema`, the method will attempt to cast the array data from the file schema - /// to the table schema where possible. + /// If the provided `file_schema` contains columns of a different type to + /// the expected `table_schema`, the method will attempt to cast the array + /// data from the file schema to the table schema where possible. /// /// Returns a [`SchemaMapping`] that can be applied to the output batch /// along with an ordered list of columns to project from the file @@ -189,36 +297,45 @@ impl SchemaAdapter for DefaultSchemaAdapter { } } -/// The SchemaMapping struct holds a mapping from the file schema to the table schema -/// and any necessary type conversions that need to be applied. +/// The SchemaMapping struct holds a mapping from the file schema to the table +/// schema and any necessary type conversions. +/// +/// Note, because `map_batch` and `map_partial_batch` functions have different +/// needs, this struct holds two schemas: +/// +/// 1. The projected **table** schema +/// 2. The full table schema /// -/// This needs both the projected table schema and full table schema because its different -/// functions have different needs. The [`map_batch`] function is only used by the ParquetOpener to -/// produce a RecordBatch which has the projected schema, since that's the schema which is supposed -/// to come out of the execution of this query. [`map_partial_batch`], however, is used to create a -/// RecordBatch with a schema that can be used for Parquet pushdown, meaning that it may contain -/// fields which are not in the projected schema (as the fields that parquet pushdown filters -/// operate can be completely distinct from the fields that are projected (output) out of the -/// ParquetExec). +/// [`map_batch`] is used by the ParquetOpener to produce a RecordBatch which +/// has the projected schema, since that's the schema which is supposed to come +/// out of the execution of this query. Thus `map_batch` uses +/// `projected_table_schema` as it can only operate on the projected fields. /// -/// [`map_partial_batch`] uses `table_schema` to create the resulting RecordBatch (as it could be -/// operating on any fields in the schema), while [`map_batch`] uses `projected_table_schema` (as -/// it can only operate on the projected fields). +/// [`map_partial_batch`] is used to create a RecordBatch with a schema that +/// can be used for Parquet predicate pushdown, meaning that it may contain +/// fields which are not in the projected schema (as the fields that parquet +/// pushdown filters operate can be completely distinct from the fields that are +/// projected (output) out of the ParquetExec). `map_partial_batch` thus uses +/// `table_schema` to create the resulting RecordBatch (as it could be operating +/// on any fields in the schema). /// /// [`map_batch`]: Self::map_batch /// [`map_partial_batch`]: Self::map_partial_batch #[derive(Debug)] pub struct SchemaMapping { - /// The schema of the table. This is the expected schema after conversion and it should match - /// the schema of the query result. + /// The schema of the table. This is the expected schema after conversion + /// and it should match the schema of the query result. projected_table_schema: SchemaRef, - /// Mapping from field index in `projected_table_schema` to index in projected file_schema. - /// They are Options instead of just plain `usize`s because the table could have fields that - /// don't exist in the file. + /// Mapping from field index in `projected_table_schema` to index in + /// projected file_schema. + /// + /// They are Options instead of just plain `usize`s because the table could + /// have fields that don't exist in the file. field_mappings: Vec>, - /// The entire table schema, as opposed to the projected_table_schema (which only contains the - /// columns that we are projecting out of this query). This contains all fields in the table, - /// regardless of if they will be projected out or not. + /// The entire table schema, as opposed to the projected_table_schema (which + /// only contains the columns that we are projecting out of this query). + /// This contains all fields in the table, regardless of if they will be + /// projected out or not. table_schema: SchemaRef, } @@ -331,8 +448,9 @@ mod tests { use crate::datasource::listing::PartitionedFile; use crate::datasource::schema_adapter::{ - SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; + use datafusion_common::record_batch; #[cfg(feature = "parquet")] use parquet::arrow::ArrowWriter; use tempfile::TempDir; @@ -360,7 +478,7 @@ mod tests { writer.close().unwrap(); let location = Path::parse(path.to_str().unwrap()).unwrap(); - let metadata = std::fs::metadata(path.as_path()).expect("Local file metadata"); + let metadata = fs::metadata(path.as_path()).expect("Local file metadata"); let meta = ObjectMeta { location, last_modified: metadata.modified().map(chrono::DateTime::from).unwrap(), @@ -405,6 +523,58 @@ mod tests { assert_batches_sorted_eq!(expected, &read); } + #[test] + fn default_schema_adapter() { + let table_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Utf8, true), + ]); + + // file has a subset of the table schema fields and different type + let file_schema = Schema::new(vec![ + Field::new("c", DataType::Float64, true), // not in table schema + Field::new("b", DataType::Float64, true), + ]); + + let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); + let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(indices, vec![1]); + + let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); + + let mapped_batch = mapper.map_batch(file_batch).unwrap(); + + // the mapped batch has the correct schema and the "b" column has been cast to Utf8 + let expected_batch = record_batch!( + ("a", Int32, vec![None, None]), // missing column filled with nulls + ("b", Utf8, vec!["1.0", "2.0"]) // b was cast to string and order was changed + ) + .unwrap(); + assert_eq!(mapped_batch, expected_batch); + } + + #[test] + fn default_schema_adapter_non_nullable_columns() { + let table_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), // "a"" is declared non nullable + Field::new("b", DataType::Utf8, true), + ]); + let file_schema = Schema::new(vec![ + // since file doesn't have "a" it will be filled with nulls + Field::new("b", DataType::Float64, true), + ]); + + let adapter = DefaultSchemaAdapterFactory::from_schema(Arc::new(table_schema)); + let (mapper, indices) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(indices, vec![0]); + + let file_batch = record_batch!(("b", Float64, vec![1.0, 2.0])).unwrap(); + + // Mapping fails because it tries to fill in a non-nullable column with nulls + let err = mapper.map_batch(file_batch).unwrap_err().to_string(); + assert!(err.contains("Invalid argument error: Column 'a' is declared as non-nullable but contains null values"), "{err}"); + } + #[derive(Debug)] struct TestSchemaAdapterFactory; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 606759aae5ee0..333f83c673cc2 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -2139,9 +2139,9 @@ mod tests { fn create_physical_expr( &self, _expr: &Expr, - _input_dfschema: &crate::common::DFSchema, + _input_dfschema: &DFSchema, _session_state: &SessionState, - ) -> Result> { + ) -> Result> { unimplemented!() } } diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 4953eecd66e39..d50c912dd2fdc 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -512,7 +512,7 @@ impl SessionState { /// [`catalog::resolve_table_references`]: crate::catalog_common::resolve_table_references pub fn resolve_table_references( &self, - statement: &datafusion_sql::parser::Statement, + statement: &Statement, ) -> datafusion_common::Result> { let enable_ident_normalization = self.config.options().sql_parser.enable_ident_normalization; @@ -526,7 +526,7 @@ impl SessionState { /// Convert an AST Statement into a LogicalPlan pub async fn statement_to_plan( &self, - statement: datafusion_sql::parser::Statement, + statement: Statement, ) -> datafusion_common::Result { let references = self.resolve_table_references(&statement)?; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index c971e61506339..aa4bcb6837493 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1416,8 +1416,8 @@ pub(crate) mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{ - expressions, expressions::binary, expressions::lit, LexOrdering, - PhysicalSortExpr, PhysicalSortRequirement, + expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, + PhysicalSortRequirement, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::PlanProperties; @@ -1646,8 +1646,7 @@ pub(crate) mod tests { .enumerate() .map(|(index, (_col, name))| { ( - Arc::new(expressions::Column::new(name, index)) - as Arc, + Arc::new(Column::new(name, index)) as Arc, name.clone(), ) }) diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 08740daa0c8e7..9ac75c8f3efb3 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -69,7 +69,7 @@ pub fn create_table_dual() -> Arc { let batch = RecordBatch::try_new( dual_schema.clone(), vec![ - Arc::new(array::Int32Array::from(vec![1])), + Arc::new(Int32Array::from(vec![1])), Arc::new(array::StringArray::from(vec!["a"])), ], ) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 194d864b75c1c..75eb5afb774a0 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -1445,9 +1445,7 @@ async fn unnest_analyze_metrics() -> Result<()> { .explain(false, true)? .collect() .await?; - let formatted = arrow::util::pretty::pretty_format_batches(&results) - .unwrap() - .to_string(); + let formatted = pretty_format_batches(&results).unwrap().to_string(); assert_contains!(&formatted, "elapsed_compute="); assert_contains!(&formatted, "input_batches=1"); assert_contains!(&formatted, "input_rows=5"); diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 800a087587da5..68785b7a5a45c 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -29,10 +29,10 @@ use datafusion_expr::expr::ScalarFunction; use datafusion_expr::logical_plan::builder::table_scan_with_filters; use datafusion_expr::simplify::SimplifyInfo; use datafusion_expr::{ - expr, table_scan, Cast, ColumnarValue, ExprSchemable, LogicalPlan, - LogicalPlanBuilder, ScalarUDF, Volatility, + table_scan, Cast, ColumnarValue, ExprSchemable, LogicalPlan, LogicalPlanBuilder, + ScalarUDF, Volatility, }; -use datafusion_functions::{math, string}; +use datafusion_functions::math; use datafusion_optimizer::optimizer::Optimizer; use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyExpressions}; use datafusion_optimizer::{OptimizerContext, OptimizerRule}; @@ -368,13 +368,13 @@ fn test_const_evaluator() { #[test] fn test_const_evaluator_scalar_functions() { // concat("foo", "bar") --> "foobar" - let expr = string::expr_fn::concat(vec![lit("foo"), lit("bar")]); + let expr = concat(vec![lit("foo"), lit("bar")]); test_evaluate(expr, lit("foobar")); // ensure arguments are also constant folded // concat("foo", concat("bar", "baz")) --> "foobarbaz" - let concat1 = string::expr_fn::concat(vec![lit("bar"), lit("baz")]); - let expr = string::expr_fn::concat(vec![lit("foo"), concat1]); + let concat1 = concat(vec![lit("bar"), lit("baz")]); + let expr = concat(vec![lit("foo"), concat1]); test_evaluate(expr, lit("foobarbaz")); // Check non string arguments @@ -407,7 +407,7 @@ fn test_const_evaluator_scalar_functions() { #[test] fn test_const_evaluator_now() { let ts_nanos = 1599566400000000000i64; - let time = chrono::Utc.timestamp_nanos(ts_nanos); + let time = Utc.timestamp_nanos(ts_nanos); let ts_string = "2020-09-08T12:05:00+00:00"; // now() --> ts test_evaluate_with_start_time(now(), lit_timestamp_nano(ts_nanos), &time); @@ -429,7 +429,7 @@ fn test_evaluator_udfs() { // immutable UDF should get folded // udf_add(1+2, 30+40) --> 73 - let expr = Expr::ScalarFunction(expr::ScalarFunction::new_udf( + let expr = Expr::ScalarFunction(ScalarFunction::new_udf( make_udf_add(Volatility::Immutable), args.clone(), )); @@ -438,21 +438,16 @@ fn test_evaluator_udfs() { // stable UDF should be entirely folded // udf_add(1+2, 30+40) --> 73 let fun = make_udf_add(Volatility::Stable); - let expr = Expr::ScalarFunction(expr::ScalarFunction::new_udf( - Arc::clone(&fun), - args.clone(), - )); + let expr = + Expr::ScalarFunction(ScalarFunction::new_udf(Arc::clone(&fun), args.clone())); test_evaluate(expr, lit(73)); // volatile UDF should have args folded // udf_add(1+2, 30+40) --> udf_add(3, 70) let fun = make_udf_add(Volatility::Volatile); - let expr = - Expr::ScalarFunction(expr::ScalarFunction::new_udf(Arc::clone(&fun), args)); - let expected_expr = Expr::ScalarFunction(expr::ScalarFunction::new_udf( - Arc::clone(&fun), - folded_args, - )); + let expr = Expr::ScalarFunction(ScalarFunction::new_udf(Arc::clone(&fun), args)); + let expected_expr = + Expr::ScalarFunction(ScalarFunction::new_udf(Arc::clone(&fun), folded_args)); test_evaluate(expr, expected_expr); } diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 1035fa31da08c..28901b14b5b7d 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -164,6 +164,8 @@ fn baseline_config() -> DatasetGeneratorConfig { ColumnDescr::new("u16", DataType::UInt16), ColumnDescr::new("u32", DataType::UInt32), ColumnDescr::new("u64", DataType::UInt64), + ColumnDescr::new("date32", DataType::Date32), + ColumnDescr::new("date64", DataType::Date64), // TODO: date/time columns // todo decimal columns // begin string columns diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index 44f96d5a1a073..ef9b5a7f355a7 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -17,6 +17,10 @@ use std::sync::Arc; +use arrow::datatypes::{ + Date32Type, Date64Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, + Int8Type, UInt16Type, UInt32Type, UInt64Type, UInt8Type, +}; use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result}; @@ -222,7 +226,7 @@ macro_rules! generate_string_array { } macro_rules! generate_primitive_array { - ($SELF:ident, $NUM_ROWS:ident, $BATCH_GEN_RNG:ident, $ARRAY_GEN_RNG:ident, $DATA_TYPE:ident) => { + ($SELF:ident, $NUM_ROWS:ident, $BATCH_GEN_RNG:ident, $ARRAY_GEN_RNG:ident, $ARROW_TYPE:ident) => { paste::paste! {{ let null_pct_idx = $BATCH_GEN_RNG.gen_range(0..$SELF.candidate_null_pcts.len()); let null_pct = $SELF.candidate_null_pcts[null_pct_idx]; @@ -239,7 +243,7 @@ macro_rules! generate_primitive_array { rng: $ARRAY_GEN_RNG, }; - generator.[< gen_data_ $DATA_TYPE >]() + generator.gen_data::<$ARROW_TYPE>() }}} } @@ -297,7 +301,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i8 + Int8Type ) } DataType::Int16 => { @@ -306,7 +310,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i16 + Int16Type ) } DataType::Int32 => { @@ -315,7 +319,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i32 + Int32Type ) } DataType::Int64 => { @@ -324,7 +328,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - i64 + Int64Type ) } DataType::UInt8 => { @@ -333,7 +337,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u8 + UInt8Type ) } DataType::UInt16 => { @@ -342,7 +346,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u16 + UInt16Type ) } DataType::UInt32 => { @@ -351,7 +355,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u32 + UInt32Type ) } DataType::UInt64 => { @@ -360,7 +364,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - u64 + UInt64Type ) } DataType::Float32 => { @@ -369,7 +373,7 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - f32 + Float32Type ) } DataType::Float64 => { @@ -378,7 +382,25 @@ impl RecordBatchGenerator { num_rows, batch_gen_rng, array_gen_rng, - f64 + Float64Type + ) + } + DataType::Date32 => { + generate_primitive_array!( + self, + num_rows, + batch_gen_rng, + array_gen_rng, + Date32Type + ) + } + DataType::Date64 => { + generate_primitive_array!( + self, + num_rows, + batch_gen_rng, + array_gen_rng, + Date64Type ) } DataType::Utf8 => { diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs index 898d1081ff13a..0704bafa0318a 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs @@ -104,7 +104,7 @@ impl AggregationFuzzerBuilder { } } -impl std::default::Default for AggregationFuzzerBuilder { +impl Default for AggregationFuzzerBuilder { fn default() -> Self { Self::new() } @@ -375,7 +375,7 @@ pub struct QueryBuilder { } impl QueryBuilder { pub fn new() -> Self { - std::default::Default::default() + Default::default() } /// return the table name if any diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index ca2c2bf4e4387..c8478db22bd4a 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -41,6 +41,7 @@ use datafusion::physical_plan::joins::{ }; use datafusion::physical_plan::memory::MemoryExec; +use crate::fuzz_cases::join_fuzz::JoinTestType::NljHj; use datafusion::prelude::{SessionConfig, SessionContext}; use test_utils::stagger_batch_with_seed; @@ -89,6 +90,7 @@ fn col_lt_col_filter(schema1: Arc, schema2: Arc) -> JoinFilter { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_inner_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -101,6 +103,7 @@ async fn test_inner_join_1k_filtered() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_inner_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -113,6 +116,7 @@ async fn test_inner_join_1k() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_left_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -125,6 +129,7 @@ async fn test_left_join_1k() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_left_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -137,6 +142,7 @@ async fn test_left_join_1k_filtered() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_right_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -149,6 +155,7 @@ async fn test_right_join_1k() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_right_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -161,6 +168,7 @@ async fn test_right_join_1k_filtered() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_full_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -173,6 +181,7 @@ async fn test_full_join_1k() { } #[tokio::test] +#[allow(unused_qualifications)] // flaky for HjSmj case // https://github.com/apache/datafusion/issues/12359 async fn test_full_join_1k_filtered() { @@ -187,6 +196,7 @@ async fn test_full_join_1k_filtered() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_semi_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -199,6 +209,7 @@ async fn test_semi_join_1k() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_semi_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -211,6 +222,7 @@ async fn test_semi_join_1k_filtered() { } #[tokio::test] +#[allow(unused_qualifications)] async fn test_anti_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -223,9 +235,7 @@ async fn test_anti_join_1k() { } #[tokio::test] -// flaky for HjSmj case, giving 1 rows difference sometimes -// https://github.com/apache/datafusion/issues/11555 -#[ignore] +#[allow(unused_qualifications)] async fn test_anti_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), @@ -233,7 +243,7 @@ async fn test_anti_join_1k_filtered() { JoinType::LeftAnti, Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::NljHj], false) + .run_test(&[JoinTestType::HjSmj, NljHj], false) .await } @@ -451,6 +461,7 @@ impl JoinFuzzTestCase { /// `join_tests` - identifies what join types to test /// if `debug` flag is set the test will save randomly generated inputs and outputs to user folders, /// so it is easy to debug a test on top of the failed data + #[allow(unused_qualifications)] async fn run_test(&self, join_tests: &[JoinTestType], debug: bool) { for batch_size in self.batch_sizes { let session_config = SessionConfig::new().with_batch_size(*batch_size); diff --git a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs index 95d97709f3195..c52acdd82764c 100644 --- a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs @@ -341,7 +341,7 @@ async fn run_limit_test(fetch: usize, data: &SortedData) { /// Return random ASCII String with len fn get_random_string(len: usize) -> String { - rand::thread_rng() + thread_rng() .sample_iter(rand::distributions::Alphanumeric) .take(len) .map(char::from) diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index a72affc2b079f..353db86683631 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -174,7 +174,7 @@ mod sp_repartition_fuzz_tests { }) .unzip(); - let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; + let sort_arrs = lexsort(&sort_columns, None)?; for (idx, arr) in izip!(indices, sort_arrs) { schema_vec[idx] = Some(arr); } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index d649919f1b6a6..61b4e32ad6c9e 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -293,7 +293,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { vec![window_expr], memory_exec.clone(), vec![], - InputOrderMode::Linear, + Linear, )?); let task_ctx = ctx.task_ctx(); let mut collected_results = @@ -592,7 +592,7 @@ async fn run_window_test( orderby_columns: Vec<&str>, search_mode: InputOrderMode, ) -> Result<()> { - let is_linear = !matches!(search_mode, InputOrderMode::Sorted); + let is_linear = !matches!(search_mode, Sorted); let mut rng = StdRng::seed_from_u64(random_seed); let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); diff --git a/datafusion/core/tests/macro_hygiene/mod.rs b/datafusion/core/tests/macro_hygiene/mod.rs index 72ac6e64fb0c5..c35e46c0c558f 100644 --- a/datafusion/core/tests/macro_hygiene/mod.rs +++ b/datafusion/core/tests/macro_hygiene/mod.rs @@ -37,3 +37,13 @@ mod plan_datafusion_err { plan_datafusion_err!("foo"); } } + +mod record_batch { + // NO other imports! + use datafusion_common::record_batch; + + #[test] + fn test_macro() { + record_batch!(("column_name", Int32, vec![1, 2, 3])).unwrap(); + } +} diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 18d8300fb254d..4b5d22bfa71ff 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -28,7 +28,6 @@ use datafusion::execution::context::SessionState; use datafusion::prelude::SessionContext; use datafusion_common::stats::Precision; use datafusion_execution::cache::cache_manager::CacheManagerConfig; -use datafusion_execution::cache::cache_unit; use datafusion_execution::cache::cache_unit::{ DefaultFileStatisticsCache, DefaultListFilesCache, }; @@ -211,8 +210,8 @@ fn get_cache_runtime_state() -> ( SessionState, ) { let cache_config = CacheManagerConfig::default(); - let file_static_cache = Arc::new(cache_unit::DefaultFileStatisticsCache::default()); - let list_file_cache = Arc::new(cache_unit::DefaultListFilesCache::default()); + let file_static_cache = Arc::new(DefaultFileStatisticsCache::default()); + let list_file_cache = Arc::new(DefaultListFilesCache::default()); let cache_config = cache_config .with_files_statistics_cache(Some(file_static_cache.clone())) diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index d6991711f581c..6859e2f1468ce 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -375,7 +375,7 @@ fn test_has_filter() -> Result<()> { // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec // the `a > 1` filter is applied in the AggregateExec let filter_expr = Some(expressions::binary( - expressions::col("a", &schema)?, + col("a", &schema)?, Operator::Gt, cast(expressions::lit(1u32), &schema, DataType::Int32)?, &schema, @@ -408,7 +408,7 @@ fn test_has_filter() -> Result<()> { #[test] fn test_has_order_by() -> Result<()> { let sort_key = vec![PhysicalSortExpr { - expr: expressions::col("a", &schema()).unwrap(), + expr: col("a", &schema()).unwrap(), options: SortOptions::default(), }]; let source = parquet_exec_with_sort(vec![sort_key]); diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index addabc8a36127..fab92c0f9c2bf 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -33,7 +33,7 @@ async fn join_change_in_planner() -> Result<()> { Field::new("a2", DataType::UInt32, false), ])); // Specify the ordering: - let file_sort_order = vec![[datafusion_expr::col("a1")] + let file_sort_order = vec![[col("a1")] .into_iter() .map(|e| { let ascending = true; @@ -101,7 +101,7 @@ async fn join_no_order_on_filter() -> Result<()> { Field::new("a3", DataType::UInt32, false), ])); // Specify the ordering: - let file_sort_order = vec![[datafusion_expr::col("a1")] + let file_sort_order = vec![[col("a1")] .into_iter() .map(|e| { let ascending = true; diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index dc9d047860213..177427b47d218 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -65,7 +65,7 @@ pub mod select; mod sql_api; async fn register_aggregate_csv_by_sql(ctx: &SessionContext) { - let testdata = datafusion::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let df = ctx .sql(&format!( @@ -103,7 +103,7 @@ async fn register_aggregate_csv_by_sql(ctx: &SessionContext) { } async fn register_aggregate_csv(ctx: &SessionContext) -> Result<()> { - let testdata = datafusion::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let schema = test_util::aggr_test_schema(); ctx.register_csv( "aggregate_test_100", @@ -227,7 +227,7 @@ fn result_vec(results: &[RecordBatch]) -> Vec> { } async fn register_alltypes_parquet(ctx: &SessionContext) { - let testdata = datafusion::test_util::parquet_test_data(); + let testdata = test_util::parquet_test_data(); ctx.register_parquet( "alltypes_plain", &format!("{testdata}/alltypes_plain.parquet"), diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 1e0d3d9d514e8..497addd23094a 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -747,7 +747,7 @@ impl Accumulator for FirstSelector { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } @@ -816,7 +816,7 @@ impl Accumulator for TestGroupsAccumulator { } fn size(&self) -> usize { - std::mem::size_of::() + size_of::() } fn state(&mut self) -> Result> { @@ -864,6 +864,6 @@ impl GroupsAccumulator for TestGroupsAccumulator { } fn size(&self) -> usize { - std::mem::size_of::() + size_of::() } } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 6c4e3c66e3975..c962567844022 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -513,11 +513,7 @@ impl Debug for TopKExec { } impl DisplayAs for TopKExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!(f, "TopKExec: k={}", self.k) diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 0887645b8cbf0..f1b1728623998 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -936,11 +936,11 @@ struct ScalarFunctionWrapper { name: String, expr: Expr, signature: Signature, - return_type: arrow_schema::DataType, + return_type: DataType, } impl ScalarUDFImpl for ScalarFunctionWrapper { - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } @@ -948,21 +948,15 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { &self.name } - fn signature(&self) -> &datafusion_expr::Signature { + fn signature(&self) -> &Signature { &self.signature } - fn return_type( - &self, - _arg_types: &[arrow_schema::DataType], - ) -> Result { + fn return_type(&self, _arg_types: &[DataType]) -> Result { Ok(self.return_type.clone()) } - fn invoke( - &self, - _args: &[datafusion_expr::ColumnarValue], - ) -> Result { + fn invoke(&self, _args: &[ColumnarValue]) -> Result { internal_err!("This function should not get invoked!") } @@ -1042,10 +1036,7 @@ impl TryFrom for ScalarFunctionWrapper { .into_iter() .map(|a| a.data_type) .collect(), - definition - .params - .behavior - .unwrap_or(datafusion_expr::Volatility::Volatile), + definition.params.behavior.unwrap_or(Volatility::Volatile), ), }) } @@ -1350,7 +1341,7 @@ fn custom_sqrt(args: &[ColumnarValue]) -> Result { } async fn register_aggregate_csv(ctx: &SessionContext) -> Result<()> { - let testdata = datafusion::test_util::arrow_test_data(); + let testdata = test_util::arrow_test_data(); let schema = test_util::aggr_test_schema(); ctx.register_csv( "aggregate_test_100", @@ -1362,7 +1353,7 @@ async fn register_aggregate_csv(ctx: &SessionContext) -> Result<()> { } async fn register_alltypes_parquet(ctx: &SessionContext) -> Result<()> { - let testdata = datafusion::test_util::parquet_test_data(); + let testdata = test_util::parquet_test_data(); ctx.register_parquet( "alltypes_plain", &format!("{testdata}/alltypes_plain.parquet"), diff --git a/datafusion/core/tests/user_defined/user_defined_window_functions.rs b/datafusion/core/tests/user_defined/user_defined_window_functions.rs index 3760328934bc4..8fe028eedd443 100644 --- a/datafusion/core/tests/user_defined/user_defined_window_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_window_functions.rs @@ -593,11 +593,7 @@ impl PartitionEvaluator for OddCounter { Ok(scalar) } - fn evaluate_all( - &mut self, - values: &[arrow_array::ArrayRef], - num_rows: usize, - ) -> Result { + fn evaluate_all(&mut self, values: &[ArrayRef], num_rows: usize) -> Result { println!("evaluate_all, values: {values:#?}, num_rows: {num_rows}"); self.test_state.inc_evaluate_all_called(); @@ -641,7 +637,7 @@ fn odd_count(arr: &Int64Array) -> i64 { } /// returns an array of num_rows that has the number of odd values in `arr` -fn odd_count_arr(arr: &Int64Array, num_rows: usize) -> arrow_array::ArrayRef { +fn odd_count_arr(arr: &Int64Array, num_rows: usize) -> ArrayRef { let array: Int64Array = std::iter::repeat(odd_count(arr)).take(num_rows).collect(); Arc::new(array) } diff --git a/datafusion/execution/src/disk_manager.rs b/datafusion/execution/src/disk_manager.rs index c98d7e5579f0f..38c259fcbdc8e 100644 --- a/datafusion/execution/src/disk_manager.rs +++ b/datafusion/execution/src/disk_manager.rs @@ -173,7 +173,7 @@ fn create_local_dirs(local_dirs: Vec) -> Result>> { local_dirs .iter() .map(|root| { - if !std::path::Path::new(root).exists() { + if !Path::new(root).exists() { std::fs::create_dir(root)?; } Builder::new() diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index d87ce1ebfed75..5bf30b724d0b9 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -334,13 +334,17 @@ impl Drop for MemoryReservation { } } -const TB: u64 = 1 << 40; -const GB: u64 = 1 << 30; -const MB: u64 = 1 << 20; -const KB: u64 = 1 << 10; +pub mod units { + pub const TB: u64 = 1 << 40; + pub const GB: u64 = 1 << 30; + pub const MB: u64 = 1 << 20; + pub const KB: u64 = 1 << 10; +} /// Present size in human readable form pub fn human_readable_size(size: usize) -> String { + use units::*; + let size = size as u64; let (value, unit) = { if size >= 2 * TB { diff --git a/datafusion/expr-common/src/columnar_value.rs b/datafusion/expr-common/src/columnar_value.rs index 57056d0806a74..4b9454ed739d7 100644 --- a/datafusion/expr-common/src/columnar_value.rs +++ b/datafusion/expr-common/src/columnar_value.rs @@ -19,7 +19,7 @@ use arrow::array::{Array, ArrayRef, NullArray}; use arrow::compute::{kernels, CastOptions}; -use arrow::datatypes::{DataType, TimeUnit}; +use arrow::datatypes::DataType; use datafusion_common::format::DEFAULT_CAST_OPTIONS; use datafusion_common::{internal_err, Result, ScalarValue}; use std::sync::Arc; @@ -193,28 +193,9 @@ impl ColumnarValue { ColumnarValue::Array(array) => Ok(ColumnarValue::Array( kernels::cast::cast_with_options(array, cast_type, &cast_options)?, )), - ColumnarValue::Scalar(scalar) => { - let scalar_array = - if cast_type == &DataType::Timestamp(TimeUnit::Nanosecond, None) { - if let ScalarValue::Float64(Some(float_ts)) = scalar { - ScalarValue::Int64(Some( - (float_ts * 1_000_000_000_f64).trunc() as i64, - )) - .to_array()? - } else { - scalar.to_array()? - } - } else { - scalar.to_array()? - }; - let cast_array = kernels::cast::cast_with_options( - &scalar_array, - cast_type, - &cast_options, - )?; - let cast_scalar = ScalarValue::try_from_array(&cast_array, 0)?; - Ok(ColumnarValue::Scalar(cast_scalar)) - } + ColumnarValue::Scalar(scalar) => Ok(ColumnarValue::Scalar( + scalar.cast_to_with_options(cast_type, &cast_options)?, + )), } } } diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index 2f806bf76d166..31fe6a59baee7 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -89,7 +89,7 @@ fn signature(lhs: &DataType, op: &Operator, rhs: &DataType) -> Result And | Or => if matches!((lhs, rhs), (Boolean | Null, Boolean | Null)) { // Logical binary boolean operators can only be evaluated for // boolean or null arguments. - Ok(Signature::uniform(DataType::Boolean)) + Ok(Signature::uniform(Boolean)) } else { plan_err!( "Cannot infer common argument type for logical boolean operation {lhs} {op} {rhs}" @@ -1225,9 +1225,9 @@ pub fn like_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Option { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { - (DataType::Null, Utf8View | Utf8 | LargeUtf8) => Some(rhs_type.clone()), - (Utf8View | Utf8 | LargeUtf8, DataType::Null) => Some(lhs_type.clone()), - (DataType::Null, DataType::Null) => Some(Utf8), + (Null, Utf8View | Utf8 | LargeUtf8) => Some(rhs_type.clone()), + (Utf8View | Utf8 | LargeUtf8, Null) => Some(lhs_type.clone()), + (Null, Null) => Some(Utf8), _ => None, } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 4d73c2a044867..bda4d7ae3d7fa 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -29,8 +29,8 @@ use crate::logical_plan::Subquery; use crate::utils::expr_to_columns; use crate::Volatility; use crate::{ - built_in_window_function, udaf, BuiltInWindowFunction, ExprSchemable, Operator, - Signature, WindowFrame, WindowUDF, + udaf, BuiltInWindowFunction, ExprSchemable, Operator, Signature, WindowFrame, + WindowUDF, }; use arrow::datatypes::{DataType, FieldRef}; @@ -695,11 +695,11 @@ impl AggregateFunction { pub enum WindowFunctionDefinition { /// A built in aggregate function that leverages an aggregate function /// A a built-in window function - BuiltInWindowFunction(built_in_window_function::BuiltInWindowFunction), + BuiltInWindowFunction(BuiltInWindowFunction), /// A user defined aggregate function AggregateUDF(Arc), /// A user defined aggregate function - WindowUDF(Arc), + WindowUDF(Arc), } impl WindowFunctionDefinition { @@ -742,14 +742,12 @@ impl WindowFunctionDefinition { } } -impl fmt::Display for WindowFunctionDefinition { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { +impl Display for WindowFunctionDefinition { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { - WindowFunctionDefinition::BuiltInWindowFunction(fun) => { - std::fmt::Display::fmt(fun, f) - } - WindowFunctionDefinition::AggregateUDF(fun) => std::fmt::Display::fmt(fun, f), - WindowFunctionDefinition::WindowUDF(fun) => std::fmt::Display::fmt(fun, f), + WindowFunctionDefinition::BuiltInWindowFunction(fun) => Display::fmt(fun, f), + WindowFunctionDefinition::AggregateUDF(fun) => Display::fmt(fun, f), + WindowFunctionDefinition::WindowUDF(fun) => Display::fmt(fun, f), } } } @@ -833,9 +831,7 @@ pub fn find_df_window_func(name: &str) -> Option { // may have different implementations for these cases. If the sought // function is not found among built-in window functions, we search for // it among aggregate functions. - if let Ok(built_in_function) = - built_in_window_function::BuiltInWindowFunction::from_str(name.as_str()) - { + if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { Some(WindowFunctionDefinition::BuiltInWindowFunction( built_in_function, )) @@ -2141,8 +2137,8 @@ pub fn schema_name_from_sorts(sorts: &[Sort]) -> Result { /// Format expressions for display as part of a logical plan. In many cases, this will produce /// similar output to `Expr.name()` except that column names will be prefixed with '#'. -impl fmt::Display for Expr { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { +impl Display for Expr { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { Expr::Alias(Alias { expr, name, .. }) => write!(f, "{expr} AS {name}"), Expr::Column(c) => write!(f, "{c}"), @@ -2346,7 +2342,7 @@ impl fmt::Display for Expr { } fn fmt_function( - f: &mut fmt::Formatter, + f: &mut Formatter, fun: &str, distinct: bool, args: &[Expr], @@ -2588,13 +2584,13 @@ mod test { assert_eq!( find_df_window_func("first_value"), Some(WindowFunctionDefinition::BuiltInWindowFunction( - built_in_window_function::BuiltInWindowFunction::FirstValue + BuiltInWindowFunction::FirstValue )) ); assert_eq!( find_df_window_func("LAST_value"), Some(WindowFunctionDefinition::BuiltInWindowFunction( - built_in_window_function::BuiltInWindowFunction::LastValue + BuiltInWindowFunction::LastValue )) ); assert_eq!(find_df_window_func("not_exist"), None) diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index 199a91bf5ace1..23ffc83e3549c 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -27,6 +27,7 @@ pub use datafusion_functions_aggregate_common::accumulator::{ AccumulatorArgs, AccumulatorFactoryFunction, StateFieldsArgs, }; +pub use datafusion_functions_window_common::expr::ExpressionArgs; pub use datafusion_functions_window_common::field::WindowUDFFieldArgs; pub use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index aef531a9dbf71..2547aa23d3cdf 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1482,6 +1482,15 @@ pub fn validate_unique_names<'a>( /// [`TypeCoercionRewriter::coerce_union`]: https://docs.rs/datafusion-optimizer/latest/datafusion_optimizer/analyzer/type_coercion/struct.TypeCoercionRewriter.html#method.coerce_union /// [`coerce_union_schema`]: https://docs.rs/datafusion-optimizer/latest/datafusion_optimizer/analyzer/type_coercion/fn.coerce_union_schema.html pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result { + if left_plan.schema().fields().len() != right_plan.schema().fields().len() { + return plan_err!( + "UNION queries have different number of columns: \ + left has {} columns whereas right has {} columns", + left_plan.schema().fields().len(), + right_plan.schema().fields().len() + ); + } + // Temporarily use the schema from the left input and later rely on the analyzer to // coerce the two schemas into a common one. @@ -1669,7 +1678,7 @@ impl TableSource for LogicalTableSource { fn supports_filters_pushdown( &self, filters: &[&Expr], - ) -> Result> { + ) -> Result> { Ok(vec![TableProviderFilterPushDown::Exact; filters.len()]) } } diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index c4fa9f4c3feda..93e8b5fd045e7 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -120,7 +120,7 @@ impl DdlStatement { /// children. /// /// See [crate::LogicalPlan::display] for an example - pub fn display(&self) -> impl fmt::Display + '_ { + pub fn display(&self) -> impl Display + '_ { struct Wrapper<'a>(&'a DdlStatement); impl<'a> Display for Wrapper<'a> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { diff --git a/datafusion/expr/src/logical_plan/dml.rs b/datafusion/expr/src/logical_plan/dml.rs index 68b3ac41fa083..669bc8e8a7d34 100644 --- a/datafusion/expr/src/logical_plan/dml.rs +++ b/datafusion/expr/src/logical_plan/dml.rs @@ -165,7 +165,7 @@ impl WriteOp { } impl Display for WriteOp { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { write!(f, "{}", self.name()) } } @@ -196,7 +196,7 @@ impl InsertOp { } impl Display for InsertOp { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { write!(f, "{}", self.name()) } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 572285defba03..a301c48659d7c 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -3382,8 +3382,8 @@ pub struct ColumnUnnestList { pub depth: usize, } -impl fmt::Display for ColumnUnnestList { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { +impl Display for ColumnUnnestList { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { write!(f, "{}|depth={}", self.output_column, self.depth) } } diff --git a/datafusion/expr/src/logical_plan/statement.rs b/datafusion/expr/src/logical_plan/statement.rs index ed06375157c94..7ad18ce7bbf77 100644 --- a/datafusion/expr/src/logical_plan/statement.rs +++ b/datafusion/expr/src/logical_plan/statement.rs @@ -61,7 +61,7 @@ impl Statement { /// children. /// /// See [crate::LogicalPlan::display] for an example - pub fn display(&self) -> impl fmt::Display + '_ { + pub fn display(&self) -> impl Display + '_ { struct Wrapper<'a>(&'a Statement); impl<'a> Display for Wrapper<'a> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index b4f768085fcc3..262aa99e50075 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -34,7 +34,6 @@ use crate::{ function::{AccumulatorArgs, StateFieldsArgs}, utils::AggregateOrderSensitivity, Accumulator, AggregateUDFImpl, Expr, GroupsAccumulator, ReversedUDAF, Signature, - Volatility, }; macro_rules! create_func { @@ -106,7 +105,7 @@ pub struct Sum { impl Sum { pub fn new() -> Self { Self { - signature: Signature::user_defined(Volatility::Immutable), + signature: Signature::user_defined(Immutable), } } } @@ -236,13 +235,13 @@ impl Count { pub fn new() -> Self { Self { aliases: vec!["count".to_string()], - signature: Signature::variadic_any(Volatility::Immutable), + signature: Signature::variadic_any(Immutable), } } } impl AggregateUDFImpl for Count { - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } @@ -318,13 +317,13 @@ impl Default for Min { impl Min { pub fn new() -> Self { Self { - signature: Signature::variadic_any(Volatility::Immutable), + signature: Signature::variadic_any(Immutable), } } } impl AggregateUDFImpl for Min { - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } @@ -403,13 +402,13 @@ impl Default for Max { impl Max { pub fn new() -> Self { Self { - signature: Signature::variadic_any(Volatility::Immutable), + signature: Signature::variadic_any(Immutable), } } } impl AggregateUDFImpl for Max { - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } diff --git a/datafusion/expr/src/udf_docs.rs b/datafusion/expr/src/udf_docs.rs index 63d1a964345db..a124361e42a3d 100644 --- a/datafusion/expr/src/udf_docs.rs +++ b/datafusion/expr/src/udf_docs.rs @@ -47,6 +47,8 @@ pub struct Documentation { /// Left member of a pair is the argument name, right is a /// description for the argument pub arguments: Option>, + /// A list of alternative syntax examples for a function + pub alternative_syntax: Option>, /// Related functions if any. Values should match the related /// udf's name exactly. Related udf's must be of the same /// UDF type (scalar, aggregate or window) for proper linking to @@ -96,6 +98,7 @@ pub struct DocumentationBuilder { pub syntax_example: Option, pub sql_example: Option, pub arguments: Option>, + pub alternative_syntax: Option>, pub related_udfs: Option>, } @@ -107,6 +110,7 @@ impl DocumentationBuilder { syntax_example: None, sql_example: None, arguments: None, + alternative_syntax: None, related_udfs: None, } } @@ -172,6 +176,13 @@ impl DocumentationBuilder { self.with_argument(arg_name, description) } + pub fn with_alternative_syntax(mut self, syntax_name: impl Into) -> Self { + let mut alternative_syntax_array = self.alternative_syntax.unwrap_or_default(); + alternative_syntax_array.push(syntax_name.into()); + self.alternative_syntax = Some(alternative_syntax_array); + self + } + pub fn with_related_udf(mut self, related_udf: impl Into) -> Self { let mut related = self.related_udfs.unwrap_or_default(); related.push(related_udf.into()); @@ -186,6 +197,7 @@ impl DocumentationBuilder { syntax_example, sql_example, arguments, + alternative_syntax, related_udfs, } = self; @@ -205,6 +217,7 @@ impl DocumentationBuilder { syntax_example: syntax_example.unwrap(), sql_example, arguments, + alternative_syntax, related_udfs, }) } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index bb5496c0f7998..29c62440abb12 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -981,6 +981,7 @@ pub fn can_hash(data_type: &DataType) -> bool { }, DataType::Utf8 => true, DataType::LargeUtf8 => true, + DataType::Utf8View => true, DataType::Decimal128(_, _) => true, DataType::Date32 => true, DataType::Date64 => true, @@ -1398,7 +1399,7 @@ pub fn format_state_name(name: &str, state_name: &str) -> String { mod tests { use super::*; use crate::{ - col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, + col, cube, expr_vec_fmt, grouping_set, lit, rollup, test::function_stub::max_udaf, test::function_stub::min_udaf, test::function_stub::sum_udaf, Cast, ExprFunctionExt, WindowFunctionDefinition, }; @@ -1413,19 +1414,19 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys_empty_window() -> Result<()> { - let max1 = Expr::WindowFunction(expr::WindowFunction::new( + let max1 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], )); - let max2 = Expr::WindowFunction(expr::WindowFunction::new( + let max2 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], )); - let min3 = Expr::WindowFunction(expr::WindowFunction::new( + let min3 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(min_udaf()), vec![col("name")], )); - let sum4 = Expr::WindowFunction(expr::WindowFunction::new( + let sum4 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(sum_udaf()), vec![col("age")], )); @@ -1440,28 +1441,28 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys() -> Result<()> { - let age_asc = expr::Sort::new(col("age"), true, true); - let name_desc = expr::Sort::new(col("name"), false, true); - let created_at_desc = expr::Sort::new(col("created_at"), false, true); - let max1 = Expr::WindowFunction(expr::WindowFunction::new( + let age_asc = Sort::new(col("age"), true, true); + let name_desc = Sort::new(col("name"), false, true); + let created_at_desc = Sort::new(col("created_at"), false, true); + let max1 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], )) .order_by(vec![age_asc.clone(), name_desc.clone()]) .build() .unwrap(); - let max2 = Expr::WindowFunction(expr::WindowFunction::new( + let max2 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], )); - let min3 = Expr::WindowFunction(expr::WindowFunction::new( + let min3 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(min_udaf()), vec![col("name")], )) .order_by(vec![age_asc.clone(), name_desc.clone()]) .build() .unwrap(); - let sum4 = Expr::WindowFunction(expr::WindowFunction::new( + let sum4 = Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(sum_udaf()), vec![col("age")], )) diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 349968c3fa2f4..222914315d700 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -94,7 +94,7 @@ pub struct WindowFrame { } impl fmt::Display for WindowFrame { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { write!( f, "{} BETWEEN {} AND {}", @@ -416,7 +416,7 @@ fn convert_frame_bound_to_scalar_value( } impl fmt::Display for WindowFrameBound { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self { WindowFrameBound::Preceding(n) => { if n.is_null() { @@ -457,7 +457,7 @@ pub enum WindowFrameUnits { } impl fmt::Display for WindowFrameUnits { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { f.write_str(match self { WindowFrameUnits::Rows => "ROWS", WindowFrameUnits::Range => "RANGE", diff --git a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs index ee61128979e10..07fa4efc990e5 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/bytes.rs @@ -25,6 +25,7 @@ use datafusion_expr_common::accumulator::Accumulator; use datafusion_physical_expr_common::binary_map::{ArrowBytesSet, OutputType}; use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewSet; use std::fmt::Debug; +use std::mem::size_of_val; use std::sync::Arc; /// Specialized implementation of @@ -86,7 +87,7 @@ impl Accumulator for BytesDistinctCountAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + self.0.size() + size_of_val(self) + self.0.size() } } @@ -146,6 +147,6 @@ impl Accumulator for BytesViewDistinctCountAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + self.0.size() + size_of_val(self) + self.0.size() } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs index d128a8af58eef..405b2c2db7bdd 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/native.rs @@ -23,6 +23,7 @@ use std::collections::HashSet; use std::fmt::Debug; use std::hash::Hash; +use std::mem::size_of_val; use std::sync::Arc; use ahash::RandomState; @@ -117,8 +118,7 @@ where fn size(&self) -> usize { let num_elements = self.values.len(); - let fixed_size = - std::mem::size_of_val(self) + std::mem::size_of_val(&self.values); + let fixed_size = size_of_val(self) + size_of_val(&self.values); estimate_memory_size::(num_elements, fixed_size).unwrap() } @@ -206,8 +206,7 @@ where fn size(&self) -> usize { let num_elements = self.values.len(); - let fixed_size = - std::mem::size_of_val(self) + std::mem::size_of_val(&self.values); + let fixed_size = size_of_val(self) + size_of_val(&self.values); estimate_memory_size::(num_elements, fixed_size).unwrap() } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index c936c80cbed71..03e4ef557269f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -23,6 +23,8 @@ pub mod bool_op; pub mod nulls; pub mod prim_op; +use std::mem::{size_of, size_of_val}; + use arrow::array::new_empty_array; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, @@ -122,9 +124,7 @@ impl AccumulatorState { /// Returns the amount of memory taken by this structure and its accumulator fn size(&self) -> usize { - self.accumulator.size() - + std::mem::size_of_val(self) - + self.indices.allocated_size() + self.accumulator.size() + size_of_val(self) + self.indices.allocated_size() } } @@ -464,7 +464,7 @@ pub trait VecAllocExt { impl VecAllocExt for Vec { type T = T; fn allocated_size(&self) -> usize { - std::mem::size_of::() * self.capacity() + size_of::() * self.capacity() } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs index 8bbcf756c37c1..078982c983fc7 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::mem::size_of; use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; @@ -195,6 +196,6 @@ where } fn size(&self) -> usize { - self.values.capacity() * std::mem::size_of::() + self.null_state.size() + self.values.capacity() * size_of::() + self.null_state.size() } } diff --git a/datafusion/functions-aggregate-common/src/tdigest.rs b/datafusion/functions-aggregate-common/src/tdigest.rs index e6723b54b3727..786d7ea3e3610 100644 --- a/datafusion/functions-aggregate-common/src/tdigest.rs +++ b/datafusion/functions-aggregate-common/src/tdigest.rs @@ -33,6 +33,7 @@ use datafusion_common::cast::as_primitive_array; use datafusion_common::Result; use datafusion_common::ScalarValue; use std::cmp::Ordering; +use std::mem::{size_of, size_of_val}; pub const DEFAULT_MAX_SIZE: usize = 100; @@ -203,8 +204,7 @@ impl TDigest { /// Size in bytes including `Self`. pub fn size(&self) -> usize { - std::mem::size_of_val(self) - + (std::mem::size_of::() * self.centroids.capacity()) + size_of_val(self) + (size_of::() * self.centroids.capacity()) } } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 83b9f714fa896..53fcfd641ddfc 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -17,6 +17,7 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; +use std::mem::size_of_val; use std::sync::{Arc, OnceLock}; use arrow::array::{Array, RecordBatch}; @@ -486,10 +487,9 @@ impl Accumulator for ApproxPercentileAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + self.digest.size() - - std::mem::size_of_val(&self.digest) + size_of_val(self) + self.digest.size() - size_of_val(&self.digest) + self.return_type.size() - - std::mem::size_of_val(&self.return_type) + - size_of_val(&self.return_type) } } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs index b86fec1e037ed..5458d0f792b92 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs @@ -17,6 +17,7 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; +use std::mem::size_of_val; use std::sync::{Arc, OnceLock}; use arrow::{ @@ -239,8 +240,7 @@ impl Accumulator for ApproxPercentileWithWeightAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - - std::mem::size_of_val(&self.approx_percentile_cont_accumulator) + size_of_val(self) - size_of_val(&self.approx_percentile_cont_accumulator) + self.approx_percentile_cont_accumulator.size() } } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 6f523756832e9..b3e04c5584ef8 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -34,6 +34,7 @@ use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; use datafusion_functions_aggregate_common::utils::ordering_fields; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use std::collections::{HashSet, VecDeque}; +use std::mem::{size_of, size_of_val}; use std::sync::{Arc, OnceLock}; make_udaf_expr_and_func!( @@ -245,15 +246,15 @@ impl Accumulator for ArrayAggAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - + (std::mem::size_of::() * self.values.capacity()) + size_of_val(self) + + (size_of::() * self.values.capacity()) + self .values .iter() .map(|arr| arr.get_array_memory_size()) .sum::() + self.datatype.size() - - std::mem::size_of_val(&self.datatype) + - size_of_val(&self.datatype) } } @@ -318,10 +319,10 @@ impl Accumulator for DistinctArrayAggAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + ScalarValue::size_of_hashset(&self.values) - - std::mem::size_of_val(&self.values) + size_of_val(self) + ScalarValue::size_of_hashset(&self.values) + - size_of_val(&self.values) + self.datatype.size() - - std::mem::size_of_val(&self.datatype) + - size_of_val(&self.datatype) } } @@ -486,25 +487,23 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } fn size(&self) -> usize { - let mut total = std::mem::size_of_val(self) - + ScalarValue::size_of_vec(&self.values) - - std::mem::size_of_val(&self.values); + let mut total = size_of_val(self) + ScalarValue::size_of_vec(&self.values) + - size_of_val(&self.values); // Add size of the `self.ordering_values` - total += - std::mem::size_of::>() * self.ordering_values.capacity(); + total += size_of::>() * self.ordering_values.capacity(); for row in &self.ordering_values { - total += ScalarValue::size_of_vec(row) - std::mem::size_of_val(row); + total += ScalarValue::size_of_vec(row) - size_of_val(row); } // Add size of the `self.datatypes` - total += std::mem::size_of::() * self.datatypes.capacity(); + total += size_of::() * self.datatypes.capacity(); for dtype in &self.datatypes { - total += dtype.size() - std::mem::size_of_val(dtype); + total += dtype.size() - size_of_val(dtype); } // Add size of the `self.ordering_req` - total += std::mem::size_of::() * self.ordering_req.capacity(); + total += size_of::() * self.ordering_req.capacity(); // TODO: Calculate size of each `PhysicalSortExpr` more accurately. total } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 67b824c2ea792..710b7e69ac5c3 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -18,8 +18,8 @@ //! Defines `Avg` & `Mean` aggregate & accumulators use arrow::array::{ - self, Array, ArrayRef, ArrowNativeTypeOp, ArrowNumericType, ArrowPrimitiveType, - AsArray, BooleanArray, PrimitiveArray, PrimitiveBuilder, UInt64Array, + Array, ArrayRef, ArrowNativeTypeOp, ArrowNumericType, ArrowPrimitiveType, AsArray, + BooleanArray, PrimitiveArray, PrimitiveBuilder, UInt64Array, }; use arrow::compute::sum; @@ -47,6 +47,7 @@ use datafusion_functions_aggregate_common::utils::DecimalAverager; use log::debug; use std::any::Any; use std::fmt::Debug; +use std::mem::{size_of, size_of_val}; use std::sync::{Arc, OnceLock}; make_udaf_expr_and_func!( @@ -294,7 +295,7 @@ impl Accumulator for AvgAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { @@ -372,7 +373,7 @@ impl Accumulator for DecimalAvgAccumu } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { @@ -471,7 +472,7 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&array::BooleanArray>, + opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 1, "single argument to update_batch"); @@ -554,7 +555,7 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&array::BooleanArray>, + opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 2, "two arguments to merge_batch"); @@ -614,7 +615,6 @@ where } fn size(&self) -> usize { - self.counts.capacity() * std::mem::size_of::() - + self.sums.capacity() * std::mem::size_of::() + self.counts.capacity() * size_of::() + self.sums.capacity() * size_of::() } } diff --git a/datafusion/functions-aggregate/src/bit_and_or_xor.rs b/datafusion/functions-aggregate/src/bit_and_or_xor.rs index 0a281ad814679..249ff02e72221 100644 --- a/datafusion/functions-aggregate/src/bit_and_or_xor.rs +++ b/datafusion/functions-aggregate/src/bit_and_or_xor.rs @@ -20,6 +20,7 @@ use std::any::Any; use std::collections::HashSet; use std::fmt::{Display, Formatter}; +use std::mem::{size_of, size_of_val}; use ahash::RandomState; use arrow::array::{downcast_integer, Array, ArrayRef, AsArray}; @@ -347,7 +348,7 @@ where } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { @@ -392,7 +393,7 @@ where } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { @@ -446,7 +447,7 @@ where } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { @@ -509,8 +510,7 @@ where } fn size(&self) -> usize { - std::mem::size_of_val(self) - + self.values.capacity() * std::mem::size_of::() + size_of_val(self) + self.values.capacity() * size_of::() } fn state(&mut self) -> Result> { diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs index b410bfa139e9a..87293ccfa21f5 100644 --- a/datafusion/functions-aggregate/src/bool_and_or.rs +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -18,6 +18,7 @@ //! Defines physical expressions that can evaluated at runtime during query execution use std::any::Any; +use std::mem::size_of_val; use std::sync::OnceLock; use arrow::array::ArrayRef; @@ -229,7 +230,7 @@ impl Accumulator for BoolAndAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { @@ -378,7 +379,7 @@ impl Accumulator for BoolOrAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn state(&mut self) -> Result> { diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index 40429289d768a..187a43ecbea3c 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -19,6 +19,7 @@ use std::any::Any; use std::fmt::Debug; +use std::mem::size_of_val; use std::sync::{Arc, OnceLock}; use arrow::compute::{and, filter, is_not_null}; @@ -204,11 +205,10 @@ impl Accumulator for CorrelationAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.covar) - + self.covar.size() - - std::mem::size_of_val(&self.stddev1) + size_of_val(self) - size_of_val(&self.covar) + self.covar.size() + - size_of_val(&self.stddev1) + self.stddev1.size() - - std::mem::size_of_val(&self.stddev2) + - size_of_val(&self.stddev2) + self.stddev2.size() } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index b4eeb937d4fb4..bade589a908a7 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -21,6 +21,7 @@ use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewD use datafusion_physical_expr::expressions; use std::collections::HashSet; use std::fmt::Debug; +use std::mem::{size_of, size_of_val}; use std::ops::BitAnd; use std::sync::{Arc, OnceLock}; @@ -394,7 +395,7 @@ impl Accumulator for CountAccumulator { fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { let counts = downcast_value!(states[0], Int64Array); - let delta = &arrow::compute::sum(counts); + let delta = &compute::sum(counts); if let Some(d) = delta { self.count += *d; } @@ -410,7 +411,7 @@ impl Accumulator for CountAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } @@ -583,7 +584,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { } fn size(&self) -> usize { - self.counts.capacity() * std::mem::size_of::() + self.counts.capacity() * size_of::() } } @@ -627,28 +628,28 @@ impl DistinctCountAccumulator { // number of batches This method is faster than .full_size(), however it is // not suitable for variable length values like strings or complex types fn fixed_size(&self) -> usize { - std::mem::size_of_val(self) - + (std::mem::size_of::() * self.values.capacity()) + size_of_val(self) + + (size_of::() * self.values.capacity()) + self .values .iter() .next() - .map(|vals| ScalarValue::size(vals) - std::mem::size_of_val(vals)) + .map(|vals| ScalarValue::size(vals) - size_of_val(vals)) .unwrap_or(0) - + std::mem::size_of::() + + size_of::() } // calculates the size as accurately as possible. Note that calling this // method is expensive fn full_size(&self) -> usize { - std::mem::size_of_val(self) - + (std::mem::size_of::() * self.values.capacity()) + size_of_val(self) + + (size_of::() * self.values.capacity()) + self .values .iter() - .map(|vals| ScalarValue::size(vals) - std::mem::size_of_val(vals)) + .map(|vals| ScalarValue::size(vals) - size_of_val(vals)) .sum::() - + std::mem::size_of::() + + size_of::() } } diff --git a/datafusion/functions-aggregate/src/covariance.rs b/datafusion/functions-aggregate/src/covariance.rs index 4b2b21059d16f..063aaa92059dd 100644 --- a/datafusion/functions-aggregate/src/covariance.rs +++ b/datafusion/functions-aggregate/src/covariance.rs @@ -18,6 +18,7 @@ //! [`CovarianceSample`]: covariance sample aggregations. use std::fmt::Debug; +use std::mem::size_of_val; use std::sync::OnceLock; use arrow::{ @@ -448,6 +449,6 @@ impl Accumulator for CovarianceAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index c708d23ae6c59..da3fc62f8c8c6 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -19,6 +19,7 @@ use std::any::Any; use std::fmt::Debug; +use std::mem::size_of_val; use std::sync::{Arc, OnceLock}; use arrow::array::{ArrayRef, AsArray, BooleanArray}; @@ -365,10 +366,10 @@ impl Accumulator for FirstValueAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.first) + size_of_val(self) - size_of_val(&self.first) + self.first.size() + ScalarValue::size_of_vec(&self.orderings) - - std::mem::size_of_val(&self.orderings) + - size_of_val(&self.orderings) } } @@ -698,10 +699,10 @@ impl Accumulator for LastValueAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.last) + size_of_val(self) - size_of_val(&self.last) + self.last.size() + ScalarValue::size_of_vec(&self.orderings) - - std::mem::size_of_val(&self.orderings) + - size_of_val(&self.orderings) } } @@ -795,7 +796,7 @@ mod tests { let mut states = vec![]; for idx in 0..state1.len() { - states.push(arrow::compute::concat(&[ + states.push(compute::concat(&[ &state1[idx].to_array()?, &state2[idx].to_array()?, ])?); @@ -825,7 +826,7 @@ mod tests { let mut states = vec![]; for idx in 0..state1.len() { - states.push(arrow::compute::concat(&[ + states.push(compute::concat(&[ &state1[idx].to_array()?, &state2[idx].to_array()?, ])?); diff --git a/datafusion/functions-aggregate/src/grouping.rs b/datafusion/functions-aggregate/src/grouping.rs index 558d3055f1bf9..27949aa3df277 100644 --- a/datafusion/functions-aggregate/src/grouping.rs +++ b/datafusion/functions-aggregate/src/grouping.rs @@ -45,7 +45,7 @@ pub struct Grouping { } impl fmt::Debug for Grouping { - fn fmt(&self, f: &mut std::fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { f.debug_struct("Grouping") .field("name", &self.name()) .field("signature", &self.signature) diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index e0011e2e0f694..ff0a930d490bf 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -17,6 +17,7 @@ use std::collections::HashSet; use std::fmt::{Debug, Formatter}; +use std::mem::{size_of, size_of_val}; use std::sync::{Arc, OnceLock}; use arrow::array::{downcast_integer, ArrowNumericType}; @@ -62,7 +63,7 @@ pub struct Median { } impl Debug for Median { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { f.debug_struct("Median") .field("name", &self.name()) .field("signature", &self.signature) @@ -195,7 +196,7 @@ struct MedianAccumulator { all_values: Vec, } -impl std::fmt::Debug for MedianAccumulator { +impl Debug for MedianAccumulator { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "MedianAccumulator({})", self.data_type) } @@ -235,8 +236,7 @@ impl Accumulator for MedianAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - + self.all_values.capacity() * std::mem::size_of::() + size_of_val(self) + self.all_values.capacity() * size_of::() } } @@ -252,7 +252,7 @@ struct DistinctMedianAccumulator { distinct_values: HashSet>, } -impl std::fmt::Debug for DistinctMedianAccumulator { +impl Debug for DistinctMedianAccumulator { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "DistinctMedianAccumulator({})", self.data_type) } @@ -307,8 +307,7 @@ impl Accumulator for DistinctMedianAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - + self.distinct_values.capacity() * std::mem::size_of::() + size_of_val(self) + self.distinct_values.capacity() * size_of::() } } diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 8102d0e4794bf..b4256508e3515 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -60,6 +60,7 @@ use datafusion_expr::{ }; use datafusion_expr::{GroupsAccumulator, StatisticsArgs}; use half::f16; +use std::mem::size_of_val; use std::ops::Deref; use std::sync::OnceLock; @@ -923,7 +924,7 @@ impl Accumulator for MaxAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.max) + self.max.size() + size_of_val(self) - size_of_val(&self.max) + self.max.size() } } @@ -982,7 +983,7 @@ impl Accumulator for SlidingMaxAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.max) + self.max.size() + size_of_val(self) - size_of_val(&self.max) + self.max.size() } } @@ -1231,7 +1232,7 @@ impl Accumulator for MinAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.min) + self.min.size() + size_of_val(self) - size_of_val(&self.min) + self.min.size() } } @@ -1294,7 +1295,7 @@ impl Accumulator for SlidingMinAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - std::mem::size_of_val(&self.min) + self.min.size() + size_of_val(self) - size_of_val(&self.min) + self.min.size() } } diff --git a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs index e3f01b91bf3ec..501454edf77ca 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -22,6 +22,7 @@ use arrow_schema::DataType; use datafusion_common::{internal_err, Result}; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::apply_filter_as_nulls; +use std::mem::size_of; use std::sync::Arc; /// Implements fast Min/Max [`GroupsAccumulator`] for "bytes" types ([`StringArray`], @@ -509,7 +510,6 @@ impl MinMaxBytesState { } fn size(&self) -> usize { - self.total_data_bytes - + self.min_max.len() * std::mem::size_of::>>() + self.total_data_bytes + self.min_max.len() * size_of::>>() } } diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 3e7f51af52657..2a1778d8b232b 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -20,6 +20,7 @@ use std::any::Any; use std::collections::VecDeque; +use std::mem::{size_of, size_of_val}; use std::sync::{Arc, OnceLock}; use arrow::array::{new_empty_array, ArrayRef, AsArray, StructArray}; @@ -378,25 +379,23 @@ impl Accumulator for NthValueAccumulator { } fn size(&self) -> usize { - let mut total = std::mem::size_of_val(self) - + ScalarValue::size_of_vec_deque(&self.values) - - std::mem::size_of_val(&self.values); + let mut total = size_of_val(self) + ScalarValue::size_of_vec_deque(&self.values) + - size_of_val(&self.values); // Add size of the `self.ordering_values` - total += - std::mem::size_of::>() * self.ordering_values.capacity(); + total += size_of::>() * self.ordering_values.capacity(); for row in &self.ordering_values { - total += ScalarValue::size_of_vec(row) - std::mem::size_of_val(row); + total += ScalarValue::size_of_vec(row) - size_of_val(row); } // Add size of the `self.datatypes` - total += std::mem::size_of::() * self.datatypes.capacity(); + total += size_of::() * self.datatypes.capacity(); for dtype in &self.datatypes { - total += dtype.size() - std::mem::size_of_val(dtype); + total += dtype.size() - size_of_val(dtype); } // Add size of the `self.ordering_req` - total += std::mem::size_of::() * self.ordering_req.capacity(); + total += size_of::() * self.ordering_req.capacity(); // TODO: Calculate size of each `PhysicalSortExpr` more accurately. total } diff --git a/datafusion/functions-aggregate/src/regr.rs b/datafusion/functions-aggregate/src/regr.rs index a1fc5b094276b..bf1e81949d23a 100644 --- a/datafusion/functions-aggregate/src/regr.rs +++ b/datafusion/functions-aggregate/src/regr.rs @@ -36,6 +36,7 @@ use datafusion_expr::{ use std::any::Any; use std::collections::HashMap; use std::fmt::Debug; +use std::mem::size_of_val; use std::sync::OnceLock; macro_rules! make_regr_udaf_expr_and_func { @@ -614,6 +615,6 @@ impl Accumulator for RegrAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 0d1821687524f..355d1d5ad2db9 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -19,6 +19,7 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; +use std::mem::align_of_val; use std::sync::{Arc, OnceLock}; use arrow::array::Float64Array; @@ -343,8 +344,7 @@ impl Accumulator for StddevAccumulator { } fn size(&self) -> usize { - std::mem::align_of_val(self) - std::mem::align_of_val(&self.variance) - + self.variance.size() + align_of_val(self) - align_of_val(&self.variance) + self.variance.size() } fn supports_retract_batch(&self) -> bool { diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 66fc19910696c..68267b9f72c7d 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -29,6 +29,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr::expressions::Literal; use std::any::Any; +use std::mem::size_of_val; use std::sync::OnceLock; make_udaf_expr_and_func!( @@ -179,7 +180,7 @@ impl Accumulator for StringAggAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) + self.values.as_ref().map(|v| v.capacity()).unwrap_or(0) + self.delimiter.capacity() } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 943f66a92c000..6ad376db4fb9c 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -21,6 +21,7 @@ use ahash::RandomState; use datafusion_expr::utils::AggregateOrderSensitivity; use std::any::Any; use std::collections::HashSet; +use std::mem::{size_of, size_of_val}; use std::sync::OnceLock; use arrow::array::Array; @@ -310,7 +311,7 @@ impl Accumulator for SumAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } @@ -370,7 +371,7 @@ impl Accumulator for SlidingSumAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { @@ -464,7 +465,6 @@ impl Accumulator for DistinctSumAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) - + self.values.capacity() * std::mem::size_of::() + size_of_val(self) + self.values.capacity() * size_of::() } } diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 8453c9d3010be..810247a2884a9 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -24,6 +24,7 @@ use arrow::{ compute::kernels::cast, datatypes::{DataType, Field}, }; +use std::mem::{size_of, size_of_val}; use std::sync::OnceLock; use std::{fmt::Debug, sync::Arc}; @@ -424,7 +425,7 @@ impl Accumulator for VarianceAccumulator { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } fn supports_retract_batch(&self) -> bool { @@ -529,7 +530,7 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&arrow::array::BooleanArray>, + opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 1, "single argument to update_batch"); @@ -555,7 +556,7 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&arrow::array::BooleanArray>, + opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 3, "two arguments to merge_batch"); @@ -606,8 +607,8 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { } fn size(&self) -> usize { - self.m2s.capacity() * std::mem::size_of::() - + self.means.capacity() * std::mem::size_of::() - + self.counts.capacity() * std::mem::size_of::() + self.m2s.capacity() * size_of::() + + self.means.capacity() * size_of::() + + self.counts.capacity() * size_of::() } } diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs index 19a22690980b1..4f890e4166e9f 100644 --- a/datafusion/functions-nested/src/distance.rs +++ b/datafusion/functions-nested/src/distance.rs @@ -247,7 +247,7 @@ fn compute_array_distance( /// Converts an array of any numeric type to a Float64Array. fn convert_to_f64_array(array: &ArrayRef) -> Result { match array.data_type() { - DataType::Float64 => Ok(as_float64_array(array)?.clone()), + Float64 => Ok(as_float64_array(array)?.clone()), DataType::Float32 => { let array = as_float32_array(array)?; let converted: Float64Array = diff --git a/datafusion/functions-nested/src/make_array.rs b/datafusion/functions-nested/src/make_array.rs index abd7649e9ec79..c2c6f24948b8f 100644 --- a/datafusion/functions-nested/src/make_array.rs +++ b/datafusion/functions-nested/src/make_array.rs @@ -122,7 +122,7 @@ impl ScalarUDFImpl for MakeArray { if let Some(new_type) = type_union_resolution(arg_types) { // TODO: Move FixedSizeList to List in type_union_resolution if let DataType::FixedSizeList(field, _) = new_type { - Ok(vec![DataType::List(field); arg_types.len()]) + Ok(vec![List(field); arg_types.len()]) } else if new_type.is_null() { Ok(vec![DataType::Int64; arg_types.len()]) } else { @@ -174,7 +174,7 @@ fn get_make_array_doc() -> &'static Documentation { // Empty array is a special case that is useful for many other array functions pub(super) fn empty_array_type() -> DataType { - DataType::List(Arc::new(Field::new("item", DataType::Int64, true))) + List(Arc::new(Field::new("item", DataType::Int64, true))) } /// `make_array_inner` is the implementation of the `make_array` function. diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs index f28de1c3b2c76..03e381e372f64 100644 --- a/datafusion/functions-nested/src/map_keys.rs +++ b/datafusion/functions-nested/src/map_keys.rs @@ -66,7 +66,7 @@ impl ScalarUDFImpl for MapKeysFunc { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + fn return_type(&self, arg_types: &[DataType]) -> Result { if arg_types.len() != 1 { return exec_err!("map_keys expects single argument"); } @@ -79,7 +79,7 @@ impl ScalarUDFImpl for MapKeysFunc { )))) } - fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + fn invoke(&self, args: &[ColumnarValue]) -> Result { make_scalar_function(map_keys_inner)(args) } diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs index 2b19d9fbbc768..dc7d9c9db8eec 100644 --- a/datafusion/functions-nested/src/map_values.rs +++ b/datafusion/functions-nested/src/map_values.rs @@ -66,7 +66,7 @@ impl ScalarUDFImpl for MapValuesFunc { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + fn return_type(&self, arg_types: &[DataType]) -> Result { if arg_types.len() != 1 { return exec_err!("map_values expects single argument"); } @@ -79,7 +79,7 @@ impl ScalarUDFImpl for MapValuesFunc { )))) } - fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + fn invoke(&self, args: &[ColumnarValue]) -> Result { make_scalar_function(map_values_inner)(args) } diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index 342f99274acae..b2c7f06d58685 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -124,7 +124,7 @@ impl ScalarUDFImpl for NamedStructFunc { fn return_type_from_exprs( &self, - args: &[datafusion_expr::Expr], + args: &[Expr], schema: &dyn datafusion_common::ExprSchema, _arg_types: &[DataType], ) -> Result { diff --git a/datafusion/functions/src/core/planner.rs b/datafusion/functions/src/core/planner.rs index 5873b4e1af41c..717a74797c0b5 100644 --- a/datafusion/functions/src/core/planner.rs +++ b/datafusion/functions/src/core/planner.rs @@ -49,7 +49,7 @@ impl ExprPlanner for CoreFunctionPlanner { Ok(PlannerResult::Planned(Expr::ScalarFunction( ScalarFunction::new_udf( if is_named_struct { - crate::core::named_struct() + named_struct() } else { crate::core::r#struct() }, diff --git a/datafusion/functions/src/datetime/date_part.rs b/datafusion/functions/src/datetime/date_part.rs index 3fefa5051376d..01e094bc4e0b7 100644 --- a/datafusion/functions/src/datetime/date_part.rs +++ b/datafusion/functions/src/datetime/date_part.rs @@ -255,6 +255,7 @@ fn get_date_part_doc() -> &'static Documentation { "expression", "Time expression to operate on. Can be a constant, column, or function.", ) + .with_alternative_syntax("extract(field FROM source)") .build() .unwrap() }) diff --git a/datafusion/functions/src/datetime/make_date.rs b/datafusion/functions/src/datetime/make_date.rs index 78bd7c63a4124..c8ef349dfbeb5 100644 --- a/datafusion/functions/src/datetime/make_date.rs +++ b/datafusion/functions/src/datetime/make_date.rs @@ -89,9 +89,9 @@ impl ScalarUDFImpl for MakeDateFunc { ColumnarValue::Array(a) => Some(a.len()), }); - let years = args[0].cast_to(&DataType::Int32, None)?; - let months = args[1].cast_to(&DataType::Int32, None)?; - let days = args[2].cast_to(&DataType::Int32, None)?; + let years = args[0].cast_to(&Int32, None)?; + let months = args[1].cast_to(&Int32, None)?; + let days = args[2].cast_to(&Int32, None)?; let scalar_value_fn = |col: &ColumnarValue| -> Result { let ColumnarValue::Scalar(s) = col else { diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index 430dcedd92cf8..f0c4a02c15230 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -54,34 +54,34 @@ impl ToCharFunc { vec![ Exact(vec![Date32, Utf8]), Exact(vec![Date64, Utf8]), + Exact(vec![Time64(Nanosecond), Utf8]), + Exact(vec![Time64(Microsecond), Utf8]), Exact(vec![Time32(Millisecond), Utf8]), Exact(vec![Time32(Second), Utf8]), - Exact(vec![Time64(Microsecond), Utf8]), - Exact(vec![Time64(Nanosecond), Utf8]), - Exact(vec![Timestamp(Second, None), Utf8]), Exact(vec![ - Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), Utf8, ]), - Exact(vec![Timestamp(Millisecond, None), Utf8]), + Exact(vec![Timestamp(Nanosecond, None), Utf8]), Exact(vec![ - Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), Utf8, ]), Exact(vec![Timestamp(Microsecond, None), Utf8]), Exact(vec![ - Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), Utf8, ]), - Exact(vec![Timestamp(Nanosecond, None), Utf8]), + Exact(vec![Timestamp(Millisecond, None), Utf8]), Exact(vec![ - Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), + Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), Utf8, ]), - Exact(vec![Duration(Second), Utf8]), - Exact(vec![Duration(Millisecond), Utf8]), - Exact(vec![Duration(Microsecond), Utf8]), + Exact(vec![Timestamp(Second, None), Utf8]), Exact(vec![Duration(Nanosecond), Utf8]), + Exact(vec![Duration(Microsecond), Utf8]), + Exact(vec![Duration(Millisecond), Utf8]), + Exact(vec![Duration(Second), Utf8]), ], Volatility::Immutable, ), @@ -222,10 +222,7 @@ fn _to_char_scalar( if is_scalar_expression { return Ok(ColumnarValue::Scalar(ScalarValue::Utf8(None))); } else { - return Ok(ColumnarValue::Array(new_null_array( - &DataType::Utf8, - array.len(), - ))); + return Ok(ColumnarValue::Array(new_null_array(&Utf8, array.len()))); } } diff --git a/datafusion/functions/src/datetime/to_local_time.rs b/datafusion/functions/src/datetime/to_local_time.rs index 7646137ce6560..376cb6f5f2f83 100644 --- a/datafusion/functions/src/datetime/to_local_time.rs +++ b/datafusion/functions/src/datetime/to_local_time.rs @@ -68,7 +68,7 @@ impl ToLocalTimeFunc { let time_value = &args[0]; let arg_type = time_value.data_type(); match arg_type { - DataType::Timestamp(_, None) => { + Timestamp(_, None) => { // if no timezone specified, just return the input Ok(time_value.clone()) } @@ -78,7 +78,7 @@ impl ToLocalTimeFunc { // for more details. // // Then remove the timezone in return type, i.e. return None - DataType::Timestamp(_, Some(timezone)) => { + Timestamp(_, Some(timezone)) => { let tz: Tz = timezone.parse()?; match time_value { diff --git a/datafusion/functions/src/datetime/to_timestamp.rs b/datafusion/functions/src/datetime/to_timestamp.rs index 9479e25fe61fb..60482ee3c74a6 100644 --- a/datafusion/functions/src/datetime/to_timestamp.rs +++ b/datafusion/functions/src/datetime/to_timestamp.rs @@ -374,7 +374,7 @@ impl ScalarUDFImpl for ToTimestampMillisFunc { static TO_TIMESTAMP_MILLIS_DOC: OnceLock = OnceLock::new(); fn get_to_timestamp_millis_doc() -> &'static Documentation { - crate::datetime::to_timestamp::TO_TIMESTAMP_MILLIS_DOC.get_or_init(|| { + TO_TIMESTAMP_MILLIS_DOC.get_or_init(|| { Documentation::builder() .with_doc_section(DOC_SECTION_DATETIME) .with_description("Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.") @@ -1008,7 +1008,7 @@ mod tests { for udf in &udfs { for array in arrays { let rt = udf.return_type(&[array.data_type()]).unwrap(); - assert!(matches!(rt, DataType::Timestamp(_, Some(_)))); + assert!(matches!(rt, Timestamp(_, Some(_)))); let res = udf .invoke(&[array.clone()]) @@ -1018,7 +1018,7 @@ mod tests { _ => panic!("Expected a columnar array"), }; let ty = array.data_type(); - assert!(matches!(ty, DataType::Timestamp(_, Some(_)))); + assert!(matches!(ty, Timestamp(_, Some(_)))); } } @@ -1051,7 +1051,7 @@ mod tests { for udf in &udfs { for array in arrays { let rt = udf.return_type(&[array.data_type()]).unwrap(); - assert!(matches!(rt, DataType::Timestamp(_, None))); + assert!(matches!(rt, Timestamp(_, None))); let res = udf .invoke(&[array.clone()]) @@ -1061,7 +1061,7 @@ mod tests { _ => panic!("Expected a columnar array"), }; let ty = array.data_type(); - assert!(matches!(ty, DataType::Timestamp(_, None))); + assert!(matches!(ty, Timestamp(_, None))); } } } @@ -1137,10 +1137,7 @@ mod tests { .expect("that to_timestamp with format args parsed values without error"); if let ColumnarValue::Array(parsed_array) = parsed_timestamps { assert_eq!(parsed_array.len(), 1); - assert!(matches!( - parsed_array.data_type(), - DataType::Timestamp(_, None) - )); + assert!(matches!(parsed_array.data_type(), Timestamp(_, None))); match time_unit { Nanosecond => { diff --git a/datafusion/functions/src/math/factorial.rs b/datafusion/functions/src/math/factorial.rs index 4b87284744d3e..bacdf47524f4f 100644 --- a/datafusion/functions/src/math/factorial.rs +++ b/datafusion/functions/src/math/factorial.rs @@ -94,7 +94,7 @@ fn get_factorial_doc() -> &'static Documentation { /// Factorial SQL function fn factorial(args: &[ArrayRef]) -> Result { match args[0].data_type() { - DataType::Int64 => { + Int64 => { let arg = downcast_arg!((&args[0]), "value", Int64Array); Ok(arg .iter() diff --git a/datafusion/functions/src/math/round.rs b/datafusion/functions/src/math/round.rs index cf0f53a80a43a..6000e5d765de1 100644 --- a/datafusion/functions/src/math/round.rs +++ b/datafusion/functions/src/math/round.rs @@ -138,7 +138,7 @@ pub fn round(args: &[ArrayRef]) -> Result { } match args[0].data_type() { - DataType::Float64 => match decimal_places { + Float64 => match decimal_places { ColumnarValue::Scalar(ScalarValue::Int64(Some(decimal_places))) => { let decimal_places: i32 = decimal_places.try_into().map_err(|e| { exec_datafusion_err!( @@ -181,7 +181,7 @@ pub fn round(args: &[ArrayRef]) -> Result { } }, - DataType::Float32 => match decimal_places { + Float32 => match decimal_places { ColumnarValue::Scalar(ScalarValue::Int64(Some(decimal_places))) => { let decimal_places: i32 = decimal_places.try_into().map_err(|e| { exec_datafusion_err!( diff --git a/datafusion/functions/src/string/btrim.rs b/datafusion/functions/src/string/btrim.rs index f689f27d9d24a..e215b18d9c3ce 100644 --- a/datafusion/functions/src/string/btrim.rs +++ b/datafusion/functions/src/string/btrim.rs @@ -124,6 +124,8 @@ fn get_btrim_doc() -> &'static Documentation { ```"#) .with_standard_argument("str", Some("String")) .with_argument("trim_str", "String expression to operate on. Can be a constant, column, or function, and any combination of operators. _Default is whitespace characters._") + .with_alternative_syntax("trim(BOTH trim_str FROM str)") + .with_alternative_syntax("trim(trim_str FROM str)") .with_related_udf("ltrim") .with_related_udf("rtrim") .build() diff --git a/datafusion/functions/src/string/ltrim.rs b/datafusion/functions/src/string/ltrim.rs index 91809d6916471..0b4c197646b6d 100644 --- a/datafusion/functions/src/string/ltrim.rs +++ b/datafusion/functions/src/string/ltrim.rs @@ -124,6 +124,7 @@ fn get_ltrim_doc() -> &'static Documentation { ```"#) .with_standard_argument("str", Some("String")) .with_argument("trim_str", "String expression to trim from the beginning of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._") + .with_alternative_syntax("trim(LEADING trim_str FROM str)") .with_related_udf("btrim") .with_related_udf("rtrim") .build() diff --git a/datafusion/functions/src/string/rtrim.rs b/datafusion/functions/src/string/rtrim.rs index 06c8a85c38dd4..e934147efbbe3 100644 --- a/datafusion/functions/src/string/rtrim.rs +++ b/datafusion/functions/src/string/rtrim.rs @@ -124,6 +124,7 @@ fn get_rtrim_doc() -> &'static Documentation { ```"#) .with_standard_argument("str", Some("String")) .with_argument("trim_str", "String expression to trim from the end of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._") + .with_alternative_syntax("trim(TRAILING trim_str FROM str)") .with_related_udf("btrim") .with_related_udf("ltrim") .build() diff --git a/datafusion/functions/src/strings.rs b/datafusion/functions/src/strings.rs index 2e0e2c48390f4..e0cec3cb5756f 100644 --- a/datafusion/functions/src/strings.rs +++ b/datafusion/functions/src/strings.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::mem::size_of; + use arrow::array::{ make_view, Array, ArrayAccessor, ArrayDataBuilder, ArrayIter, ByteView, GenericStringArray, LargeStringArray, OffsetSizeTrait, StringArray, StringViewArray, @@ -122,9 +124,8 @@ pub struct StringArrayBuilder { impl StringArrayBuilder { pub fn with_capacity(item_capacity: usize, data_capacity: usize) -> Self { - let mut offsets_buffer = MutableBuffer::with_capacity( - (item_capacity + 1) * std::mem::size_of::(), - ); + let mut offsets_buffer = + MutableBuffer::with_capacity((item_capacity + 1) * size_of::()); // SAFETY: the first offset value is definitely not going to exceed the bounds. unsafe { offsets_buffer.push_unchecked(0_i32) }; Self { @@ -186,7 +187,7 @@ impl StringArrayBuilder { pub fn finish(self, null_buffer: Option) -> StringArray { let array_builder = ArrayDataBuilder::new(DataType::Utf8) - .len(self.offsets_buffer.len() / std::mem::size_of::() - 1) + .len(self.offsets_buffer.len() / size_of::() - 1) .add_buffer(self.offsets_buffer.into()) .add_buffer(self.value_buffer.into()) .nulls(null_buffer); @@ -273,9 +274,8 @@ pub struct LargeStringArrayBuilder { impl LargeStringArrayBuilder { pub fn with_capacity(item_capacity: usize, data_capacity: usize) -> Self { - let mut offsets_buffer = MutableBuffer::with_capacity( - (item_capacity + 1) * std::mem::size_of::(), - ); + let mut offsets_buffer = + MutableBuffer::with_capacity((item_capacity + 1) * size_of::()); // SAFETY: the first offset value is definitely not going to exceed the bounds. unsafe { offsets_buffer.push_unchecked(0_i64) }; Self { @@ -337,7 +337,7 @@ impl LargeStringArrayBuilder { pub fn finish(self, null_buffer: Option) -> LargeStringArray { let array_builder = ArrayDataBuilder::new(DataType::LargeUtf8) - .len(self.offsets_buffer.len() / std::mem::size_of::() - 1) + .len(self.offsets_buffer.len() / size_of::() - 1) .add_buffer(self.offsets_buffer.into()) .add_buffer(self.value_buffer.into()) .nulls(null_buffer); diff --git a/datafusion/functions/src/unicode/strpos.rs b/datafusion/functions/src/unicode/strpos.rs index 152623b0e5dcb..9c84590f7f94e 100644 --- a/datafusion/functions/src/unicode/strpos.rs +++ b/datafusion/functions/src/unicode/strpos.rs @@ -97,6 +97,7 @@ fn get_strpos_doc() -> &'static Documentation { ```"#) .with_standard_argument("str", Some("String")) .with_argument("substr", "Substring expression to search for.") + .with_alternative_syntax("position(substr in origstr)") .build() .unwrap() }) diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 5a8c2500900b6..edfe57210b711 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -173,6 +173,7 @@ fn get_substr_doc() -> &'static Documentation { .with_standard_argument("str", Some("String")) .with_argument("start_pos", "Character position to start the substring at. The first character in the string has a position of 1.") .with_argument("length", "Number of characters to extract. If not specified, returns the rest of the string after the start position.") + .with_alternative_syntax("substring(str from start_pos for length)") .build() .unwrap() }) diff --git a/datafusion/functions/src/utils.rs b/datafusion/functions/src/utils.rs index 818b4c64bd20c..4d6574d2bd6cd 100644 --- a/datafusion/functions/src/utils.rs +++ b/datafusion/functions/src/utils.rs @@ -107,7 +107,7 @@ where }; arg.clone().into_array(expansion_len) }) - .collect::>>()?; + .collect::>>()?; let result = (inner)(&args); if is_scalar { diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index b3b24724552a3..454afa24b628c 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -101,7 +101,7 @@ mod tests { use datafusion_expr::expr::Sort; use datafusion_expr::ExprFunctionExt; use datafusion_expr::{ - col, exists, expr, in_subquery, logical_plan::LogicalPlanBuilder, out_ref_col, + col, exists, in_subquery, logical_plan::LogicalPlanBuilder, out_ref_col, scalar_subquery, wildcard, WindowFrame, WindowFrameBound, WindowFrameUnits, }; use datafusion_functions_aggregate::count::count_udaf; @@ -219,7 +219,7 @@ mod tests { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .window(vec![Expr::WindowFunction(expr::WindowFunction::new( + .window(vec![Expr::WindowFunction(WindowFunction::new( WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index e01ae625ed9c1..0ffc954388f5a 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -113,7 +113,9 @@ pub fn check_subquery_expr( | LogicalPlan::Join(_) => Ok(()), _ => plan_err!( "In/Exist subquery can only be used in \ - Projection, Filter, Window functions, Aggregate and Join plan nodes" + Projection, Filter, Window functions, Aggregate and Join plan nodes, \ + but was used in [{}]", + outer_plan.display() ), }?; check_correlations_in_subquery(inner_plan) @@ -303,7 +305,7 @@ mod test { vec![] } - fn schema(&self) -> &datafusion_common::DFSchemaRef { + fn schema(&self) -> &DFSchemaRef { &self.empty_schema } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 33eea1a661c6d..5d33b58a02411 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -1243,7 +1243,7 @@ mod test { } fn return_type(&self, _args: &[DataType]) -> Result { - Ok(DataType::Utf8) + Ok(Utf8) } fn invoke(&self, _args: &[ColumnarValue]) -> Result { @@ -1446,7 +1446,7 @@ mod test { cast(lit("2002-05-08"), DataType::Date32) + lit(ScalarValue::new_interval_ym(0, 1)), ); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Filter(Filter::try_new(expr, empty)?); let expected = "Filter: a BETWEEN Utf8(\"2002-05-08\") AND CAST(CAST(Utf8(\"2002-05-08\") AS Date32) + IntervalYearMonth(\"1\") AS Utf8)\ @@ -1462,7 +1462,7 @@ mod test { + lit(ScalarValue::new_interval_ym(0, 1)), lit("2002-12-08"), ); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Filter(Filter::try_new(expr, empty)?); // TODO: we should cast col(a). let expected = @@ -1517,7 +1517,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let expected = "Projection: a LIKE Utf8(\"abc\")\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1525,7 +1525,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::Null)); let like_expr = Expr::Like(Like::new(false, expr, pattern, None, false)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![like_expr], empty)?); let expected = "Projection: a LIKE CAST(NULL AS Utf8)\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1545,7 +1545,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::new_utf8("abc"))); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let expected = "Projection: a ILIKE Utf8(\"abc\")\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1553,7 +1553,7 @@ mod test { let expr = Box::new(col("a")); let pattern = Box::new(lit(ScalarValue::Null)); let ilike_expr = Expr::Like(Like::new(false, expr, pattern, None, true)); - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![ilike_expr], empty)?); let expected = "Projection: a ILIKE CAST(NULL AS Utf8)\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; @@ -1581,7 +1581,7 @@ mod test { let expected = "Projection: a IS UNKNOWN\n EmptyRelation"; assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected)?; - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let plan = LogicalPlan::Projection(Projection::try_new(vec![expr], empty)?); let ret = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected); let err = ret.unwrap_err().to_string(); @@ -1599,7 +1599,7 @@ mod test { #[test] fn concat_for_type_coercion() -> Result<()> { - let empty = empty_with_type(DataType::Utf8); + let empty = empty_with_type(Utf8); let args = [col("a"), lit("b"), lit(true), lit(false), lit(13)]; // concat-type signature @@ -1734,7 +1734,7 @@ mod test { true, ), Field::new("binary", DataType::Binary, true), - Field::new("string", DataType::Utf8, true), + Field::new("string", Utf8, true), Field::new("decimal", DataType::Decimal128(10, 10), true), ] .into(), @@ -1751,7 +1751,7 @@ mod test { else_expr: None, }; let case_when_common_type = DataType::Boolean; - let then_else_common_type = DataType::Utf8; + let then_else_common_type = Utf8; let expected = cast_helper( case.clone(), &case_when_common_type, @@ -1770,8 +1770,8 @@ mod test { ], else_expr: Some(Box::new(col("string"))), }; - let case_when_common_type = DataType::Utf8; - let then_else_common_type = DataType::Utf8; + let case_when_common_type = Utf8; + let then_else_common_type = Utf8; let expected = cast_helper( case.clone(), &case_when_common_type, @@ -1861,7 +1861,7 @@ mod test { Some("list"), vec![(Box::new(col("large_list")), Box::new(lit("1")))], DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), - DataType::Utf8, + Utf8, schema ); @@ -1869,7 +1869,7 @@ mod test { Some("large_list"), vec![(Box::new(col("list")), Box::new(lit("1")))], DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), - DataType::Utf8, + Utf8, schema ); @@ -1877,7 +1877,7 @@ mod test { Some("list"), vec![(Box::new(col("fixed_list")), Box::new(lit("1")))], DataType::List(Arc::new(Field::new("item", DataType::Int64, true))), - DataType::Utf8, + Utf8, schema ); @@ -1885,7 +1885,7 @@ mod test { Some("fixed_list"), vec![(Box::new(col("list")), Box::new(lit("1")))], DataType::List(Arc::new(Field::new("item", DataType::Int64, true))), - DataType::Utf8, + Utf8, schema ); @@ -1893,7 +1893,7 @@ mod test { Some("fixed_list"), vec![(Box::new(col("large_list")), Box::new(lit("1")))], DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), - DataType::Utf8, + Utf8, schema ); @@ -1901,7 +1901,7 @@ mod test { Some("large_list"), vec![(Box::new(col("fixed_list")), Box::new(lit("1")))], DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), - DataType::Utf8, + Utf8, schema ); Ok(()) diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index cdffa8c645ea5..cc1687cffe921 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -357,9 +357,9 @@ fn build_join( .for_each(|cols| all_correlated_cols.extend(cols.clone())); // alias the join filter - let join_filter_opt = - conjunction(pull_up.join_filters).map_or(Ok(None), |filter| { - replace_qualified_name(filter, &all_correlated_cols, &alias).map(Option::Some) + let join_filter_opt = conjunction(pull_up.join_filters) + .map_or(Ok(None), |filter| { + replace_qualified_name(filter, &all_correlated_cols, &alias).map(Some) })?; if let Some(join_filter) = match (join_filter_opt, in_predicate_opt) { diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index 829d4c2d2217c..267615c3e0d93 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -57,10 +57,7 @@ impl OptimizerRule for EliminateLimit { &self, plan: LogicalPlan, _config: &dyn OptimizerConfig, - ) -> Result< - datafusion_common::tree_node::Transformed, - datafusion_common::DataFusionError, - > { + ) -> Result, datafusion_common::DataFusionError> { match plan { LogicalPlan::Limit(limit) => { // Only supports rewriting for literal fetch diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index a6c0a7310610d..a0262d7d95dfe 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -36,7 +36,7 @@ use datafusion_expr::{ }; use crate::optimizer::ApplyOrder; -use crate::utils::has_all_column_refs; +use crate::utils::{has_all_column_refs, is_restrict_null_predicate}; use crate::{OptimizerConfig, OptimizerRule}; /// Optimizer rule for pushing (moving) filter expressions down in a plan so @@ -558,10 +558,6 @@ fn infer_join_predicates( predicates: &[Expr], on_filters: &[Expr], ) -> Result> { - if join.join_type != JoinType::Inner { - return Ok(vec![]); - } - // Only allow both side key is column. let join_col_keys = join .on @@ -573,55 +569,176 @@ fn infer_join_predicates( }) .collect::>(); - // TODO refine the logic, introduce EquivalenceProperties to logical plan and infer additional filters to push down - // For inner joins, duplicate filters for joined columns so filters can be pushed down - // to both sides. Take the following query as an example: - // - // ```sql - // SELECT * FROM t1 JOIN t2 on t1.id = t2.uid WHERE t1.id > 1 - // ``` - // - // `t1.id > 1` predicate needs to be pushed down to t1 table scan, while - // `t2.uid > 1` predicate needs to be pushed down to t2 table scan. - // - // Join clauses with `Using` constraints also take advantage of this logic to make sure - // predicates reference the shared join columns are pushed to both sides. - // This logic should also been applied to conditions in JOIN ON clause - predicates - .iter() - .chain(on_filters.iter()) - .filter_map(|predicate| { - let mut join_cols_to_replace = HashMap::new(); - - let columns = predicate.column_refs(); - - for &col in columns.iter() { - for (l, r) in join_col_keys.iter() { - if col == *l { - join_cols_to_replace.insert(col, *r); - break; - } else if col == *r { - join_cols_to_replace.insert(col, *l); - break; - } - } - } + let join_type = join.join_type; - if join_cols_to_replace.is_empty() { - return None; - } + let mut inferred_predicates = InferredPredicates::new(join_type); - let join_side_predicate = - match replace_col(predicate.clone(), &join_cols_to_replace) { - Ok(p) => p, - Err(e) => { - return Some(Err(e)); - } - }; + infer_join_predicates_from_predicates( + &join_col_keys, + predicates, + &mut inferred_predicates, + )?; - Some(Ok(join_side_predicate)) - }) - .collect::>>() + infer_join_predicates_from_on_filters( + &join_col_keys, + join_type, + on_filters, + &mut inferred_predicates, + )?; + + Ok(inferred_predicates.predicates) +} + +/// Inferred predicates collector. +/// When the JoinType is not Inner, we need to detect whether the inferred predicate can strictly +/// filter out NULL, otherwise ignore it. e.g. +/// ```text +/// SELECT * FROM t1 LEFT JOIN t2 ON t1.c0 = t2.c0 WHERE t2.c0 IS NULL; +/// ``` +/// We cannot infer the predicate `t1.c0 IS NULL`, otherwise the predicate will be pushed down to +/// the left side, resulting in the wrong result. +struct InferredPredicates { + predicates: Vec, + is_inner_join: bool, +} + +impl InferredPredicates { + fn new(join_type: JoinType) -> Self { + Self { + predicates: vec![], + is_inner_join: matches!(join_type, JoinType::Inner), + } + } + + fn try_build_predicate( + &mut self, + predicate: Expr, + replace_map: &HashMap<&Column, &Column>, + ) -> Result<()> { + if self.is_inner_join + || matches!( + is_restrict_null_predicate( + predicate.clone(), + replace_map.keys().cloned() + ), + Ok(true) + ) + { + self.predicates.push(replace_col(predicate, replace_map)?); + } + + Ok(()) + } +} + +/// Infer predicates from the pushed down predicates. +/// +/// Parameters +/// * `join_col_keys` column pairs from the join ON clause +/// +/// * `predicates` the pushed down predicates +/// +/// * `inferred_predicates` the inferred results +/// +fn infer_join_predicates_from_predicates( + join_col_keys: &[(&Column, &Column)], + predicates: &[Expr], + inferred_predicates: &mut InferredPredicates, +) -> Result<()> { + infer_join_predicates_impl::( + join_col_keys, + predicates, + inferred_predicates, + ) +} + +/// Infer predicates from the join filter. +/// +/// Parameters +/// * `join_col_keys` column pairs from the join ON clause +/// +/// * `join_type` the JoinType of Join +/// +/// * `on_filters` filters from the join ON clause that have not already been +/// identified as join predicates +/// +/// * `inferred_predicates` the inferred results +/// +fn infer_join_predicates_from_on_filters( + join_col_keys: &[(&Column, &Column)], + join_type: JoinType, + on_filters: &[Expr], + inferred_predicates: &mut InferredPredicates, +) -> Result<()> { + match join_type { + JoinType::Full | JoinType::LeftAnti | JoinType::RightAnti => Ok(()), + JoinType::Inner => infer_join_predicates_impl::( + join_col_keys, + on_filters, + inferred_predicates, + ), + JoinType::Left | JoinType::LeftSemi => infer_join_predicates_impl::( + join_col_keys, + on_filters, + inferred_predicates, + ), + JoinType::Right | JoinType::RightSemi => { + infer_join_predicates_impl::( + join_col_keys, + on_filters, + inferred_predicates, + ) + } + } +} + +/// Infer predicates from the given predicates. +/// +/// Parameters +/// * `join_col_keys` column pairs from the join ON clause +/// +/// * `input_predicates` the given predicates. It can be the pushed down predicates, +/// or it can be the filters of the Join +/// +/// * `inferred_predicates` the inferred results +/// +/// * `ENABLE_LEFT_TO_RIGHT` indicates that the right table related predicate can +/// be inferred from the left table related predicate +/// +/// * `ENABLE_RIGHT_TO_LEFT` indicates that the left table related predicate can +/// be inferred from the right table related predicate +/// +fn infer_join_predicates_impl< + const ENABLE_LEFT_TO_RIGHT: bool, + const ENABLE_RIGHT_TO_LEFT: bool, +>( + join_col_keys: &[(&Column, &Column)], + input_predicates: &[Expr], + inferred_predicates: &mut InferredPredicates, +) -> Result<()> { + for predicate in input_predicates { + let mut join_cols_to_replace = HashMap::new(); + + for &col in &predicate.column_refs() { + for (l, r) in join_col_keys.iter() { + if ENABLE_LEFT_TO_RIGHT && col == *l { + join_cols_to_replace.insert(col, *r); + break; + } + if ENABLE_RIGHT_TO_LEFT && col == *r { + join_cols_to_replace.insert(col, *l); + break; + } + } + } + if join_cols_to_replace.is_empty() { + continue; + } + + inferred_predicates + .try_build_predicate(predicate.clone(), &join_cols_to_replace)?; + } + Ok(()) } impl OptimizerRule for PushDownFilter { @@ -1992,7 +2109,7 @@ mod tests { let expected = "\ Filter: test2.a <= Int64(1)\ \n Left Join: Using test.a = test2.a\ - \n TableScan: test\ + \n TableScan: test, full_filters=[test.a <= Int64(1)]\ \n Projection: test2.a\ \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) @@ -2032,7 +2149,7 @@ mod tests { \n Right Join: Using test.a = test2.a\ \n TableScan: test\ \n Projection: test2.a\ - \n TableScan: test2"; + \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) } @@ -2387,7 +2504,7 @@ mod tests { .collect()) } - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } } @@ -2814,6 +2931,46 @@ Projection: a, b assert_optimized_plan_eq(optimized_plan, expected) } + #[test] + fn left_semi_join() -> Result<()> { + let left = test_table_scan_with_name("test1")?; + let right_table_scan = test_table_scan_with_name("test2")?; + let right = LogicalPlanBuilder::from(right_table_scan) + .project(vec![col("a"), col("b")])? + .build()?; + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::LeftSemi, + ( + vec![Column::from_qualified_name("test1.a")], + vec![Column::from_qualified_name("test2.a")], + ), + None, + )? + .filter(col("test2.a").lt_eq(lit(1i64)))? + .build()?; + + // not part of the test, just good to know: + assert_eq!( + format!("{plan}"), + "Filter: test2.a <= Int64(1)\ + \n LeftSemi Join: test1.a = test2.a\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2" + ); + + // Inferred the predicate `test1.a <= Int64(1)` and push it down to the left side. + let expected = "\ + Filter: test2.a <= Int64(1)\ + \n LeftSemi Join: test1.a = test2.a\ + \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2"; + assert_optimized_plan_eq(plan, expected) + } + #[test] fn left_semi_join_with_filters() -> Result<()> { let left = test_table_scan_with_name("test1")?; @@ -2855,6 +3012,46 @@ Projection: a, b assert_optimized_plan_eq(plan, expected) } + #[test] + fn right_semi_join() -> Result<()> { + let left = test_table_scan_with_name("test1")?; + let right_table_scan = test_table_scan_with_name("test2")?; + let right = LogicalPlanBuilder::from(right_table_scan) + .project(vec![col("a"), col("b")])? + .build()?; + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::RightSemi, + ( + vec![Column::from_qualified_name("test1.a")], + vec![Column::from_qualified_name("test2.a")], + ), + None, + )? + .filter(col("test1.a").lt_eq(lit(1i64)))? + .build()?; + + // not part of the test, just good to know: + assert_eq!( + format!("{plan}"), + "Filter: test1.a <= Int64(1)\ + \n RightSemi Join: test1.a = test2.a\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2", + ); + + // Inferred the predicate `test2.a <= Int64(1)` and push it down to the right side. + let expected = "\ + Filter: test1.a <= Int64(1)\ + \n RightSemi Join: test1.a = test2.a\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + assert_optimized_plan_eq(plan, expected) + } + #[test] fn right_semi_join_with_filters() -> Result<()> { let left = test_table_scan_with_name("test1")?; @@ -2896,6 +3093,51 @@ Projection: a, b assert_optimized_plan_eq(plan, expected) } + #[test] + fn left_anti_join() -> Result<()> { + let table_scan = test_table_scan_with_name("test1")?; + let left = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b")])? + .build()?; + let right_table_scan = test_table_scan_with_name("test2")?; + let right = LogicalPlanBuilder::from(right_table_scan) + .project(vec![col("a"), col("b")])? + .build()?; + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::LeftAnti, + ( + vec![Column::from_qualified_name("test1.a")], + vec![Column::from_qualified_name("test2.a")], + ), + None, + )? + .filter(col("test2.a").gt(lit(2u32)))? + .build()?; + + // not part of the test, just good to know: + assert_eq!( + format!("{plan}"), + "Filter: test2.a > UInt32(2)\ + \n LeftAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2", + ); + + // For left anti, filter of the right side filter can be pushed down. + let expected = "\ + Filter: test2.a > UInt32(2)\ + \n LeftAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2"; + assert_optimized_plan_eq(plan, expected) + } + #[test] fn left_anti_join_with_filters() -> Result<()> { let table_scan = test_table_scan_with_name("test1")?; @@ -2942,6 +3184,51 @@ Projection: a, b assert_optimized_plan_eq(plan, expected) } + #[test] + fn right_anti_join() -> Result<()> { + let table_scan = test_table_scan_with_name("test1")?; + let left = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b")])? + .build()?; + let right_table_scan = test_table_scan_with_name("test2")?; + let right = LogicalPlanBuilder::from(right_table_scan) + .project(vec![col("a"), col("b")])? + .build()?; + let plan = LogicalPlanBuilder::from(left) + .join( + right, + JoinType::RightAnti, + ( + vec![Column::from_qualified_name("test1.a")], + vec![Column::from_qualified_name("test2.a")], + ), + None, + )? + .filter(col("test1.a").gt(lit(2u32)))? + .build()?; + + // not part of the test, just good to know: + assert_eq!( + format!("{plan}"), + "Filter: test1.a > UInt32(2)\ + \n RightAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2", + ); + + // For right anti, filter of the left side can be pushed down. + let expected = "\ + Filter: test1.a > UInt32(2)\ + \n RightAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; + assert_optimized_plan_eq(plan, expected) + } + #[test] fn right_anti_join_with_filters() -> Result<()> { let table_scan = test_table_scan_with_name("test1")?; diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index 7b931e73abf98..2e2c8fb1d6f8c 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -318,8 +318,7 @@ fn build_join( // alias the join filter let join_filter_opt = conjunction(pull_up.join_filters).map_or(Ok(None), |filter| { - replace_qualified_name(filter, &all_correlated_cols, subquery_alias) - .map(Option::Some) + replace_qualified_name(filter, &all_correlated_cols, subquery_alias).map(Some) })?; // join our sub query into the main plan diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index f9dfadc70826e..ce6734616b805 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -1537,7 +1537,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { // i.e. `a = 1 OR a = 2 OR a = 3` -> `a IN (1, 2, 3)` Expr::BinaryExpr(BinaryExpr { left, - op: Operator::Or, + op: Or, right, }) if are_inlist_and_eq(left.as_ref(), right.as_ref()) => { let lhs = to_inlist(*left).unwrap(); @@ -1577,7 +1577,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { // 8. `a in (1,2,3,4) AND a not in (5,6,7,8) -> a in (1,2,3,4)` Expr::BinaryExpr(BinaryExpr { left, - op: Operator::And, + op: And, right, }) if are_inlist_and_eq_and_match_neg( left.as_ref(), @@ -1597,7 +1597,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { Expr::BinaryExpr(BinaryExpr { left, - op: Operator::And, + op: And, right, }) if are_inlist_and_eq_and_match_neg( left.as_ref(), @@ -1617,7 +1617,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { Expr::BinaryExpr(BinaryExpr { left, - op: Operator::And, + op: And, right, }) if are_inlist_and_eq_and_match_neg( left.as_ref(), @@ -1637,7 +1637,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { Expr::BinaryExpr(BinaryExpr { left, - op: Operator::And, + op: And, right, }) if are_inlist_and_eq_and_match_neg( left.as_ref(), @@ -1657,7 +1657,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { Expr::BinaryExpr(BinaryExpr { left, - op: Operator::Or, + op: Or, right, }) if are_inlist_and_eq_and_match_neg( left.as_ref(), @@ -3818,7 +3818,7 @@ mod tests { fn test_simplify_udaf() { let udaf = AggregateUDF::new_from_impl(SimplifyMockUdaf::new_with_simplify()); let aggregate_function_expr = - Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( + Expr::AggregateFunction(expr::AggregateFunction::new_udf( udaf.into(), vec![], false, @@ -3832,7 +3832,7 @@ mod tests { let udaf = AggregateUDF::new_from_impl(SimplifyMockUdaf::new_without_simplify()); let aggregate_function_expr = - Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( + Expr::AggregateFunction(expr::AggregateFunction::new_udf( udaf.into(), vec![], false, @@ -3882,7 +3882,7 @@ mod tests { fn accumulator( &self, - _acc_args: function::AccumulatorArgs, + _acc_args: AccumulatorArgs, ) -> Result> { unimplemented!("not needed for tests") } @@ -3912,9 +3912,8 @@ mod tests { let udwf = WindowFunctionDefinition::WindowUDF( WindowUDF::new_from_impl(SimplifyMockUdwf::new_with_simplify()).into(), ); - let window_function_expr = Expr::WindowFunction( - datafusion_expr::expr::WindowFunction::new(udwf, vec![]), - ); + let window_function_expr = + Expr::WindowFunction(WindowFunction::new(udwf, vec![])); let expected = col("result_column"); assert_eq!(simplify(window_function_expr), expected); @@ -3922,9 +3921,8 @@ mod tests { let udwf = WindowFunctionDefinition::WindowUDF( WindowUDF::new_from_impl(SimplifyMockUdwf::new_without_simplify()).into(), ); - let window_function_expr = Expr::WindowFunction( - datafusion_expr::expr::WindowFunction::new(udwf, vec![]), - ); + let window_function_expr = + Expr::WindowFunction(WindowFunction::new(udwf, vec![])); let expected = window_function_expr.clone(); assert_eq!(simplify(window_function_expr), expected); diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 74251e5caad2b..01875349c922a 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -279,7 +279,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { mod tests { use super::*; use crate::test::*; - use datafusion_expr::expr::{self, GroupingSet}; + use datafusion_expr::expr::GroupingSet; use datafusion_expr::ExprFunctionExt; use datafusion_expr::{lit, logical_plan::builder::LogicalPlanBuilder}; use datafusion_functions_aggregate::count::count_udaf; @@ -288,7 +288,7 @@ mod tests { use datafusion_functions_aggregate::sum::sum_udaf; fn max_distinct(expr: Expr) -> Expr { - Expr::AggregateFunction(datafusion_expr::expr::AggregateFunction::new_udf( + Expr::AggregateFunction(AggregateFunction::new_udf( max_udaf(), vec![expr], true, @@ -569,7 +569,7 @@ mod tests { let table_scan = test_table_scan()?; // sum(a) FILTER (WHERE a > 5) - let expr = Expr::AggregateFunction(expr::AggregateFunction::new_udf( + let expr = Expr::AggregateFunction(AggregateFunction::new_udf( sum_udaf(), vec![col("a")], false, @@ -612,7 +612,7 @@ mod tests { let table_scan = test_table_scan()?; // SUM(a ORDER BY a) - let expr = Expr::AggregateFunction(expr::AggregateFunction::new_udf( + let expr = Expr::AggregateFunction(AggregateFunction::new_udf( sum_udaf(), vec![col("a")], false, diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index cabeafd8e7dea..94d07a0791b3b 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -133,20 +133,6 @@ pub fn assert_analyzed_plan_with_config_eq( Ok(()) } -pub fn assert_analyzed_plan_ne( - rule: Arc, - plan: LogicalPlan, - expected: &str, -) -> Result<()> { - let options = ConfigOptions::default(); - let analyzed_plan = - Analyzer::with_rules(vec![rule]).execute_and_check(plan, &options, |_, _| {})?; - let formatted_plan = format!("{analyzed_plan}"); - assert_ne!(formatted_plan, expected); - - Ok(()) -} - pub fn assert_analyzed_plan_eq_display_indent( rule: Arc, plan: LogicalPlan, diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 6972c16c0ddf8..9f325bc01b1d0 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -21,11 +21,18 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{Column, DFSchema, Result}; +use crate::analyzer::type_coercion::TypeCoercionRewriter; +use arrow::array::{new_null_array, Array, RecordBatch}; +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion_common::cast::as_boolean_array; +use datafusion_common::tree_node::{TransformedResult, TreeNode}; +use datafusion_common::{Column, DFSchema, Result, ScalarValue}; +use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr_rewriter::replace_col; -use datafusion_expr::{logical_plan::LogicalPlan, Expr}; - +use datafusion_expr::{logical_plan::LogicalPlan, ColumnarValue, Expr}; +use datafusion_physical_expr::create_physical_expr; use log::{debug, trace}; +use std::sync::Arc; /// Re-export of `NamesPreserver` for backwards compatibility, /// as it was initially placed here and then moved elsewhere. @@ -117,3 +124,161 @@ pub fn log_plan(description: &str, plan: &LogicalPlan) { debug!("{description}:\n{}\n", plan.display_indent()); trace!("{description}::\n{}\n", plan.display_indent_schema()); } + +/// Determine whether a predicate can restrict NULLs. e.g. +/// `c0 > 8` return true; +/// `c0 IS NULL` return false. +pub fn is_restrict_null_predicate<'a>( + predicate: Expr, + join_cols_of_predicate: impl IntoIterator, +) -> Result { + if matches!(predicate, Expr::Column(_)) { + return Ok(true); + } + + static DUMMY_COL_NAME: &str = "?"; + let schema = Schema::new(vec![Field::new(DUMMY_COL_NAME, DataType::Null, true)]); + let input_schema = DFSchema::try_from(schema.clone())?; + let column = new_null_array(&DataType::Null, 1); + let input_batch = RecordBatch::try_new(Arc::new(schema.clone()), vec![column])?; + let execution_props = ExecutionProps::default(); + let null_column = Column::from_name(DUMMY_COL_NAME); + + let join_cols_to_replace = join_cols_of_predicate + .into_iter() + .map(|column| (column, &null_column)) + .collect::>(); + + let replaced_predicate = replace_col(predicate, &join_cols_to_replace)?; + let coerced_predicate = coerce(replaced_predicate, &input_schema)?; + let phys_expr = + create_physical_expr(&coerced_predicate, &input_schema, &execution_props)?; + + let result_type = phys_expr.data_type(&schema)?; + if !matches!(&result_type, DataType::Boolean) { + return Ok(false); + } + + // If result is single `true`, return false; + // If result is single `NULL` or `false`, return true; + Ok(match phys_expr.evaluate(&input_batch)? { + ColumnarValue::Array(array) => { + if array.len() == 1 { + let boolean_array = as_boolean_array(&array)?; + boolean_array.is_null(0) || !boolean_array.value(0) + } else { + false + } + } + ColumnarValue::Scalar(scalar) => matches!( + scalar, + ScalarValue::Boolean(None) | ScalarValue::Boolean(Some(false)) + ), + }) +} + +fn coerce(expr: Expr, schema: &DFSchema) -> Result { + let mut expr_rewrite = TypeCoercionRewriter { schema }; + expr.rewrite(&mut expr_rewrite).data() +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion_expr::{binary_expr, case, col, in_list, is_null, lit, Operator}; + + #[test] + fn expr_is_restrict_null_predicate() -> Result<()> { + let test_cases = vec![ + // a + (col("a"), true), + // a IS NULL + (is_null(col("a")), false), + // a IS NOT NULL + (Expr::IsNotNull(Box::new(col("a"))), true), + // a = NULL + ( + binary_expr(col("a"), Operator::Eq, Expr::Literal(ScalarValue::Null)), + true, + ), + // a > 8 + (binary_expr(col("a"), Operator::Gt, lit(8i64)), true), + // a <= 8 + (binary_expr(col("a"), Operator::LtEq, lit(8i32)), true), + // CASE a WHEN 1 THEN true WHEN 0 THEN false ELSE NULL END + ( + case(col("a")) + .when(lit(1i64), lit(true)) + .when(lit(0i64), lit(false)) + .otherwise(lit(ScalarValue::Null))?, + true, + ), + // CASE a WHEN 1 THEN true ELSE false END + ( + case(col("a")) + .when(lit(1i64), lit(true)) + .otherwise(lit(false))?, + true, + ), + // CASE a WHEN 0 THEN false ELSE true END + ( + case(col("a")) + .when(lit(0i64), lit(false)) + .otherwise(lit(true))?, + false, + ), + // (CASE a WHEN 0 THEN false ELSE true END) OR false + ( + binary_expr( + case(col("a")) + .when(lit(0i64), lit(false)) + .otherwise(lit(true))?, + Operator::Or, + lit(false), + ), + false, + ), + // (CASE a WHEN 0 THEN true ELSE false END) OR false + ( + binary_expr( + case(col("a")) + .when(lit(0i64), lit(true)) + .otherwise(lit(false))?, + Operator::Or, + lit(false), + ), + true, + ), + // a IN (1, 2, 3) + ( + in_list(col("a"), vec![lit(1i64), lit(2i64), lit(3i64)], false), + true, + ), + // a NOT IN (1, 2, 3) + ( + in_list(col("a"), vec![lit(1i64), lit(2i64), lit(3i64)], true), + true, + ), + // a IN (NULL) + ( + in_list(col("a"), vec![Expr::Literal(ScalarValue::Null)], false), + true, + ), + // a NOT IN (NULL) + ( + in_list(col("a"), vec![Expr::Literal(ScalarValue::Null)], true), + true, + ), + ]; + + let column_a = Column::from_name("a"); + for (predicate, expected) in test_cases { + let join_cols_of_predicate = std::iter::once(&column_a); + let actual = + is_restrict_null_predicate(predicate.clone(), join_cols_of_predicate)?; + assert_eq!(actual, expected, "{}", predicate); + } + + Ok(()) + } +} diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index 03ac4769d9d9a..80c4963ae0354 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -31,7 +31,7 @@ use datafusion_common::hash_utils::create_hashes; use datafusion_common::utils::proxy::{RawTableAllocExt, VecAllocExt}; use std::any::type_name; use std::fmt::Debug; -use std::mem; +use std::mem::{size_of, swap}; use std::ops::Range; use std::sync::Arc; @@ -260,7 +260,7 @@ where /// the same output type pub fn take(&mut self) -> Self { let mut new_self = Self::new(self.output_type); - mem::swap(self, &mut new_self); + swap(self, &mut new_self); new_self } @@ -545,7 +545,7 @@ where /// this set, not including `self` pub fn size(&self) -> usize { self.map_size - + self.buffer.capacity() * mem::size_of::() + + self.buffer.capacity() * size_of::() + self.offsets.allocated_size() + self.hashes_buffer.allocated_size() } @@ -575,7 +575,7 @@ where } /// Maximum size of a value that can be inlined in the hash table -const SHORT_VALUE_LEN: usize = mem::size_of::(); +const SHORT_VALUE_LEN: usize = size_of::(); /// Entry in the hash table -- see [`ArrowBytesMap`] for more details #[derive(Debug, PartialEq, Eq, Hash, Clone, Copy)] diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 6c4bf156ce568..d825bfe7e2643 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -143,7 +143,7 @@ impl Hash for PhysicalSortExpr { } impl Display for PhysicalSortExpr { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { write!(f, "{} {}", self.expr, to_str(&self.options)) } } @@ -188,7 +188,7 @@ impl PhysicalSortExpr { pub fn format_list(input: &[PhysicalSortExpr]) -> impl Display + '_ { struct DisplayableList<'a>(&'a [PhysicalSortExpr]); impl<'a> Display for DisplayableList<'a> { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { let mut first = true; for sort_expr in self.0 { if first { @@ -260,7 +260,7 @@ impl PartialEq for PhysicalSortRequirement { } impl Display for PhysicalSortRequirement { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { let opts_string = self.options.as_ref().map_or("NA", to_str); write!(f, "{} {}", self.expr, opts_string) } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index a0cc29685f772..9a16b205ae25b 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -1113,7 +1113,7 @@ impl EquivalenceProperties { /// order: [[a ASC, b ASC], [a ASC, c ASC]], eq: [[a = b], [a = c]], const: [a = 1] /// ``` impl Display for EquivalenceProperties { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { if self.eq_group.is_empty() && self.oeq_class.is_empty() && self.constants.is_empty() diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index ffb431b200f28..981e49d73750c 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -1096,16 +1096,15 @@ mod tests { let expr2 = Arc::clone(&expr) .transform(|e| { - let transformed = - match e.as_any().downcast_ref::() { - Some(lit_value) => match lit_value.value() { - ScalarValue::Utf8(Some(str_value)) => { - Some(lit(str_value.to_uppercase())) - } - _ => None, - }, + let transformed = match e.as_any().downcast_ref::() { + Some(lit_value) => match lit_value.value() { + ScalarValue::Utf8(Some(str_value)) => { + Some(lit(str_value.to_uppercase())) + } _ => None, - }; + }, + _ => None, + }; Ok(if let Some(transformed) = transformed { Transformed::yes(transformed) } else { @@ -1117,16 +1116,15 @@ mod tests { let expr3 = Arc::clone(&expr) .transform_down(|e| { - let transformed = - match e.as_any().downcast_ref::() { - Some(lit_value) => match lit_value.value() { - ScalarValue::Utf8(Some(str_value)) => { - Some(lit(str_value.to_uppercase())) - } - _ => None, - }, + let transformed = match e.as_any().downcast_ref::() { + Some(lit_value) => match lit_value.value() { + ScalarValue::Utf8(Some(str_value)) => { + Some(lit(str_value.to_uppercase())) + } _ => None, - }; + }, + _ => None, + }; Ok(if let Some(transformed) = transformed { Transformed::yes(transformed) } else { diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 5621473c4fdb1..457c47097a19a 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -693,7 +693,7 @@ mod tests { let result = cast( col("a", &schema).unwrap(), &schema, - DataType::Interval(IntervalUnit::MonthDayNano), + Interval(IntervalUnit::MonthDayNano), ); result.expect_err("expected Invalid CAST"); } diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index 4aad959584ac4..3e2d49e9fa693 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -107,7 +107,7 @@ impl std::fmt::Display for Column { impl PhysicalExpr for Column { /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 0a3e5fcefcf6a..cf57ce3e0e21a 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -1102,7 +1102,7 @@ mod tests { let mut phy_exprs = vec![ lit(1i64), expressions::cast(lit(2i32), &schema, DataType::Int64)?, - expressions::try_cast(lit(3.13f32), &schema, DataType::Int64)?, + try_cast(lit(3.13f32), &schema, DataType::Int64)?, ]; let result = try_cast_static_filter_to_set(&phy_exprs, &schema).unwrap(); @@ -1130,7 +1130,7 @@ mod tests { try_cast_static_filter_to_set(&phy_exprs, &schema).unwrap(); // column - phy_exprs.push(expressions::col("a", &schema)?); + phy_exprs.push(col("a", &schema)?); assert!(try_cast_static_filter_to_set(&phy_exprs, &schema).is_err()); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index b5ebc250cb896..399ebde9f726d 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -257,7 +257,7 @@ mod tests { #[test] fn test_negation_valid_types() -> Result<()> { let negatable_types = [ - DataType::Int8, + Int8, DataType::Timestamp(TimeUnit::Second, None), DataType::Interval(IntervalUnit::YearMonth), ]; diff --git a/datafusion/physical-expr/src/expressions/unknown_column.rs b/datafusion/physical-expr/src/expressions/unknown_column.rs index cb7221e7fa151..590efd5779638 100644 --- a/datafusion/physical-expr/src/expressions/unknown_column.rs +++ b/datafusion/physical-expr/src/expressions/unknown_column.rs @@ -57,7 +57,7 @@ impl std::fmt::Display for UnKnownColumn { impl PhysicalExpr for UnKnownColumn { /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index f05ac3624b8e2..8084a52c78d80 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -19,6 +19,7 @@ use std::collections::HashSet; use std::fmt::{Display, Formatter}; +use std::mem::{size_of, size_of_val}; use std::sync::Arc; use super::utils::{ @@ -128,12 +129,11 @@ impl ExprIntervalGraph { /// Estimate size of bytes including `Self`. pub fn size(&self) -> usize { let node_memory_usage = self.graph.node_count() - * (std::mem::size_of::() - + std::mem::size_of::()); - let edge_memory_usage = self.graph.edge_count() - * (std::mem::size_of::() + std::mem::size_of::() * 2); + * (size_of::() + size_of::()); + let edge_memory_usage = + self.graph.edge_count() * (size_of::() + size_of::() * 2); - std::mem::size_of_val(self) + node_memory_usage + edge_memory_usage + size_of_val(self) + node_memory_usage + edge_memory_usage } } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 01f72a8efd9a5..98c0c864b9f70 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -121,8 +121,8 @@ pub enum Partitioning { UnknownPartitioning(usize), } -impl fmt::Display for Partitioning { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { +impl Display for Partitioning { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { Partitioning::RoundRobinBatch(size) => write!(f, "RoundRobinBatch({size})"), Partitioning::Hash(phy_exprs, size) => { diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs index f789af8b8a024..013c027e7306c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs @@ -19,6 +19,7 @@ use crate::aggregates::group_values::GroupValues; use arrow_array::{Array, ArrayRef, OffsetSizeTrait, RecordBatch}; use datafusion_expr::EmitTo; use datafusion_physical_expr_common::binary_map::{ArrowBytesMap, OutputType}; +use std::mem::size_of; /// A [`GroupValues`] storing single column of Utf8/LargeUtf8/Binary/LargeBinary values /// @@ -73,7 +74,7 @@ impl GroupValues for GroupValuesByes { } fn size(&self) -> usize { - self.map.size() + std::mem::size_of::() + self.map.size() + size_of::() } fn is_empty(&self) -> bool { diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs index 1a0cb90a16d47..7379b7a538b49 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs @@ -20,6 +20,7 @@ use arrow_array::{Array, ArrayRef, RecordBatch}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewMap; +use std::mem::size_of; /// A [`GroupValues`] storing single column of Utf8View/BinaryView values /// @@ -74,7 +75,7 @@ impl GroupValues for GroupValuesBytesView { } fn size(&self) -> usize { - self.map.size() + std::mem::size_of::() + self.map.size() + size_of::() } fn is_empty(&self) -> bool { diff --git a/datafusion/physical-plan/src/aggregates/group_values/column.rs b/datafusion/physical-plan/src/aggregates/group_values/column.rs index 4ad75844f7b76..958a4b58d8004 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/column.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/column.rs @@ -35,8 +35,8 @@ use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; - use hashbrown::raw::RawTable; +use std::mem::size_of; /// A [`GroupValues`] that stores multiple columns of group values. /// @@ -351,7 +351,7 @@ impl GroupValues for GroupValuesColumn { self.group_values.clear(); self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared - self.map_size = self.map.capacity() * std::mem::size_of::<(u64, usize)>(); + self.map_size = self.map.capacity() * size_of::<(u64, usize)>(); self.hashes_buffer.clear(); self.hashes_buffer.shrink_to(count); } diff --git a/datafusion/physical-plan/src/aggregates/group_values/group_column.rs b/datafusion/physical-plan/src/aggregates/group_values/group_column.rs index 41534958602e7..bba59b6d0caa5 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/group_column.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/group_column.rs @@ -37,7 +37,7 @@ use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; use arrow_array::types::GenericStringType; use datafusion_physical_expr_common::binary_map::{OutputType, INITIAL_BUFFER_CAPACITY}; use std::marker::PhantomData; -use std::mem; +use std::mem::{replace, size_of}; use std::sync::Arc; use std::vec; @@ -292,7 +292,7 @@ where } fn size(&self) -> usize { - self.buffer.capacity() * std::mem::size_of::() + self.buffer.capacity() * size_of::() + self.offsets.allocated_size() + self.nulls.allocated_size() } @@ -488,7 +488,7 @@ impl ByteViewGroupValueBuilder { // If current block isn't big enough, flush it and create a new in progress block if require_cap > self.max_block_size { - let flushed_block = mem::replace( + let flushed_block = replace( &mut self.in_progress, Vec::with_capacity(self.max_block_size), ); @@ -611,7 +611,7 @@ impl ByteViewGroupValueBuilder { // The `n == len` case, we need to take all if self.len() == n { let new_builder = Self::new().with_max_block_size(self.max_block_size); - let cur_builder = std::mem::replace(self, new_builder); + let cur_builder = replace(self, new_builder); return cur_builder.build_inner(); } @@ -759,7 +759,7 @@ impl ByteViewGroupValueBuilder { } fn flush_in_progress(&mut self) { - let flushed_block = mem::replace( + let flushed_block = replace( &mut self.in_progress, Vec::with_capacity(self.max_block_size), ); @@ -785,14 +785,14 @@ impl GroupColumn for ByteViewGroupValueBuilder { let buffers_size = self .completed .iter() - .map(|buf| buf.capacity() * std::mem::size_of::()) + .map(|buf| buf.capacity() * size_of::()) .sum::(); self.nulls.allocated_size() - + self.views.capacity() * std::mem::size_of::() - + self.in_progress.capacity() * std::mem::size_of::() + + self.views.capacity() * size_of::() + + self.in_progress.capacity() * size_of::() + buffers_size - + std::mem::size_of::() + + size_of::() } fn build(self: Box) -> ArrayRef { diff --git a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs index d5b7f1b11ac55..05214ec10d68b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs @@ -30,6 +30,7 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; use half::f16; use hashbrown::raw::RawTable; +use std::mem::size_of; use std::sync::Arc; /// A trait to allow hashing of floating point numbers @@ -151,7 +152,7 @@ where } fn size(&self) -> usize { - self.map.capacity() * std::mem::size_of::() + self.values.allocated_size() + self.map.capacity() * size_of::() + self.values.allocated_size() } fn is_empty(&self) -> bool { diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 8ca88257bf1a7..de0ae2e07dd29 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -27,6 +27,7 @@ use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use hashbrown::raw::RawTable; +use std::mem::size_of; use std::sync::Arc; /// A [`GroupValues`] making use of [`Rows`] @@ -231,10 +232,8 @@ impl GroupValues for GroupValuesRows { // https://github.com/apache/datafusion/issues/7647 for (field, array) in self.schema.fields.iter().zip(&mut output) { let expected = field.data_type(); - *array = dictionary_encode_if_necessary( - Arc::::clone(array), - expected, - )?; + *array = + dictionary_encode_if_necessary(Arc::::clone(array), expected)?; } self.group_values = Some(group_values); @@ -249,7 +248,7 @@ impl GroupValues for GroupValuesRows { }); self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared - self.map_size = self.map.capacity() * std::mem::size_of::<(u64, usize)>(); + self.map_size = self.map.capacity() * size_of::<(u64, usize)>(); self.hashes_buffer.clear(); self.hashes_buffer.shrink_to(count); } @@ -267,7 +266,7 @@ fn dictionary_encode_if_necessary( .zip(struct_array.columns()) .map(|(expected_field, column)| { dictionary_encode_if_necessary( - Arc::::clone(column), + Arc::::clone(column), expected_field.data_type(), ) }) @@ -286,13 +285,13 @@ fn dictionary_encode_if_necessary( Arc::::clone(expected_field), list.offsets().clone(), dictionary_encode_if_necessary( - Arc::::clone(list.values()), + Arc::::clone(list.values()), expected_field.data_type(), )?, list.nulls().cloned(), )?)) } (DataType::Dictionary(_, _), _) => Ok(cast(array.as_ref(), expected)?), - (_, _) => Ok(Arc::::clone(&array)), + (_, _) => Ok(Arc::::clone(&array)), } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f36bd920e83c2..48a03af19dbd5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1485,7 +1485,7 @@ mod tests { )?); let result = - common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; let expected = if spill { // In spill mode, we test with the limited memory, if the mem usage exceeds, @@ -1557,8 +1557,7 @@ mod tests { input_schema, )?); - let result = - common::collect(merged_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + let result = collect(merged_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; let batch = concat_batches(&result[0].schema(), &result)?; assert_eq!(batch.num_columns(), 4); assert_eq!(batch.num_rows(), 12); @@ -1625,7 +1624,7 @@ mod tests { )?); let result = - common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; + collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; let expected = if spill { vec![ @@ -1671,7 +1670,7 @@ mod tests { } else { Arc::clone(&task_ctx) }; - let result = common::collect(merged_aggregate.execute(0, task_ctx)?).await?; + let result = collect(merged_aggregate.execute(0, task_ctx)?).await?; let batch = concat_batches(&result[0].schema(), &result)?; assert_eq!(batch.num_columns(), 2); assert_eq!(batch.num_rows(), 3); @@ -1971,7 +1970,7 @@ mod tests { } let stream: SendableRecordBatchStream = stream.into(); - let err = common::collect(stream).await.unwrap_err(); + let err = collect(stream).await.unwrap_err(); // error root cause traversal is a bit complicated, see #4172. let err = err.find_root(); @@ -2522,7 +2521,7 @@ mod tests { let input = Arc::new(MemoryExec::try_new( &[vec![batch.clone()]], - Arc::::clone(&batch.schema()), + Arc::::clone(&batch.schema()), None, )?); let aggregate_exec = Arc::new(AggregateExec::try_new( diff --git a/datafusion/physical-plan/src/aggregates/order/full.rs b/datafusion/physical-plan/src/aggregates/order/full.rs index d64c99ba1bee3..218855459b1e2 100644 --- a/datafusion/physical-plan/src/aggregates/order/full.rs +++ b/datafusion/physical-plan/src/aggregates/order/full.rs @@ -16,6 +16,7 @@ // under the License. use datafusion_expr::EmitTo; +use std::mem::size_of; /// Tracks grouping state when the data is ordered entirely by its /// group keys @@ -139,7 +140,7 @@ impl GroupOrderingFull { } pub(crate) fn size(&self) -> usize { - std::mem::size_of::() + size_of::() } } diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 483150ee61af6..accb2fda11316 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -20,6 +20,7 @@ use arrow_schema::Schema; use datafusion_common::Result; use datafusion_expr::EmitTo; use datafusion_physical_expr::PhysicalSortExpr; +use std::mem::size_of; mod full; mod partial; @@ -118,7 +119,7 @@ impl GroupOrdering { /// Return the size of memory used by the ordering state, in bytes pub fn size(&self) -> usize { - std::mem::size_of::() + size_of::() + match self { GroupOrdering::None => 0, GroupOrdering::Partial(partial) => partial.size(), diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 2cbe3bbb784ec..2dd1ea8a5449e 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -22,6 +22,7 @@ use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; use datafusion_physical_expr::PhysicalSortExpr; +use std::mem::size_of; use std::sync::Arc; /// Tracks grouping state when the data is ordered by some subset of @@ -244,7 +245,7 @@ impl GroupOrderingPartial { /// Return the size of memory allocated by this structure pub(crate) fn size(&self) -> usize { - std::mem::size_of::() + size_of::() + self.order_indices.allocated_size() + self.row_converter.size() } diff --git a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs index 232b87de32314..34df643b6cf0c 100644 --- a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs @@ -109,7 +109,7 @@ impl StringHashTable { Self { owned, map: TopKHashTable::new(limit, limit * 10), - rnd: ahash::RandomState::default(), + rnd: RandomState::default(), } } } @@ -181,7 +181,7 @@ where Self { owned, map: TopKHashTable::new(limit, limit * 10), - rnd: ahash::RandomState::default(), + rnd: RandomState::default(), } } } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 4e936fb37a12a..e79b3c817bd1b 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -125,7 +125,7 @@ impl<'a> DisplayableExecutionPlan<'a> { show_schema: bool, } impl<'a> fmt::Display for Wrapper<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut visitor = IndentVisitor { t: self.format_type, f, @@ -164,7 +164,7 @@ impl<'a> DisplayableExecutionPlan<'a> { show_statistics: bool, } impl<'a> fmt::Display for Wrapper<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { let t = DisplayFormatType::Default; let mut visitor = GraphvizVisitor { @@ -203,7 +203,7 @@ impl<'a> DisplayableExecutionPlan<'a> { } impl<'a> fmt::Display for Wrapper<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut visitor = IndentVisitor { f, t: DisplayFormatType::Default, @@ -257,7 +257,7 @@ struct IndentVisitor<'a, 'b> { /// How to format each node t: DisplayFormatType, /// Write to this formatter - f: &'a mut fmt::Formatter<'b>, + f: &'a mut Formatter<'b>, /// Indent size indent: usize, /// How to show metrics @@ -318,7 +318,7 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { } struct GraphvizVisitor<'a, 'b> { - f: &'a mut fmt::Formatter<'b>, + f: &'a mut Formatter<'b>, /// How to format each node t: DisplayFormatType, /// How to show metrics @@ -349,8 +349,8 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { struct Wrapper<'a>(&'a dyn ExecutionPlan, DisplayFormatType); - impl<'a> std::fmt::Display for Wrapper<'a> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + impl<'a> fmt::Display for Wrapper<'a> { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { self.0.fmt_as(self.1, f) } } @@ -422,14 +422,14 @@ pub trait DisplayAs { /// different from the default one /// /// Should not include a newline - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result; + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; } /// A newtype wrapper to display `T` implementing`DisplayAs` using the `Default` mode pub struct DefaultDisplay(pub T); impl fmt::Display for DefaultDisplay { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { self.0.fmt_as(DisplayFormatType::Default, f) } } @@ -438,7 +438,7 @@ impl fmt::Display for DefaultDisplay { pub struct VerboseDisplay(pub T); impl fmt::Display for VerboseDisplay { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { self.0.fmt_as(DisplayFormatType::Verbose, f) } } @@ -448,7 +448,7 @@ impl fmt::Display for VerboseDisplay { pub struct ProjectSchemaDisplay<'a>(pub &'a SchemaRef); impl<'a> fmt::Display for ProjectSchemaDisplay<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { let parts: Vec<_> = self .0 .fields() @@ -464,7 +464,7 @@ impl<'a> fmt::Display for ProjectSchemaDisplay<'a> { pub struct OutputOrderingDisplay<'a>(pub &'a [PhysicalSortExpr]); impl<'a> fmt::Display for OutputOrderingDisplay<'a> { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { write!(f, "[")?; for (i, e) in self.0.iter().enumerate() { if i > 0 { diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index dda45ebebb0c6..8b3ef5ae01e47 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -93,7 +93,7 @@ pub struct DataSinkExec { cache: PlanProperties, } -impl fmt::Debug for DataSinkExec { +impl Debug for DataSinkExec { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "DataSinkExec schema: {:?}", self.count_schema) } @@ -148,11 +148,7 @@ impl DataSinkExec { } impl DisplayAs for DataSinkExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!(f, "DataSinkExec: sink=")?; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8f2bef56da766..8f49885068fd3 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -418,7 +418,7 @@ impl Stream for CrossJoinStream { fn poll_next( mut self: std::pin::Pin<&mut Self>, cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { + ) -> Poll> { self.poll_next_impl(cx) } } @@ -429,7 +429,7 @@ impl CrossJoinStream { fn poll_next_impl( &mut self, cx: &mut std::task::Context<'_>, - ) -> std::task::Poll>> { + ) -> Poll>> { loop { return match self.state { CrossJoinStreamState::WaitBuildSide => { diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 3b730c01291c8..2d11e03814a31 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -18,6 +18,7 @@ //! [`HashJoinExec`] Partitioned Hash Join Operator use std::fmt; +use std::mem::size_of; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; use std::task::Poll; @@ -849,7 +850,7 @@ async fn collect_left_input( // Estimation of memory size, required for hashtable, prior to allocation. // Final result can be verified using `RawTable.allocation_info()` - let fixed_size = std::mem::size_of::(); + let fixed_size = size_of::(); let estimated_hashtable_size = estimate_memory_size::<(u64, u64)>(num_rows, fixed_size)?; @@ -1524,7 +1525,7 @@ impl Stream for HashJoinStream { fn poll_next( mut self: std::pin::Pin<&mut Self>, cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { + ) -> Poll> { self.poll_next_impl(cx) } } @@ -3594,10 +3595,7 @@ mod tests { let stream = join.execute(0, task_ctx).unwrap(); // Expect that an error is returned - let result_string = crate::common::collect(stream) - .await - .unwrap_err() - .to_string(); + let result_string = common::collect(stream).await.unwrap_err().to_string(); assert!( result_string.contains("bad data error"), "actual: {result_string}" diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index d5134855440a7..b299b495c5044 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -26,7 +26,7 @@ use std::collections::{HashMap, VecDeque}; use std::fmt::Formatter; use std::fs::File; use std::io::BufReader; -use std::mem; +use std::mem::size_of; use std::ops::Range; use std::pin::Pin; use std::sync::atomic::AtomicUsize; @@ -411,13 +411,13 @@ struct SortMergeJoinMetrics { /// Total time for joining probe-side batches to the build-side batches join_time: metrics::Time, /// Number of batches consumed by this operator - input_batches: metrics::Count, + input_batches: Count, /// Number of rows consumed by this operator - input_rows: metrics::Count, + input_rows: Count, /// Number of batches produced by this operator - output_batches: metrics::Count, + output_batches: Count, /// Number of rows produced by this operator - output_rows: metrics::Count, + output_rows: Count, /// Peak memory used for buffered data. /// Calculated as sum of peak memory values across partitions peak_mem_used: metrics::Gauge, @@ -630,9 +630,9 @@ impl BufferedBatch { .iter() .map(|arr| arr.get_array_memory_size()) .sum::() - + batch.num_rows().next_power_of_two() * mem::size_of::() - + mem::size_of::>() - + mem::size_of::(); + + batch.num_rows().next_power_of_two() * size_of::() + + size_of::>() + + size_of::(); let num_rows = batch.num_rows(); BufferedBatch { @@ -802,6 +802,32 @@ fn get_corrected_filter_mask( Some(corrected_mask.finish()) } + JoinType::LeftAnti => { + for i in 0..row_indices_length { + let last_index = + last_index_for_row(i, row_indices, batch_ids, row_indices_length); + + if filter_mask.value(i) { + seen_true = true; + } + + if last_index { + if !seen_true { + corrected_mask.append_value(true); + } else { + corrected_mask.append_null(); + } + + seen_true = false; + } else { + corrected_mask.append_null(); + } + } + + let null_matched = expected_size - corrected_mask.len(); + corrected_mask.extend(vec![Some(true); null_matched]); + Some(corrected_mask.finish()) + } // Only outer joins needs to keep track of processed rows and apply corrected filter mask _ => None, } @@ -835,15 +861,18 @@ impl Stream for SMJStream { JoinType::Left | JoinType::LeftSemi | JoinType::Right + | JoinType::LeftAnti ) { self.freeze_all()?; if !self.output_record_batches.batches.is_empty() - && self.buffered_data.scanning_finished() { - let out_batch = self.filter_joined_batch()?; - return Poll::Ready(Some(Ok(out_batch))); + let out_filtered_batch = + self.filter_joined_batch()?; + return Poll::Ready(Some(Ok( + out_filtered_batch, + ))); } } @@ -907,15 +936,17 @@ impl Stream for SMJStream { // because target output batch size can be hit in the middle of // filtering causing the filtering to be incomplete and causing // correctness issues - let record_batch = if !(self.filter.is_some() + if self.filter.is_some() && matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right - )) { - record_batch - } else { + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti + ) + { continue; - }; + } return Poll::Ready(Some(Ok(record_batch))); } @@ -929,7 +960,10 @@ impl Stream for SMJStream { if self.filter.is_some() && matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti ) { let out = self.filter_joined_batch()?; @@ -1273,11 +1307,7 @@ impl SMJStream { }; if matches!(self.join_type, JoinType::LeftAnti) && self.filter.is_some() { - join_streamed = !self - .streamed_batch - .join_filter_matched_idxs - .contains(&(self.streamed_batch.idx as u64)) - && !self.streamed_joined; + join_streamed = !self.streamed_joined; join_buffered = join_streamed; } } @@ -1519,7 +1549,10 @@ impl SMJStream { // Push the filtered batch which contains rows passing join filter to the output if matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti ) { self.output_record_batches .batches @@ -1654,7 +1687,10 @@ impl SMJStream { if !(self.filter.is_some() && matches!( self.join_type, - JoinType::Left | JoinType::LeftSemi | JoinType::Right + JoinType::Left + | JoinType::LeftSemi + | JoinType::Right + | JoinType::LeftAnti )) { self.output_record_batches.batches.clear(); @@ -1727,7 +1763,7 @@ impl SMJStream { &self.schema, &[filtered_record_batch, null_joined_streamed_batch], )?; - } else if matches!(self.join_type, JoinType::LeftSemi) { + } else if matches!(self.join_type, JoinType::LeftSemi | JoinType::LeftAnti) { let output_column_indices = (0..streamed_columns_length).collect::>(); filtered_record_batch = filtered_record_batch.project(&output_column_indices)?; @@ -2296,7 +2332,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_, batches) = join_collect(left, right, on, Inner).await?; let expected = [ "+----+----+----+----+----+----+", @@ -2335,7 +2371,7 @@ mod tests { ), ]; - let (_columns, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_columns, batches) = join_collect(left, right, on, Inner).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b2 | c1 | a1 | b2 | c2 |", @@ -2373,7 +2409,7 @@ mod tests { ), ]; - let (_columns, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_columns, batches) = join_collect(left, right, on, Inner).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b2 | c1 | a1 | b2 | c2 |", @@ -2412,7 +2448,7 @@ mod tests { ), ]; - let (_, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_, batches) = join_collect(left, right, on, Inner).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b2 | c1 | a1 | b2 | c2 |", @@ -2453,7 +2489,7 @@ mod tests { left, right, on, - JoinType::Inner, + Inner, vec![ SortOptions { descending: true, @@ -2503,7 +2539,7 @@ mod tests { ]; let (_, batches) = - join_collect_batch_size_equals_two(left, right, on, JoinType::Inner).await?; + join_collect_batch_size_equals_two(left, right, on, Inner).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b2 | c1 | a1 | b2 | c2 |", @@ -2538,7 +2574,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Left).await?; + let (_, batches) = join_collect(left, right, on, Left).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b1 | c2 |", @@ -2570,7 +2606,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Right).await?; + let (_, batches) = join_collect(left, right, on, Right).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b1 | c2 |", @@ -2602,7 +2638,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Full).await?; + let (_, batches) = join_collect(left, right, on, Full).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b2 | c2 |", @@ -2634,7 +2670,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::LeftAnti).await?; + let (_, batches) = join_collect(left, right, on, LeftAnti).await?; let expected = [ "+----+----+----+", "| a1 | b1 | c1 |", @@ -2665,7 +2701,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::LeftSemi).await?; + let (_, batches) = join_collect(left, right, on, LeftSemi).await?; let expected = [ "+----+----+----+", "| a1 | b1 | c1 |", @@ -2698,7 +2734,7 @@ mod tests { Arc::new(Column::new_with_schema("b", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_, batches) = join_collect(left, right, on, Inner).await?; let expected = [ "+---+---+---+----+---+----+", "| a | b | c | a | b | c |", @@ -2730,7 +2766,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_, batches) = join_collect(left, right, on, Inner).await?; let expected = ["+------------+------------+------------+------------+------------+------------+", "| a1 | b1 | c1 | a2 | b1 | c2 |", @@ -2762,7 +2798,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Inner).await?; + let (_, batches) = join_collect(left, right, on, Inner).await?; let expected = ["+-------------------------+---------------------+-------------------------+-------------------------+---------------------+-------------------------+", "| a1 | b1 | c1 | a2 | b1 | c2 |", @@ -2793,7 +2829,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Left).await?; + let (_, batches) = join_collect(left, right, on, Left).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b2 | c2 |", @@ -2829,7 +2865,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Right).await?; + let (_, batches) = join_collect(left, right, on, Right).await?; let expected = [ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b2 | c2 |", @@ -2873,7 +2909,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Left).await?; + let (_, batches) = join_collect(left, right, on, Left).await?; let expected = vec![ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b2 | c2 |", @@ -2922,7 +2958,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Right).await?; + let (_, batches) = join_collect(left, right, on, Right).await?; let expected = vec![ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b2 | c2 |", @@ -2971,7 +3007,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema())?) as _, )]; - let (_, batches) = join_collect(left, right, on, JoinType::Full).await?; + let (_, batches) = join_collect(left, right, on, Full).await?; let expected = vec![ "+----+----+----+----+----+----+", "| a1 | b1 | c1 | a2 | b2 | c2 |", @@ -3011,14 +3047,7 @@ mod tests { )]; let sort_options = vec![SortOptions::default(); on.len()]; - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; + let join_types = vec![Inner, Left, Right, Full, LeftSemi, LeftAnti]; // Disable DiskManager to prevent spilling let runtime = RuntimeEnvBuilder::new() @@ -3096,14 +3125,7 @@ mod tests { )]; let sort_options = vec![SortOptions::default(); on.len()]; - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; + let join_types = vec![Inner, Left, Right, Full, LeftSemi, LeftAnti]; // Disable DiskManager to prevent spilling let runtime = RuntimeEnvBuilder::new() @@ -3159,14 +3181,7 @@ mod tests { )]; let sort_options = vec![SortOptions::default(); on.len()]; - let join_types = [ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; + let join_types = [Inner, Left, Right, Full, LeftSemi, LeftAnti]; // Enable DiskManager to allow spilling let runtime = RuntimeEnvBuilder::new() @@ -3267,14 +3282,7 @@ mod tests { )]; let sort_options = vec![SortOptions::default(); on.len()]; - let join_types = [ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; + let join_types = [Inner, Left, Right, Full, LeftSemi, LeftAnti]; // Enable DiskManager to allow spilling let runtime = RuntimeEnvBuilder::new() @@ -3349,6 +3357,7 @@ mod tests { batch_ids: vec![], }; + // Insert already prejoined non-filtered rows batches.batches.push(RecordBatch::try_new( Arc::clone(&schema), vec![ @@ -3835,6 +3844,178 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_left_anti_join_filtered_mask() -> Result<()> { + let mut joined_batches = build_joined_record_batches()?; + let schema = joined_batches.batches.first().unwrap().schema(); + + let output = concat_batches(&schema, &joined_batches.batches)?; + let out_mask = joined_batches.filter_mask.finish(); + let out_indices = joined_batches.row_indices.finish(); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0]), + &[0usize], + &BooleanArray::from(vec![true]), + 1 + ) + .unwrap(), + BooleanArray::from(vec![None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0]), + &[0usize], + &BooleanArray::from(vec![false]), + 1 + ) + .unwrap(), + BooleanArray::from(vec![Some(true)]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0]), + &[0usize; 2], + &BooleanArray::from(vec![true, true]), + 2 + ) + .unwrap(), + BooleanArray::from(vec![None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![true, true, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![true, false, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![false, false, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![false, true, true]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, None]) + ); + + assert_eq!( + get_corrected_filter_mask( + LeftAnti, + &UInt64Array::from(vec![0, 0, 0]), + &[0usize; 3], + &BooleanArray::from(vec![false, false, false]), + 3 + ) + .unwrap(), + BooleanArray::from(vec![None, None, Some(true)]) + ); + + let corrected_mask = get_corrected_filter_mask( + LeftAnti, + &out_indices, + &joined_batches.batch_ids, + &out_mask, + output.num_rows(), + ) + .unwrap(); + + assert_eq!( + corrected_mask, + BooleanArray::from(vec![ + None, + None, + None, + None, + None, + Some(true), + None, + Some(true) + ]) + ); + + let filtered_rb = filter_record_batch(&output, &corrected_mask)?; + + assert_batches_eq!( + &[ + "+---+----+---+----+", + "| a | b | x | y |", + "+---+----+---+----+", + "| 1 | 13 | 1 | 12 |", + "| 1 | 14 | 1 | 11 |", + "+---+----+---+----+", + ], + &[filtered_rb] + ); + + // output null rows + let null_mask = arrow::compute::not(&corrected_mask)?; + assert_eq!( + null_mask, + BooleanArray::from(vec![ + None, + None, + None, + None, + None, + Some(false), + None, + Some(false), + ]) + ); + + let null_joined_batch = filter_record_batch(&output, &null_mask)?; + + assert_batches_eq!( + &[ + "+---+---+---+---+", + "| a | b | x | y |", + "+---+---+---+---+", + "+---+---+---+---+", + ], + &[null_joined_batch] + ); + Ok(()) + } + /// Returns the column names on the schema fn columns(schema: &Schema) -> Vec { schema.fields().iter().map(|f| f.name().clone()).collect() diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index bddd152341da0..02c71dab3df23 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -19,6 +19,7 @@ //! related functionality, used both in join calculations and optimization rules. use std::collections::{HashMap, VecDeque}; +use std::mem::size_of; use std::sync::Arc; use crate::joins::utils::{JoinFilter, JoinHashMapType}; @@ -153,8 +154,7 @@ impl PruningJoinHashMap { /// # Returns /// The size of the hash map in bytes. pub(crate) fn size(&self) -> usize { - self.map.allocation_info().1.size() - + self.next.capacity() * std::mem::size_of::() + self.map.allocation_info().1.size() + self.next.capacity() * size_of::() } /// Removes hash values from the map and the list based on the given pruning diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 70ada3892aead..eb6a30d17e925 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -27,6 +27,7 @@ use std::any::Any; use std::fmt::{self, Debug}; +use std::mem::{size_of, size_of_val}; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; @@ -604,7 +605,7 @@ impl Stream for SymmetricHashJoinStream { fn poll_next( mut self: std::pin::Pin<&mut Self>, - cx: &mut std::task::Context<'_>, + cx: &mut Context<'_>, ) -> Poll> { self.poll_next_impl(cx) } @@ -1004,15 +1005,15 @@ pub struct OneSideHashJoiner { impl OneSideHashJoiner { pub fn size(&self) -> usize { let mut size = 0; - size += std::mem::size_of_val(self); - size += std::mem::size_of_val(&self.build_side); + size += size_of_val(self); + size += size_of_val(&self.build_side); size += self.input_buffer.get_array_memory_size(); - size += std::mem::size_of_val(&self.on); + size += size_of_val(&self.on); size += self.hashmap.size(); - size += self.hashes_buffer.capacity() * std::mem::size_of::(); - size += self.visited_rows.capacity() * std::mem::size_of::(); - size += std::mem::size_of_val(&self.offset); - size += std::mem::size_of_val(&self.deleted_offset); + size += self.hashes_buffer.capacity() * size_of::(); + size += self.visited_rows.capacity() * size_of::(); + size += size_of_val(&self.offset); + size += size_of_val(&self.deleted_offset); size } pub fn new( @@ -1463,18 +1464,18 @@ impl SymmetricHashJoinStream { fn size(&self) -> usize { let mut size = 0; - size += std::mem::size_of_val(&self.schema); - size += std::mem::size_of_val(&self.filter); - size += std::mem::size_of_val(&self.join_type); + size += size_of_val(&self.schema); + size += size_of_val(&self.filter); + size += size_of_val(&self.join_type); size += self.left.size(); size += self.right.size(); - size += std::mem::size_of_val(&self.column_indices); + size += size_of_val(&self.column_indices); size += self.graph.as_ref().map(|g| g.size()).unwrap_or(0); - size += std::mem::size_of_val(&self.left_sorted_filter_expr); - size += std::mem::size_of_val(&self.right_sorted_filter_expr); - size += std::mem::size_of_val(&self.random_state); - size += std::mem::size_of_val(&self.null_equals_null); - size += std::mem::size_of_val(&self.metrics); + size += size_of_val(&self.left_sorted_filter_expr); + size += size_of_val(&self.right_sorted_filter_expr); + size += size_of_val(&self.random_state); + size += size_of_val(&self.null_equals_null); + size += size_of_val(&self.metrics); size } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 17a32a67c7430..090cf9aa628a7 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -369,7 +369,7 @@ impl JoinHashMapType for JoinHashMap { } } -impl fmt::Debug for JoinHashMap { +impl Debug for JoinHashMap { fn fmt(&self, _f: &mut fmt::Formatter) -> fmt::Result { Ok(()) } @@ -727,8 +727,8 @@ impl Default for OnceAsync { } } -impl std::fmt::Debug for OnceAsync { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { +impl Debug for OnceAsync { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "OnceAsync") } } @@ -1952,13 +1952,13 @@ mod tests { ) -> Statistics { Statistics { num_rows: if is_exact { - num_rows.map(Precision::Exact) + num_rows.map(Exact) } else { - num_rows.map(Precision::Inexact) + num_rows.map(Inexact) } - .unwrap_or(Precision::Absent), + .unwrap_or(Absent), column_statistics: column_stats, - total_byte_size: Precision::Absent, + total_byte_size: Absent, } } @@ -2204,17 +2204,17 @@ mod tests { assert_eq!( estimate_inner_join_cardinality( Statistics { - num_rows: Precision::Inexact(400), - total_byte_size: Precision::Absent, + num_rows: Inexact(400), + total_byte_size: Absent, column_statistics: left_col_stats, }, Statistics { - num_rows: Precision::Inexact(400), - total_byte_size: Precision::Absent, + num_rows: Inexact(400), + total_byte_size: Absent, column_statistics: right_col_stats, }, ), - Some(Precision::Inexact((400 * 400) / 200)) + Some(Inexact((400 * 400) / 200)) ); Ok(()) } @@ -2222,33 +2222,33 @@ mod tests { #[test] fn test_inner_join_cardinality_decimal_range() -> Result<()> { let left_col_stats = vec![ColumnStatistics { - distinct_count: Precision::Absent, - min_value: Precision::Inexact(ScalarValue::Decimal128(Some(32500), 14, 4)), - max_value: Precision::Inexact(ScalarValue::Decimal128(Some(35000), 14, 4)), + distinct_count: Absent, + min_value: Inexact(ScalarValue::Decimal128(Some(32500), 14, 4)), + max_value: Inexact(ScalarValue::Decimal128(Some(35000), 14, 4)), ..Default::default() }]; let right_col_stats = vec![ColumnStatistics { - distinct_count: Precision::Absent, - min_value: Precision::Inexact(ScalarValue::Decimal128(Some(33500), 14, 4)), - max_value: Precision::Inexact(ScalarValue::Decimal128(Some(34000), 14, 4)), + distinct_count: Absent, + min_value: Inexact(ScalarValue::Decimal128(Some(33500), 14, 4)), + max_value: Inexact(ScalarValue::Decimal128(Some(34000), 14, 4)), ..Default::default() }]; assert_eq!( estimate_inner_join_cardinality( Statistics { - num_rows: Precision::Inexact(100), - total_byte_size: Precision::Absent, + num_rows: Inexact(100), + total_byte_size: Absent, column_statistics: left_col_stats, }, Statistics { - num_rows: Precision::Inexact(100), - total_byte_size: Precision::Absent, + num_rows: Inexact(100), + total_byte_size: Absent, column_statistics: right_col_stats, }, ), - Some(Precision::Inexact(100)) + Some(Inexact(100)) ); Ok(()) } diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index eda75b37fe663..1fe550a930561 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -473,7 +473,7 @@ mod tests { use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common::collect; - use crate::{common, test}; + use crate::test; use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use arrow_array::RecordBatchOptions; @@ -497,7 +497,7 @@ mod tests { // The result should contain 4 batches (one per input partition) let iter = limit.execute(0, task_ctx)?; - let batches = common::collect(iter).await?; + let batches = collect(iter).await?; // There should be a total of 100 rows let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); @@ -613,7 +613,7 @@ mod tests { // The result should contain 4 batches (one per input partition) let iter = offset.execute(0, task_ctx)?; - let batches = common::collect(iter).await?; + let batches = collect(iter).await?; Ok(batches.iter().map(|batch| batch.num_rows()).sum()) } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 52a8631d5a635..dd4868d1bfcc9 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -69,11 +69,7 @@ impl fmt::Debug for MemoryExec { } impl DisplayAs for MemoryExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { let partition_sizes: Vec<_> = diff --git a/datafusion/physical-plan/src/metrics/value.rs b/datafusion/physical-plan/src/metrics/value.rs index 5a335d9f99cd2..2eb01914ee0ac 100644 --- a/datafusion/physical-plan/src/metrics/value.rs +++ b/datafusion/physical-plan/src/metrics/value.rs @@ -168,7 +168,7 @@ impl PartialEq for Time { impl Display for Time { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - let duration = std::time::Duration::from_nanos(self.value() as u64); + let duration = Duration::from_nanos(self.value() as u64); write!(f, "{duration:?}") } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 936cf742a792d..c1d3f368366f6 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -356,7 +356,6 @@ impl RecordBatchStream for ProjectionStream { mod tests { use super::*; use crate::common::collect; - use crate::expressions; use crate::test; use arrow_schema::DataType; @@ -418,8 +417,8 @@ mod tests { let schema = get_schema(); let exprs: Vec> = vec![ - Arc::new(expressions::Column::new("col1", 1)), - Arc::new(expressions::Column::new("col0", 0)), + Arc::new(Column::new("col1", 1)), + Arc::new(Column::new("col0", 0)), ]; let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); @@ -452,8 +451,8 @@ mod tests { let schema = get_schema(); let exprs: Vec> = vec![ - Arc::new(expressions::Column::new("col2", 2)), - Arc::new(expressions::Column::new("col0", 0)), + Arc::new(Column::new("col2", 2)), + Arc::new(Column::new("col0", 0)), ]; let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); diff --git a/datafusion/physical-plan/src/repartition/distributor_channels.rs b/datafusion/physical-plan/src/repartition/distributor_channels.rs index 675d26bbfb9fc..2e5ef24beac31 100644 --- a/datafusion/physical-plan/src/repartition/distributor_channels.rs +++ b/datafusion/physical-plan/src/repartition/distributor_channels.rs @@ -829,7 +829,7 @@ mod tests { { let test_waker = Arc::new(TestWaker::default()); let waker = futures::task::waker(Arc::clone(&test_waker)); - let mut cx = std::task::Context::from_waker(&waker); + let mut cx = Context::from_waker(&waker); let res = fut.poll_unpin(&mut cx); (res, test_waker) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 90e62d6f11f82..601c1e8731523 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1326,7 +1326,7 @@ mod tests { // now, purposely drop output stream 0 // *before* any outputs are produced - std::mem::drop(output_stream0); + drop(output_stream0); // Now, start sending input let mut background_task = JoinSet::new(); @@ -1401,7 +1401,7 @@ mod tests { let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); // now, purposely drop output stream 0 // *before* any outputs are produced - std::mem::drop(output_stream0); + drop(output_stream0); let mut background_task = JoinSet::new(); background_task.spawn(async move { input.wait().await; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 8e13a2e07e490..921678a4ad923 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -815,11 +815,7 @@ impl SortExec { } impl DisplayAs for SortExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { let expr = PhysicalSortExpr::format_list(&self.expr); @@ -1018,7 +1014,7 @@ mod tests { } impl DisplayAs for SortedUnboundedExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!(f, "UnboundableExec",).unwrap() diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 3d3f9dcb98ee9..31a4ed61cf9e8 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -946,7 +946,7 @@ mod tests { while let Some(batch) = stream.next().await { sender.send(batch).await.unwrap(); // This causes the MergeStream to wait for more input - tokio::time::sleep(tokio::time::Duration::from_millis(10)).await; + tokio::time::sleep(Duration::from_millis(10)).await; } Ok(()) diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 9220646653e61..ec4c9dd502a60 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -437,12 +437,12 @@ impl ObservedStream { } impl RecordBatchStream for ObservedStream { - fn schema(&self) -> arrow::datatypes::SchemaRef { + fn schema(&self) -> SchemaRef { self.inner.schema() } } -impl futures::Stream for ObservedStream { +impl Stream for ObservedStream { type Item = Result; fn poll_next( diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 0f7c75c2c90b1..cdb94af1fe8a7 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -163,7 +163,7 @@ impl StreamingTableExec { } } -impl std::fmt::Debug for StreamingTableExec { +impl Debug for StreamingTableExec { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("LazyMemTableExec").finish_non_exhaustive() } diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index d3f1a4fd96caf..9b46ad2ec7b14 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -21,6 +21,7 @@ use arrow::{ compute::interleave, row::{RowConverter, Rows, SortField}, }; +use std::mem::size_of; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; use arrow_array::{Array, ArrayRef, RecordBatch}; @@ -225,7 +226,7 @@ impl TopK { /// return the size of memory used by this operator, in bytes fn size(&self) -> usize { - std::mem::size_of::() + size_of::() + self.row_converter.size() + self.scratch_rows.size() + self.heap.size() @@ -444,8 +445,8 @@ impl TopKHeap { /// return the size of memory used by this heap, in bytes fn size(&self) -> usize { - std::mem::size_of::() - + (self.inner.capacity() * std::mem::size_of::()) + size_of::() + + (self.inner.capacity() * size_of::()) + self.store.size() + self.owned_bytes } @@ -636,9 +637,8 @@ impl RecordBatchStore { /// returns the size of memory used by this store, including all /// referenced `RecordBatch`es, in bytes pub fn size(&self) -> usize { - std::mem::size_of::() - + self.batches.capacity() - * (std::mem::size_of::() + std::mem::size_of::()) + size_of::() + + self.batches.capacity() * (size_of::() + size_of::()) + self.batches_size } } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 40ec3830ea0c6..3e312b7451bef 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -984,7 +984,7 @@ mod tests { list_array: &dyn ListArrayType, lengths: Vec, expected: Vec>, - ) -> datafusion_common::Result<()> { + ) -> Result<()> { let length_array = Int64Array::from(lengths); let unnested_array = unnest_list_array(list_array, &length_array, 3 * 6)?; let strs = unnested_array.as_string::().iter().collect::>(); @@ -993,7 +993,7 @@ mod tests { } #[test] - fn test_build_batch_list_arr_recursive() -> datafusion_common::Result<()> { + fn test_build_batch_list_arr_recursive() -> Result<()> { // col1 | col2 // [[1,2,3],null,[4,5]] | ['a','b'] // [[7,8,9,10], null, [11,12,13]] | ['c','d'] @@ -1125,7 +1125,7 @@ mod tests { } #[test] - fn test_unnest_list_array() -> datafusion_common::Result<()> { + fn test_unnest_list_array() -> Result<()> { // [A, B, C], [], NULL, [D], NULL, [NULL, F] let list_array = make_generic_array::(); verify_unnest_list_array( @@ -1173,7 +1173,7 @@ mod tests { list_arrays: &[ArrayRef], preserve_nulls: bool, expected: Vec, - ) -> datafusion_common::Result<()> { + ) -> Result<()> { let options = UnnestOptions { preserve_nulls, recursions: vec![], @@ -1191,7 +1191,7 @@ mod tests { } #[test] - fn test_longest_list_length() -> datafusion_common::Result<()> { + fn test_longest_list_length() -> Result<()> { // Test with single ListArray // [A, B, C], [], NULL, [D], NULL, [NULL, F] let list_array = Arc::new(make_generic_array::()) as ArrayRef; @@ -1223,7 +1223,7 @@ mod tests { } #[test] - fn test_create_take_indicies() -> datafusion_common::Result<()> { + fn test_create_take_indicies() -> Result<()> { let length_array = Int64Array::from(vec![2, 3, 1]); let take_indicies = create_take_indicies(&length_array, 6); let expected = Int64Array::from(vec![0, 0, 1, 1, 1, 2]); diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 3ffe5e3e76e70..9e4b331a01bfa 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -32,9 +32,6 @@ rust-version = "1.79" # Exclude proto files so crates.io consumers don't need protoc exclude = ["*.proto"] -[lints] -workspace = true - [lib] name = "datafusion_proto" path = "src/lib.rs" diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index d0f82ecac62c6..02be3e11c1cbe 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -161,7 +161,7 @@ impl LogicalExtensionCodec for CsvLogicalExtensionCodec { &self, _buf: &[u8], _inputs: &[datafusion_expr::LogicalPlan], - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result { not_impl_err!("Method not implemented") } @@ -179,7 +179,7 @@ impl LogicalExtensionCodec for CsvLogicalExtensionCodec { _buf: &[u8], _table_ref: &TableReference, _schema: arrow::datatypes::SchemaRef, - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result> { not_impl_err!("Method not implemented") } @@ -271,7 +271,7 @@ impl LogicalExtensionCodec for JsonLogicalExtensionCodec { &self, _buf: &[u8], _inputs: &[datafusion_expr::LogicalPlan], - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result { not_impl_err!("Method not implemented") } @@ -289,7 +289,7 @@ impl LogicalExtensionCodec for JsonLogicalExtensionCodec { _buf: &[u8], _table_ref: &TableReference, _schema: arrow::datatypes::SchemaRef, - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result> { not_impl_err!("Method not implemented") } @@ -570,7 +570,7 @@ impl LogicalExtensionCodec for ParquetLogicalExtensionCodec { &self, _buf: &[u8], _inputs: &[datafusion_expr::LogicalPlan], - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result { not_impl_err!("Method not implemented") } @@ -588,7 +588,7 @@ impl LogicalExtensionCodec for ParquetLogicalExtensionCodec { _buf: &[u8], _table_ref: &TableReference, _schema: arrow::datatypes::SchemaRef, - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result> { not_impl_err!("Method not implemented") } @@ -658,7 +658,7 @@ impl LogicalExtensionCodec for ArrowLogicalExtensionCodec { &self, _buf: &[u8], _inputs: &[datafusion_expr::LogicalPlan], - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result { not_impl_err!("Method not implemented") } @@ -676,7 +676,7 @@ impl LogicalExtensionCodec for ArrowLogicalExtensionCodec { _buf: &[u8], _table_ref: &TableReference, _schema: arrow::datatypes::SchemaRef, - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result> { not_impl_err!("Method not implemented") } @@ -716,7 +716,7 @@ impl LogicalExtensionCodec for AvroLogicalExtensionCodec { &self, _buf: &[u8], _inputs: &[datafusion_expr::LogicalPlan], - _ctx: &datafusion::prelude::SessionContext, + _ctx: &SessionContext, ) -> datafusion_common::Result { not_impl_err!("Method not implemented") } @@ -734,7 +734,7 @@ impl LogicalExtensionCodec for AvroLogicalExtensionCodec { _buf: &[u8], _table_ref: &TableReference, _schema: arrow::datatypes::SchemaRef, - _cts: &datafusion::prelude::SessionContext, + _cts: &SessionContext, ) -> datafusion_common::Result> { not_impl_err!("Method not implemented") } diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index d80c6b716537d..b90ae88aa74ab 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -450,7 +450,7 @@ impl AsLogicalPlan for LogicalPlanNode { )? .build() } - LogicalPlanType::CustomScan(scan) => { + CustomScan(scan) => { let schema: Schema = convert_required!(scan.schema)?; let schema = Arc::new(schema); let mut projection = None; @@ -844,13 +844,13 @@ impl AsLogicalPlan for LogicalPlanNode { .prepare(prepare.name.clone(), data_types)? .build() } - LogicalPlanType::DropView(dropview) => Ok(datafusion_expr::LogicalPlan::Ddl( - datafusion_expr::DdlStatement::DropView(DropView { + LogicalPlanType::DropView(dropview) => { + Ok(LogicalPlan::Ddl(DdlStatement::DropView(DropView { name: from_table_reference(dropview.name.as_ref(), "DropView")?, if_exists: dropview.if_exists, schema: Arc::new(convert_required!(dropview.schema)?), - }), - )), + }))) + } LogicalPlanType::CopyTo(copy) => { let input: LogicalPlan = into_logical_plan!(copy.input, ctx, extension_codec)?; @@ -859,20 +859,18 @@ impl AsLogicalPlan for LogicalPlanNode { extension_codec.try_decode_file_format(©.file_type, ctx)?, ); - Ok(datafusion_expr::LogicalPlan::Copy( - datafusion_expr::dml::CopyTo { - input: Arc::new(input), - output_url: copy.output_url.clone(), - partition_by: copy.partition_by.clone(), - file_type, - options: Default::default(), - }, - )) + Ok(LogicalPlan::Copy(dml::CopyTo { + input: Arc::new(input), + output_url: copy.output_url.clone(), + partition_by: copy.partition_by.clone(), + file_type, + options: Default::default(), + })) } LogicalPlanType::Unnest(unnest) => { let input: LogicalPlan = into_logical_plan!(unnest.input, ctx, extension_codec)?; - Ok(datafusion_expr::LogicalPlan::Unnest(Unnest { + Ok(LogicalPlan::Unnest(Unnest { input: Arc::new(input), exec_columns: unnest.exec_columns.iter().map(|c| c.into()).collect(), list_type_columns: unnest @@ -926,7 +924,7 @@ impl AsLogicalPlan for LogicalPlanNode { } as u64; let values_list = serialize_exprs(values.iter().flatten(), extension_codec)?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Values( protobuf::ValuesNode { n_cols, @@ -1018,7 +1016,7 @@ impl AsLogicalPlan for LogicalPlanNode { exprs_vec.push(expr_vec); } - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::ListingScan( protobuf::ListingTableScanNode { file_format_type: Some(file_format_type), @@ -1044,12 +1042,12 @@ impl AsLogicalPlan for LogicalPlanNode { )), }) } else if let Some(view_table) = source.downcast_ref::() { - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::ViewScan(Box::new( protobuf::ViewTableScanNode { table_name: Some(table_name.clone().into()), input: Some(Box::new( - protobuf::LogicalPlanNode::try_from_logical_plan( + LogicalPlanNode::try_from_logical_plan( view_table.logical_plan(), extension_codec, )?, @@ -1082,11 +1080,11 @@ impl AsLogicalPlan for LogicalPlanNode { } } LogicalPlan::Projection(Projection { expr, input, .. }) => { - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Projection(Box::new( protobuf::ProjectionNode { input: Some(Box::new( - protobuf::LogicalPlanNode::try_from_logical_plan( + LogicalPlanNode::try_from_logical_plan( input.as_ref(), extension_codec, )?, @@ -1098,12 +1096,11 @@ impl AsLogicalPlan for LogicalPlanNode { }) } LogicalPlan::Filter(filter) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - filter.input.as_ref(), - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + filter.input.as_ref(), + extension_codec, + )?; + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Selection(Box::new( protobuf::SelectionNode { input: Some(Box::new(input)), @@ -1116,12 +1113,11 @@ impl AsLogicalPlan for LogicalPlanNode { }) } LogicalPlan::Distinct(Distinct::All(input)) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Distinct(Box::new( protobuf::DistinctNode { input: Some(Box::new(input)), @@ -1136,16 +1132,15 @@ impl AsLogicalPlan for LogicalPlanNode { input, .. })) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; let sort_expr = match sort_expr { None => vec![], Some(sort_expr) => serialize_sorts(sort_expr, extension_codec)?, }; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::DistinctOn(Box::new( protobuf::DistinctOnNode { on_expr: serialize_exprs(on_expr, extension_codec)?, @@ -1159,12 +1154,11 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlan::Window(Window { input, window_expr, .. }) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Window(Box::new( protobuf::WindowNode { input: Some(Box::new(input)), @@ -1179,12 +1173,11 @@ impl AsLogicalPlan for LogicalPlanNode { input, .. }) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Aggregate(Box::new( protobuf::AggregateNode { input: Some(Box::new(input)), @@ -1204,16 +1197,14 @@ impl AsLogicalPlan for LogicalPlanNode { null_equals_null, .. }) => { - let left: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - left.as_ref(), - extension_codec, - )?; - let right: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - right.as_ref(), - extension_codec, - )?; + let left: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + left.as_ref(), + extension_codec, + )?; + let right: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + right.as_ref(), + extension_codec, + )?; let (left_join_key, right_join_key) = on .iter() .map(|(l, r)| { @@ -1232,7 +1223,7 @@ impl AsLogicalPlan for LogicalPlanNode { .as_ref() .map(|e| serialize_expr(e, extension_codec)) .map_or(Ok(None), |v| v.map(Some))?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Join(Box::new( protobuf::JoinNode { left: Some(Box::new(left)), @@ -1251,12 +1242,11 @@ impl AsLogicalPlan for LogicalPlanNode { not_impl_err!("LogicalPlan serde is not yet implemented for subqueries") } LogicalPlan::SubqueryAlias(SubqueryAlias { input, alias, .. }) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::SubqueryAlias(Box::new( protobuf::SubqueryAliasNode { input: Some(Box::new(input)), @@ -1266,11 +1256,10 @@ impl AsLogicalPlan for LogicalPlanNode { }) } LogicalPlan::Limit(limit) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - limit.input.as_ref(), - extension_codec, - )?; + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + limit.input.as_ref(), + extension_codec, + )?; let SkipType::Literal(skip) = limit.get_skip_type()? else { return Err(proto_error( "LogicalPlan::Limit only supports literal skip values", @@ -1282,7 +1271,7 @@ impl AsLogicalPlan for LogicalPlanNode { )); }; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Limit(Box::new( protobuf::LimitNode { input: Some(Box::new(input)), @@ -1293,14 +1282,13 @@ impl AsLogicalPlan for LogicalPlanNode { }) } LogicalPlan::Sort(Sort { input, expr, fetch }) => { - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; let sort_expr: Vec = serialize_sorts(expr, extension_codec)?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Sort(Box::new( protobuf::SortNode { input: Some(Box::new(input)), @@ -1315,11 +1303,10 @@ impl AsLogicalPlan for LogicalPlanNode { partitioning_scheme, }) => { use datafusion::logical_expr::Partitioning; - let input: protobuf::LogicalPlanNode = - protobuf::LogicalPlanNode::try_from_logical_plan( - input.as_ref(), - extension_codec, - )?; + let input: LogicalPlanNode = LogicalPlanNode::try_from_logical_plan( + input.as_ref(), + extension_codec, + )?; // Assumed common usize field was batch size // Used u64 to avoid any nastyness involving large values, most data clusters are probably uniformly 64 bits any ways @@ -1340,7 +1327,7 @@ impl AsLogicalPlan for LogicalPlanNode { } }; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Repartition(Box::new( protobuf::RepartitionNode { input: Some(Box::new(input)), @@ -1351,7 +1338,7 @@ impl AsLogicalPlan for LogicalPlanNode { } LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row, .. - }) => Ok(protobuf::LogicalPlanNode { + }) => Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::EmptyRelation( protobuf::EmptyRelationNode { produce_one_row: *produce_one_row, @@ -1390,7 +1377,7 @@ impl AsLogicalPlan for LogicalPlanNode { .insert(col_name.clone(), serialize_expr(expr, extension_codec)?); } - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CreateExternalTable( protobuf::CreateExternalTableNode { name: Some(name.clone().into()), @@ -1416,7 +1403,7 @@ impl AsLogicalPlan for LogicalPlanNode { or_replace, definition, temporary, - })) => Ok(protobuf::LogicalPlanNode { + })) => Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CreateView(Box::new( protobuf::CreateViewNode { name: Some(name.clone().into()), @@ -1436,7 +1423,7 @@ impl AsLogicalPlan for LogicalPlanNode { if_not_exists, schema: df_schema, }, - )) => Ok(protobuf::LogicalPlanNode { + )) => Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CreateCatalogSchema( protobuf::CreateCatalogSchemaNode { schema_name: schema_name.clone(), @@ -1449,7 +1436,7 @@ impl AsLogicalPlan for LogicalPlanNode { catalog_name, if_not_exists, schema: df_schema, - })) => Ok(protobuf::LogicalPlanNode { + })) => Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CreateCatalog( protobuf::CreateCatalogNode { catalog_name: catalog_name.clone(), @@ -1459,11 +1446,11 @@ impl AsLogicalPlan for LogicalPlanNode { )), }), LogicalPlan::Analyze(a) => { - let input = protobuf::LogicalPlanNode::try_from_logical_plan( + let input = LogicalPlanNode::try_from_logical_plan( a.input.as_ref(), extension_codec, )?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Analyze(Box::new( protobuf::AnalyzeNode { input: Some(Box::new(input)), @@ -1473,11 +1460,11 @@ impl AsLogicalPlan for LogicalPlanNode { }) } LogicalPlan::Explain(a) => { - let input = protobuf::LogicalPlanNode::try_from_logical_plan( + let input = LogicalPlanNode::try_from_logical_plan( a.plan.as_ref(), extension_codec, )?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Explain(Box::new( protobuf::ExplainNode { input: Some(Box::new(input)), @@ -1490,14 +1477,9 @@ impl AsLogicalPlan for LogicalPlanNode { let inputs: Vec = union .inputs .iter() - .map(|i| { - protobuf::LogicalPlanNode::try_from_logical_plan( - i, - extension_codec, - ) - }) + .map(|i| LogicalPlanNode::try_from_logical_plan(i, extension_codec)) .collect::>()?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Union( protobuf::UnionNode { inputs }, )), @@ -1511,15 +1493,10 @@ impl AsLogicalPlan for LogicalPlanNode { .node .inputs() .iter() - .map(|i| { - protobuf::LogicalPlanNode::try_from_logical_plan( - i, - extension_codec, - ) - }) + .map(|i| LogicalPlanNode::try_from_logical_plan(i, extension_codec)) .collect::>()?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Extension( LogicalExtensionNode { node: buf, inputs }, )), @@ -1530,11 +1507,9 @@ impl AsLogicalPlan for LogicalPlanNode { data_types, input, }) => { - let input = protobuf::LogicalPlanNode::try_from_logical_plan( - input, - extension_codec, - )?; - Ok(protobuf::LogicalPlanNode { + let input = + LogicalPlanNode::try_from_logical_plan(input, extension_codec)?; + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Prepare(Box::new( protobuf::PrepareNode { name: name.clone(), @@ -1556,10 +1531,8 @@ impl AsLogicalPlan for LogicalPlanNode { schema, options, }) => { - let input = protobuf::LogicalPlanNode::try_from_logical_plan( - input, - extension_codec, - )?; + let input = + LogicalPlanNode::try_from_logical_plan(input, extension_codec)?; let proto_unnest_list_items = list_type_columns .iter() .map(|(index, ul)| ColumnUnnestListItem { @@ -1570,7 +1543,7 @@ impl AsLogicalPlan for LogicalPlanNode { }), }) .collect(); - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Unnest(Box::new( protobuf::UnnestNode { input: Some(Box::new(input)), @@ -1606,7 +1579,7 @@ impl AsLogicalPlan for LogicalPlanNode { name, if_exists, schema, - })) => Ok(protobuf::LogicalPlanNode { + })) => Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::DropView( protobuf::DropViewNode { name: Some(name.clone().into()), @@ -1637,15 +1610,13 @@ impl AsLogicalPlan for LogicalPlanNode { partition_by, .. }) => { - let input = protobuf::LogicalPlanNode::try_from_logical_plan( - input, - extension_codec, - )?; + let input = + LogicalPlanNode::try_from_logical_plan(input, extension_codec)?; let mut buf = Vec::new(); extension_codec .try_encode_file_format(&mut buf, file_type_to_format(file_type)?)?; - Ok(protobuf::LogicalPlanNode { + Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::CopyTo(Box::new( protobuf::CopyToNode { input: Some(Box::new(input)), diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 634ae284c955b..326c7acab3928 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -851,7 +851,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { "physical_plan::from_proto() Unexpected expr {self:?}" )) })?; - if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { + if let ExprType::Sort(sort_expr) = expr { let expr = sort_expr .expr .as_ref() @@ -898,7 +898,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { "physical_plan::from_proto() Unexpected expr {self:?}" )) })?; - if let protobuf::physical_expr_node::ExprType::Sort(sort_expr) = expr { + if let ExprType::Sort(sort_expr) = expr { let expr = sort_expr .expr .as_ref() @@ -1713,9 +1713,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { nulls_first: expr.options.nulls_first, }); Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Sort( - sort_expr, - )), + expr_type: Some(ExprType::Sort(sort_expr)), }) }) .collect::>>()?; @@ -1782,9 +1780,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { nulls_first: expr.options.nulls_first, }); Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::Sort( - sort_expr, - )), + expr_type: Some(ExprType::Sort(sort_expr)), }) }) .collect::>>()?; diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 3fec7d1c6ea0f..14d91913e7cdd 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -2171,7 +2171,7 @@ fn roundtrip_aggregate_udf() { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } @@ -2395,7 +2395,7 @@ fn roundtrip_window() { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index 90be576a884e5..1eef1b718ba6f 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -56,6 +56,7 @@ strum = { version = "0.26.1", features = ["derive"] } ctor = { workspace = true } datafusion-functions = { workspace = true, default-features = true } datafusion-functions-aggregate = { workspace = true } +datafusion-functions-nested = { workspace = true } datafusion-functions-window = { workspace = true } env_logger = { workspace = true } paste = "^1.0" diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 34e119c45fdfe..432e8668c52e9 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -57,7 +57,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { enum StackEntry { SQLExpr(Box), - Operator(sqlparser::ast::BinaryOperator), + Operator(BinaryOperator), } // Virtual stack machine to convert SQLExpr to Expr diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 842a1c0cbec1d..1ef009132f9e3 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -53,7 +53,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // so we need to process `SELECT` and `ORDER BY` together. let oby_exprs = to_order_by_exprs(query.order_by)?; let plan = self.select_to_plan(*select, oby_exprs, planner_context)?; - let plan = self.limit(plan, query.offset, query.limit)?; + let plan = + self.limit(plan, query.offset, query.limit, planner_context)?; // Process the `SELECT INTO` after `LIMIT`. self.select_into(plan, select_into) } @@ -68,7 +69,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { None, )?; let plan = self.order_by(plan, order_by_rex)?; - self.limit(plan, query.offset, query.limit) + self.limit(plan, query.offset, query.limit, planner_context) } } } @@ -79,6 +80,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { input: LogicalPlan, skip: Option, fetch: Option, + planner_context: &mut PlannerContext, ) -> Result { if skip.is_none() && fetch.is_none() { return Ok(input); @@ -88,10 +90,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let empty_schema = DFSchema::empty(); let skip = skip - .map(|o| self.sql_to_expr(o.value, &empty_schema, &mut PlannerContext::new())) + .map(|o| self.sql_to_expr(o.value, &empty_schema, planner_context)) .transpose()?; let fetch = fetch - .map(|e| self.sql_to_expr(e, &empty_schema, &mut PlannerContext::new())) + .map(|e| self.sql_to_expr(e, &empty_schema, planner_context)) .transpose()?; LogicalPlanBuilder::from(input) .limit_by_expr(skip, fetch)? diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 29852be3bf777..abb9912b712a7 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -99,7 +99,7 @@ fn calc_inline_constraints_from_columns(columns: &[ColumnDef]) -> Vec constraints.push(ast::TableConstraint::Unique { + } => constraints.push(TableConstraint::Unique { name: name.clone(), columns: vec![column.name.clone()], characteristics: *characteristics, @@ -111,7 +111,7 @@ fn calc_inline_constraints_from_columns(columns: &[ColumnDef]) -> Vec constraints.push(ast::TableConstraint::PrimaryKey { + } => constraints.push(TableConstraint::PrimaryKey { name: name.clone(), columns: vec![column.name.clone()], characteristics: *characteristics, @@ -125,7 +125,7 @@ fn calc_inline_constraints_from_columns(columns: &[ColumnDef]) -> Vec constraints.push(ast::TableConstraint::ForeignKey { + } => constraints.push(TableConstraint::ForeignKey { name: name.clone(), columns: vec![], foreign_table: foreign_table.clone(), @@ -135,7 +135,7 @@ fn calc_inline_constraints_from_columns(columns: &[ColumnDef]) -> Vec { - constraints.push(ast::TableConstraint::Check { + constraints.push(TableConstraint::Check { name: name.clone(), expr: Box::new(expr.clone()), }) @@ -776,7 +776,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } let isolation_level: ast::TransactionIsolationLevel = modes .iter() - .filter_map(|m: &ast::TransactionMode| match m { + .filter_map(|m: &TransactionMode| match m { TransactionMode::AccessMode(_) => None, TransactionMode::IsolationLevel(level) => Some(level), }) @@ -785,7 +785,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .unwrap_or(ast::TransactionIsolationLevel::Serializable); let access_mode: ast::TransactionAccessMode = modes .iter() - .filter_map(|m: &ast::TransactionMode| match m { + .filter_map(|m: &TransactionMode| match m { TransactionMode::AccessMode(mode) => Some(mode), TransactionMode::IsolationLevel(_) => None, }) @@ -1650,7 +1650,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { None => { // If the target table has an alias, use it to qualify the column name if let Some(alias) = &table_alias { - datafusion_expr::Expr::Column(Column::new( + Expr::Column(Column::new( Some(self.ident_normalizer.normalize(alias.name.clone())), field.name(), )) diff --git a/datafusion/sql/src/unparser/ast.rs b/datafusion/sql/src/unparser/ast.rs index 71ff712985cdb..2de1ce9125a7d 100644 --- a/datafusion/sql/src/unparser/ast.rs +++ b/datafusion/sql/src/unparser/ast.rs @@ -182,7 +182,28 @@ impl SelectBuilder { self } pub fn selection(&mut self, value: Option) -> &mut Self { - self.selection = value; + // With filter pushdown optimization, the LogicalPlan can have filters defined as part of `TableScan` and `Filter` nodes. + // To avoid overwriting one of the filters, we combine the existing filter with the additional filter. + // Example: | + // | Projection: customer.c_phone AS cntrycode, customer.c_acctbal | + // | Filter: CAST(customer.c_acctbal AS Decimal128(38, 6)) > () | + // | Subquery: + // | .. | + // | TableScan: customer, full_filters=[customer.c_mktsegment = Utf8("BUILDING")] + match (&self.selection, value) { + (Some(existing_selection), Some(new_selection)) => { + self.selection = Some(ast::Expr::BinaryOp { + left: Box::new(existing_selection.clone()), + op: ast::BinaryOperator::And, + right: Box::new(new_selection), + }); + } + (None, Some(new_selection)) => { + self.selection = Some(new_selection); + } + (_, None) => (), + } + self } pub fn group_by(&mut self, value: ast::GroupByExpr) -> &mut Self { diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 02934a004d6f7..88159ab6df15c 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -59,8 +59,8 @@ pub trait Dialect: Send + Sync { /// Does the dialect use DOUBLE PRECISION to represent Float64 rather than DOUBLE? /// E.g. Postgres uses DOUBLE PRECISION instead of DOUBLE - fn float64_ast_dtype(&self) -> sqlparser::ast::DataType { - sqlparser::ast::DataType::Double + fn float64_ast_dtype(&self) -> ast::DataType { + ast::DataType::Double } /// The SQL type to use for Arrow Utf8 unparsing @@ -110,8 +110,8 @@ pub trait Dialect: Send + Sync { /// The SQL type to use for Arrow Date32 unparsing /// Most dialects use Date, but some, like SQLite require TEXT - fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { - sqlparser::ast::DataType::Date + fn date32_cast_dtype(&self) -> ast::DataType { + ast::DataType::Date } /// Does the dialect support specifying column aliases as part of alias table definition? @@ -197,8 +197,8 @@ impl Dialect for PostgreSqlDialect { IntervalStyle::PostgresVerbose } - fn float64_ast_dtype(&self) -> sqlparser::ast::DataType { - sqlparser::ast::DataType::DoublePrecision + fn float64_ast_dtype(&self) -> ast::DataType { + ast::DataType::DoublePrecision } fn scalar_function_to_sql_overrides( @@ -245,7 +245,7 @@ impl PostgreSqlDialect { } Ok(ast::Expr::Function(Function { - name: ast::ObjectName(vec![Ident { + name: ObjectName(vec![Ident { value: func_name.to_string(), quote_style: None, }]), @@ -335,8 +335,8 @@ impl Dialect for SqliteDialect { DateFieldExtractStyle::Strftime } - fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { - sqlparser::ast::DataType::Text + fn date32_cast_dtype(&self) -> ast::DataType { + ast::DataType::Text } fn supports_column_alias_in_table_alias(&self) -> bool { @@ -362,7 +362,7 @@ pub struct CustomDialect { supports_nulls_first_in_sort: bool, use_timestamp_for_date64: bool, interval_style: IntervalStyle, - float64_ast_dtype: sqlparser::ast::DataType, + float64_ast_dtype: ast::DataType, utf8_cast_dtype: ast::DataType, large_utf8_cast_dtype: ast::DataType, date_field_extract_style: DateFieldExtractStyle, @@ -370,7 +370,7 @@ pub struct CustomDialect { int32_cast_dtype: ast::DataType, timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, - date32_cast_dtype: sqlparser::ast::DataType, + date32_cast_dtype: ast::DataType, supports_column_alias_in_table_alias: bool, requires_derived_table_alias: bool, } @@ -382,7 +382,7 @@ impl Default for CustomDialect { supports_nulls_first_in_sort: true, use_timestamp_for_date64: false, interval_style: IntervalStyle::SQLStandard, - float64_ast_dtype: sqlparser::ast::DataType::Double, + float64_ast_dtype: ast::DataType::Double, utf8_cast_dtype: ast::DataType::Varchar(None), large_utf8_cast_dtype: ast::DataType::Text, date_field_extract_style: DateFieldExtractStyle::DatePart, @@ -393,7 +393,7 @@ impl Default for CustomDialect { None, TimezoneInfo::WithTimeZone, ), - date32_cast_dtype: sqlparser::ast::DataType::Date, + date32_cast_dtype: ast::DataType::Date, supports_column_alias_in_table_alias: true, requires_derived_table_alias: false, } @@ -428,7 +428,7 @@ impl Dialect for CustomDialect { self.interval_style } - fn float64_ast_dtype(&self) -> sqlparser::ast::DataType { + fn float64_ast_dtype(&self) -> ast::DataType { self.float64_ast_dtype.clone() } @@ -464,7 +464,7 @@ impl Dialect for CustomDialect { } } - fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + fn date32_cast_dtype(&self) -> ast::DataType { self.date32_cast_dtype.clone() } @@ -509,7 +509,7 @@ pub struct CustomDialectBuilder { supports_nulls_first_in_sort: bool, use_timestamp_for_date64: bool, interval_style: IntervalStyle, - float64_ast_dtype: sqlparser::ast::DataType, + float64_ast_dtype: ast::DataType, utf8_cast_dtype: ast::DataType, large_utf8_cast_dtype: ast::DataType, date_field_extract_style: DateFieldExtractStyle, @@ -535,7 +535,7 @@ impl CustomDialectBuilder { supports_nulls_first_in_sort: true, use_timestamp_for_date64: false, interval_style: IntervalStyle::PostgresVerbose, - float64_ast_dtype: sqlparser::ast::DataType::Double, + float64_ast_dtype: ast::DataType::Double, utf8_cast_dtype: ast::DataType::Varchar(None), large_utf8_cast_dtype: ast::DataType::Text, date_field_extract_style: DateFieldExtractStyle::DatePart, @@ -546,7 +546,7 @@ impl CustomDialectBuilder { None, TimezoneInfo::WithTimeZone, ), - date32_cast_dtype: sqlparser::ast::DataType::Date, + date32_cast_dtype: ast::DataType::Date, supports_column_alias_in_table_alias: true, requires_derived_table_alias: false, } @@ -604,10 +604,7 @@ impl CustomDialectBuilder { } /// Customize the dialect with a specific SQL type for Float64 casting: DOUBLE, DOUBLE PRECISION, etc. - pub fn with_float64_ast_dtype( - mut self, - float64_ast_dtype: sqlparser::ast::DataType, - ) -> Self { + pub fn with_float64_ast_dtype(mut self, float64_ast_dtype: ast::DataType) -> Self { self.float64_ast_dtype = float64_ast_dtype; self } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 8864c97bb1ffb..b41b0a54b86f0 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use datafusion_expr::expr::Unnest; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ self, BinaryOperator, Expr as AstExpr, Function, Ident, Interval, ObjectName, @@ -247,7 +248,7 @@ impl Unparser<'_> { })); Ok(ast::Expr::Function(Function { - name: ast::ObjectName(vec![Ident { + name: ObjectName(vec![Ident { value: func_name.to_string(), quote_style: None, }]), @@ -291,7 +292,7 @@ impl Unparser<'_> { None => None, }; Ok(ast::Expr::Function(Function { - name: ast::ObjectName(vec![Ident { + name: ObjectName(vec![Ident { value: func_name.to_string(), quote_style: None, }]), @@ -466,7 +467,7 @@ impl Unparser<'_> { Ok(ast::Expr::Value(ast::Value::Placeholder(p.id.to_string()))) } Expr::OuterReferenceColumn(_, col) => self.col_to_sql(col), - Expr::Unnest(_) => not_impl_err!("Unsupported Expr conversion: {expr:?}"), + Expr::Unnest(unnest) => self.unnest_to_sql(unnest), } } @@ -477,7 +478,7 @@ impl Unparser<'_> { ) -> Result { let args = self.function_args_to_sql(args)?; Ok(ast::Expr::Function(Function { - name: ast::ObjectName(vec![Ident { + name: ObjectName(vec![Ident { value: func_name.to_string(), quote_style: None, }]), @@ -518,7 +519,7 @@ impl Unparser<'_> { fn ast_type_for_date64_in_cast(&self) -> ast::DataType { if self.dialect.use_timestamp_for_date64() { - ast::DataType::Timestamp(None, ast::TimezoneInfo::None) + ast::DataType::Timestamp(None, TimezoneInfo::None) } else { ast::DataType::Datetime(None) } @@ -593,16 +594,16 @@ impl Unparser<'_> { } /// This function can create an identifier with or without quotes based on the dialect rules - pub(super) fn new_ident_quoted_if_needs(&self, ident: String) -> ast::Ident { + pub(super) fn new_ident_quoted_if_needs(&self, ident: String) -> Ident { let quote_style = self.dialect.identifier_quote_style(&ident); - ast::Ident { + Ident { value: ident, quote_style, } } - pub(super) fn new_ident_without_quote_style(&self, str: String) -> ast::Ident { - ast::Ident { + pub(super) fn new_ident_without_quote_style(&self, str: String) -> Ident { + Ident { value: str, quote_style: None, } @@ -612,7 +613,7 @@ impl Unparser<'_> { &self, lhs: ast::Expr, rhs: ast::Expr, - op: ast::BinaryOperator, + op: BinaryOperator, ) -> ast::Expr { ast::Expr::BinaryOp { left: Box::new(lhs), @@ -697,7 +698,7 @@ impl Unparser<'_> { // Closest precedence we currently have to Between is PGLikeMatch // (https://www.postgresql.org/docs/7.2/sql-precedence.html) ast::Expr::Between { .. } => { - self.sql_op_precedence(&ast::BinaryOperator::PGLikeMatch) + self.sql_op_precedence(&BinaryOperator::PGLikeMatch) } _ => 0, } @@ -727,70 +728,70 @@ impl Unparser<'_> { fn sql_to_op(&self, op: &BinaryOperator) -> Result { match op { - ast::BinaryOperator::Eq => Ok(Operator::Eq), - ast::BinaryOperator::NotEq => Ok(Operator::NotEq), - ast::BinaryOperator::Lt => Ok(Operator::Lt), - ast::BinaryOperator::LtEq => Ok(Operator::LtEq), - ast::BinaryOperator::Gt => Ok(Operator::Gt), - ast::BinaryOperator::GtEq => Ok(Operator::GtEq), - ast::BinaryOperator::Plus => Ok(Operator::Plus), - ast::BinaryOperator::Minus => Ok(Operator::Minus), - ast::BinaryOperator::Multiply => Ok(Operator::Multiply), - ast::BinaryOperator::Divide => Ok(Operator::Divide), - ast::BinaryOperator::Modulo => Ok(Operator::Modulo), - ast::BinaryOperator::And => Ok(Operator::And), - ast::BinaryOperator::Or => Ok(Operator::Or), - ast::BinaryOperator::PGRegexMatch => Ok(Operator::RegexMatch), - ast::BinaryOperator::PGRegexIMatch => Ok(Operator::RegexIMatch), - ast::BinaryOperator::PGRegexNotMatch => Ok(Operator::RegexNotMatch), - ast::BinaryOperator::PGRegexNotIMatch => Ok(Operator::RegexNotIMatch), - ast::BinaryOperator::PGILikeMatch => Ok(Operator::ILikeMatch), - ast::BinaryOperator::PGNotLikeMatch => Ok(Operator::NotLikeMatch), - ast::BinaryOperator::PGLikeMatch => Ok(Operator::LikeMatch), - ast::BinaryOperator::PGNotILikeMatch => Ok(Operator::NotILikeMatch), - ast::BinaryOperator::BitwiseAnd => Ok(Operator::BitwiseAnd), - ast::BinaryOperator::BitwiseOr => Ok(Operator::BitwiseOr), - ast::BinaryOperator::BitwiseXor => Ok(Operator::BitwiseXor), - ast::BinaryOperator::PGBitwiseShiftRight => Ok(Operator::BitwiseShiftRight), - ast::BinaryOperator::PGBitwiseShiftLeft => Ok(Operator::BitwiseShiftLeft), - ast::BinaryOperator::StringConcat => Ok(Operator::StringConcat), - ast::BinaryOperator::AtArrow => Ok(Operator::AtArrow), - ast::BinaryOperator::ArrowAt => Ok(Operator::ArrowAt), + BinaryOperator::Eq => Ok(Operator::Eq), + BinaryOperator::NotEq => Ok(Operator::NotEq), + BinaryOperator::Lt => Ok(Operator::Lt), + BinaryOperator::LtEq => Ok(Operator::LtEq), + BinaryOperator::Gt => Ok(Operator::Gt), + BinaryOperator::GtEq => Ok(Operator::GtEq), + BinaryOperator::Plus => Ok(Operator::Plus), + BinaryOperator::Minus => Ok(Operator::Minus), + BinaryOperator::Multiply => Ok(Operator::Multiply), + BinaryOperator::Divide => Ok(Operator::Divide), + BinaryOperator::Modulo => Ok(Operator::Modulo), + BinaryOperator::And => Ok(Operator::And), + BinaryOperator::Or => Ok(Operator::Or), + BinaryOperator::PGRegexMatch => Ok(Operator::RegexMatch), + BinaryOperator::PGRegexIMatch => Ok(Operator::RegexIMatch), + BinaryOperator::PGRegexNotMatch => Ok(Operator::RegexNotMatch), + BinaryOperator::PGRegexNotIMatch => Ok(Operator::RegexNotIMatch), + BinaryOperator::PGILikeMatch => Ok(Operator::ILikeMatch), + BinaryOperator::PGNotLikeMatch => Ok(Operator::NotLikeMatch), + BinaryOperator::PGLikeMatch => Ok(Operator::LikeMatch), + BinaryOperator::PGNotILikeMatch => Ok(Operator::NotILikeMatch), + BinaryOperator::BitwiseAnd => Ok(Operator::BitwiseAnd), + BinaryOperator::BitwiseOr => Ok(Operator::BitwiseOr), + BinaryOperator::BitwiseXor => Ok(Operator::BitwiseXor), + BinaryOperator::PGBitwiseShiftRight => Ok(Operator::BitwiseShiftRight), + BinaryOperator::PGBitwiseShiftLeft => Ok(Operator::BitwiseShiftLeft), + BinaryOperator::StringConcat => Ok(Operator::StringConcat), + BinaryOperator::AtArrow => Ok(Operator::AtArrow), + BinaryOperator::ArrowAt => Ok(Operator::ArrowAt), _ => not_impl_err!("unsupported operation: {op:?}"), } } - fn op_to_sql(&self, op: &Operator) -> Result { + fn op_to_sql(&self, op: &Operator) -> Result { match op { - Operator::Eq => Ok(ast::BinaryOperator::Eq), - Operator::NotEq => Ok(ast::BinaryOperator::NotEq), - Operator::Lt => Ok(ast::BinaryOperator::Lt), - Operator::LtEq => Ok(ast::BinaryOperator::LtEq), - Operator::Gt => Ok(ast::BinaryOperator::Gt), - Operator::GtEq => Ok(ast::BinaryOperator::GtEq), - Operator::Plus => Ok(ast::BinaryOperator::Plus), - Operator::Minus => Ok(ast::BinaryOperator::Minus), - Operator::Multiply => Ok(ast::BinaryOperator::Multiply), - Operator::Divide => Ok(ast::BinaryOperator::Divide), - Operator::Modulo => Ok(ast::BinaryOperator::Modulo), - Operator::And => Ok(ast::BinaryOperator::And), - Operator::Or => Ok(ast::BinaryOperator::Or), + Operator::Eq => Ok(BinaryOperator::Eq), + Operator::NotEq => Ok(BinaryOperator::NotEq), + Operator::Lt => Ok(BinaryOperator::Lt), + Operator::LtEq => Ok(BinaryOperator::LtEq), + Operator::Gt => Ok(BinaryOperator::Gt), + Operator::GtEq => Ok(BinaryOperator::GtEq), + Operator::Plus => Ok(BinaryOperator::Plus), + Operator::Minus => Ok(BinaryOperator::Minus), + Operator::Multiply => Ok(BinaryOperator::Multiply), + Operator::Divide => Ok(BinaryOperator::Divide), + Operator::Modulo => Ok(BinaryOperator::Modulo), + Operator::And => Ok(BinaryOperator::And), + Operator::Or => Ok(BinaryOperator::Or), Operator::IsDistinctFrom => not_impl_err!("unsupported operation: {op:?}"), Operator::IsNotDistinctFrom => not_impl_err!("unsupported operation: {op:?}"), - Operator::RegexMatch => Ok(ast::BinaryOperator::PGRegexMatch), - Operator::RegexIMatch => Ok(ast::BinaryOperator::PGRegexIMatch), - Operator::RegexNotMatch => Ok(ast::BinaryOperator::PGRegexNotMatch), - Operator::RegexNotIMatch => Ok(ast::BinaryOperator::PGRegexNotIMatch), - Operator::ILikeMatch => Ok(ast::BinaryOperator::PGILikeMatch), - Operator::NotLikeMatch => Ok(ast::BinaryOperator::PGNotLikeMatch), - Operator::LikeMatch => Ok(ast::BinaryOperator::PGLikeMatch), - Operator::NotILikeMatch => Ok(ast::BinaryOperator::PGNotILikeMatch), - Operator::BitwiseAnd => Ok(ast::BinaryOperator::BitwiseAnd), - Operator::BitwiseOr => Ok(ast::BinaryOperator::BitwiseOr), - Operator::BitwiseXor => Ok(ast::BinaryOperator::BitwiseXor), - Operator::BitwiseShiftRight => Ok(ast::BinaryOperator::PGBitwiseShiftRight), - Operator::BitwiseShiftLeft => Ok(ast::BinaryOperator::PGBitwiseShiftLeft), - Operator::StringConcat => Ok(ast::BinaryOperator::StringConcat), + Operator::RegexMatch => Ok(BinaryOperator::PGRegexMatch), + Operator::RegexIMatch => Ok(BinaryOperator::PGRegexIMatch), + Operator::RegexNotMatch => Ok(BinaryOperator::PGRegexNotMatch), + Operator::RegexNotIMatch => Ok(BinaryOperator::PGRegexNotIMatch), + Operator::ILikeMatch => Ok(BinaryOperator::PGILikeMatch), + Operator::NotLikeMatch => Ok(BinaryOperator::PGNotLikeMatch), + Operator::LikeMatch => Ok(BinaryOperator::PGLikeMatch), + Operator::NotILikeMatch => Ok(BinaryOperator::PGNotILikeMatch), + Operator::BitwiseAnd => Ok(BinaryOperator::BitwiseAnd), + Operator::BitwiseOr => Ok(BinaryOperator::BitwiseOr), + Operator::BitwiseXor => Ok(BinaryOperator::BitwiseXor), + Operator::BitwiseShiftRight => Ok(BinaryOperator::PGBitwiseShiftRight), + Operator::BitwiseShiftLeft => Ok(BinaryOperator::PGBitwiseShiftLeft), + Operator::StringConcat => Ok(BinaryOperator::StringConcat), Operator::AtArrow => not_impl_err!("unsupported operation: {op:?}"), Operator::ArrowAt => not_impl_err!("unsupported operation: {op:?}"), } @@ -934,17 +935,17 @@ impl Unparser<'_> { Ok(ast::Expr::Value(ast::Value::Number(ui.to_string(), false))) } ScalarValue::UInt64(None) => Ok(ast::Expr::Value(ast::Value::Null)), - ScalarValue::Utf8(Some(str)) => Ok(ast::Expr::Value( - ast::Value::SingleQuotedString(str.to_string()), - )), + ScalarValue::Utf8(Some(str)) => { + Ok(ast::Expr::Value(SingleQuotedString(str.to_string()))) + } ScalarValue::Utf8(None) => Ok(ast::Expr::Value(ast::Value::Null)), - ScalarValue::Utf8View(Some(str)) => Ok(ast::Expr::Value( - ast::Value::SingleQuotedString(str.to_string()), - )), + ScalarValue::Utf8View(Some(str)) => { + Ok(ast::Expr::Value(SingleQuotedString(str.to_string()))) + } ScalarValue::Utf8View(None) => Ok(ast::Expr::Value(ast::Value::Null)), - ScalarValue::LargeUtf8(Some(str)) => Ok(ast::Expr::Value( - ast::Value::SingleQuotedString(str.to_string()), - )), + ScalarValue::LargeUtf8(Some(str)) => { + Ok(ast::Expr::Value(SingleQuotedString(str.to_string()))) + } ScalarValue::LargeUtf8(None) => Ok(ast::Expr::Value(ast::Value::Null)), ScalarValue::Binary(Some(_)) => not_impl_err!("Unsupported scalar: {v:?}"), ScalarValue::Binary(None) => Ok(ast::Expr::Value(ast::Value::Null)), @@ -977,7 +978,7 @@ impl Unparser<'_> { Ok(ast::Expr::Cast { kind: ast::CastKind::Cast, - expr: Box::new(ast::Expr::Value(ast::Value::SingleQuotedString( + expr: Box::new(ast::Expr::Value(SingleQuotedString( date.to_string(), ))), data_type: ast::DataType::Date, @@ -1000,7 +1001,7 @@ impl Unparser<'_> { Ok(ast::Expr::Cast { kind: ast::CastKind::Cast, - expr: Box::new(ast::Expr::Value(ast::Value::SingleQuotedString( + expr: Box::new(ast::Expr::Value(SingleQuotedString( datetime.to_string(), ))), data_type: self.ast_type_for_date64_in_cast(), @@ -1242,9 +1243,9 @@ impl Unparser<'_> { IntervalStyle::SQLStandard => match v { ScalarValue::IntervalYearMonth(Some(v)) => { let interval = Interval { - value: Box::new(ast::Expr::Value( - ast::Value::SingleQuotedString(v.to_string()), - )), + value: Box::new(ast::Expr::Value(SingleQuotedString( + v.to_string(), + ))), leading_field: Some(ast::DateTimeField::Month), leading_precision: None, last_field: None, @@ -1263,11 +1264,9 @@ impl Unparser<'_> { let millis = v.milliseconds % 1_000; let interval = Interval { - value: Box::new(ast::Expr::Value( - ast::Value::SingleQuotedString(format!( - "{days} {hours}:{mins}:{secs}.{millis:3}" - )), - )), + value: Box::new(ast::Expr::Value(SingleQuotedString(format!( + "{days} {hours}:{mins}:{secs}.{millis:3}" + )))), leading_field: Some(ast::DateTimeField::Day), leading_precision: None, last_field: Some(ast::DateTimeField::Second), @@ -1278,9 +1277,9 @@ impl Unparser<'_> { ScalarValue::IntervalMonthDayNano(Some(v)) => { if v.months >= 0 && v.days == 0 && v.nanoseconds == 0 { let interval = Interval { - value: Box::new(ast::Expr::Value( - ast::Value::SingleQuotedString(v.months.to_string()), - )), + value: Box::new(ast::Expr::Value(SingleQuotedString( + v.months.to_string(), + ))), leading_field: Some(ast::DateTimeField::Month), leading_precision: None, last_field: None, @@ -1299,11 +1298,9 @@ impl Unparser<'_> { let millis = (v.nanoseconds % 1_000_000_000) / 1_000_000; let interval = Interval { - value: Box::new(ast::Expr::Value( - ast::Value::SingleQuotedString(format!( - "{days} {hours}:{mins}:{secs}.{millis:03}" - )), - )), + value: Box::new(ast::Expr::Value(SingleQuotedString( + format!("{days} {hours}:{mins}:{secs}.{millis:03}"), + ))), leading_field: Some(ast::DateTimeField::Day), leading_precision: None, last_field: Some(ast::DateTimeField::Second), @@ -1340,6 +1337,29 @@ impl Unparser<'_> { } } + /// Converts an UNNEST operation to an AST expression by wrapping it as a function call, + /// since there is no direct representation for UNNEST in the AST. + fn unnest_to_sql(&self, unnest: &Unnest) -> Result { + let args = self.function_args_to_sql(std::slice::from_ref(&unnest.expr))?; + + Ok(ast::Expr::Function(Function { + name: ObjectName(vec![Ident { + value: "UNNEST".to_string(), + quote_style: None, + }]), + args: ast::FunctionArguments::List(ast::FunctionArgumentList { + duplicate_treatment: None, + args, + clauses: vec![], + }), + filter: None, + null_treatment: None, + over: None, + within_group: vec![], + parameters: ast::FunctionArguments::None, + })) + } + fn arrow_dtype_to_ast_dtype(&self, data_type: &DataType) -> Result { match data_type { DataType::Null => { @@ -1855,6 +1875,15 @@ mod tests { }), r#"CAST(a AS DECIMAL(12,0))"#, ), + ( + Expr::Unnest(Unnest { + expr: Box::new(Expr::Column(Column { + relation: Some(TableReference::partial("schema", "table")), + name: "array_col".to_string(), + })), + }), + r#"UNNEST("schema"."table".array_col)"#, + ), ]; for (expr, expected) in tests { @@ -1929,11 +1958,8 @@ mod tests { #[test] fn custom_dialect_float64_ast_dtype() -> Result<()> { for (float64_ast_dtype, identifier) in [ - (sqlparser::ast::DataType::Double, "DOUBLE"), - ( - sqlparser::ast::DataType::DoublePrecision, - "DOUBLE PRECISION", - ), + (ast::DataType::Double, "DOUBLE"), + (ast::DataType::DoublePrecision, "DOUBLE PRECISION"), ] { let dialect = CustomDialectBuilder::new() .with_float64_ast_dtype(float64_ast_dtype) @@ -2350,10 +2376,7 @@ mod tests { expr: Box::new(Expr::Literal(ScalarValue::Utf8(Some( "variation".to_string(), )))), - data_type: DataType::Dictionary( - Box::new(DataType::Int8), - Box::new(DataType::Utf8), - ), + data_type: DataType::Dictionary(Box::new(Int8), Box::new(DataType::Utf8)), }), "'variation'", )]; diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 695027374fa09..2c38a1d36c1ea 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -26,8 +26,9 @@ use super::{ subquery_alias_inner_query_and_columns, TableAliasRewriter, }, utils::{ - find_agg_node_within_select, find_window_nodes_within_select, - unproject_sort_expr, unproject_window_exprs, + find_agg_node_within_select, find_unnest_node_within_select, + find_window_nodes_within_select, try_transform_to_simple_table_scan_with_filters, + unproject_sort_expr, unproject_unnest_expr, unproject_window_exprs, }, Unparser, }; @@ -38,8 +39,8 @@ use datafusion_common::{ Column, DataFusionError, Result, TableReference, }; use datafusion_expr::{ - expr::Alias, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, - LogicalPlanBuilder, Projection, SortExpr, TableScan, + expr::Alias, BinaryExpr, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, + LogicalPlanBuilder, Operator, Projection, SortExpr, TableScan, }; use sqlparser::ast::{self, Ident, SetExpr}; use std::sync::Arc; @@ -173,15 +174,24 @@ impl Unparser<'_> { p: &Projection, select: &mut SelectBuilder, ) -> Result<()> { + let mut exprs = p.expr.clone(); + + // If an Unnest node is found within the select, find and unproject the unnest column + if let Some(unnest) = find_unnest_node_within_select(plan) { + exprs = exprs + .into_iter() + .map(|e| unproject_unnest_expr(e, unnest)) + .collect::>>()?; + }; + match ( find_agg_node_within_select(plan, true), find_window_nodes_within_select(plan, None, true), ) { (Some(agg), window) => { let window_option = window.as_deref(); - let items = p - .expr - .iter() + let items = exprs + .into_iter() .map(|proj_expr| { let unproj = unproject_agg_exprs(proj_expr, agg, window_option)?; self.select_item_to_sql(&unproj) @@ -198,9 +208,8 @@ impl Unparser<'_> { )); } (None, Some(window)) => { - let items = p - .expr - .iter() + let items = exprs + .into_iter() .map(|proj_expr| { let unproj = unproject_window_exprs(proj_expr, &window)?; self.select_item_to_sql(&unproj) @@ -210,8 +219,7 @@ impl Unparser<'_> { select.projection(items); } _ => { - let items = p - .expr + let items = exprs .iter() .map(|e| self.select_item_to_sql(e)) .collect::>>()?; @@ -318,7 +326,8 @@ impl Unparser<'_> { if let Some(agg) = find_agg_node_within_select(plan, select.already_projected()) { - let unprojected = unproject_agg_exprs(&filter.predicate, agg, None)?; + let unprojected = + unproject_agg_exprs(filter.predicate.clone(), agg, None)?; let filter_expr = self.expr_to_sql(&unprojected)?; select.having(Some(filter_expr)); } else { @@ -459,22 +468,77 @@ impl Unparser<'_> { self.select_to_sql_recursively(input, query, select, relation) } LogicalPlan::Join(join) => { - let join_constraint = self.join_constraint_to_sql( - join.join_constraint, - &join.on, - join.filter.as_ref(), + let mut table_scan_filters = vec![]; + + let left_plan = + match try_transform_to_simple_table_scan_with_filters(&join.left)? { + Some((plan, filters)) => { + table_scan_filters.extend(filters); + Arc::new(plan) + } + None => Arc::clone(&join.left), + }; + + self.select_to_sql_recursively( + left_plan.as_ref(), + query, + select, + relation, )?; + let right_plan = + match try_transform_to_simple_table_scan_with_filters(&join.right)? { + Some((plan, filters)) => { + table_scan_filters.extend(filters); + Arc::new(plan) + } + None => Arc::clone(&join.right), + }; + let mut right_relation = RelationBuilder::default(); self.select_to_sql_recursively( - join.left.as_ref(), + right_plan.as_ref(), query, select, - relation, + &mut right_relation, + )?; + + let join_filters = if table_scan_filters.is_empty() { + join.filter.clone() + } else { + // Combine `table_scan_filters` into a single filter using `AND` + let Some(combined_filters) = + table_scan_filters.into_iter().reduce(|acc, filter| { + Expr::BinaryExpr(BinaryExpr { + left: Box::new(acc), + op: Operator::And, + right: Box::new(filter), + }) + }) + else { + return internal_err!("Failed to combine TableScan filters"); + }; + + // Combine `join.filter` with `combined_filters` using `AND` + match &join.filter { + Some(filter) => Some(Expr::BinaryExpr(BinaryExpr { + left: Box::new(filter.clone()), + op: Operator::And, + right: Box::new(combined_filters), + })), + None => Some(combined_filters), + } + }; + + let join_constraint = self.join_constraint_to_sql( + join.join_constraint, + &join.on, + join_filters.as_ref(), )?; + self.select_to_sql_recursively( - join.right.as_ref(), + right_plan.as_ref(), query, select, &mut right_relation, @@ -596,6 +660,28 @@ impl Unparser<'_> { Ok(()) } LogicalPlan::Extension(_) => not_impl_err!("Unsupported operator: {plan:?}"), + LogicalPlan::Unnest(unnest) => { + if !unnest.struct_type_columns.is_empty() { + return internal_err!( + "Struct type columns are not currently supported in UNNEST: {:?}", + unnest.struct_type_columns + ); + } + + // In the case of UNNEST, the Unnest node is followed by a duplicate Projection node that we should skip. + // Otherwise, there will be a duplicate SELECT clause. + // | Projection: table.col1, UNNEST(table.col2) + // | Unnest: UNNEST(table.col2) + // | Projection: table.col1, table.col2 AS UNNEST(table.col2) + // | Filter: table.col3 = Int64(3) + // | TableScan: table projection=None + if let LogicalPlan::Projection(p) = unnest.input.as_ref() { + // continue with projection input + self.select_to_sql_recursively(&p.input, query, select, relation) + } else { + internal_err!("Unnest input is not a Projection: {unnest:?}") + } + } _ => not_impl_err!("Unsupported operator: {plan:?}"), } } diff --git a/datafusion/sql/src/unparser/utils.rs b/datafusion/sql/src/unparser/utils.rs index 5e3a3aa600b6c..284956cef195e 100644 --- a/datafusion/sql/src/unparser/utils.rs +++ b/datafusion/sql/src/unparser/utils.rs @@ -15,20 +15,20 @@ // specific language governing permissions and limitations // under the License. -use std::cmp::Ordering; +use std::{cmp::Ordering, sync::Arc, vec}; use datafusion_common::{ internal_err, - tree_node::{Transformed, TreeNode}, - Column, Result, ScalarValue, + tree_node::{Transformed, TransformedResult, TreeNode}, + Column, DataFusionError, Result, ScalarValue, }; use datafusion_expr::{ - utils::grouping_set_to_exprlist, Aggregate, Expr, LogicalPlan, Projection, SortExpr, - Window, + expr, utils::grouping_set_to_exprlist, Aggregate, Expr, LogicalPlan, + LogicalPlanBuilder, Projection, SortExpr, Unnest, Window, }; use sqlparser::ast; -use super::{dialect::DateFieldExtractStyle, Unparser}; +use super::{dialect::DateFieldExtractStyle, rewrite::TableAliasRewriter, Unparser}; /// Recursively searches children of [LogicalPlan] to find an Aggregate node if exists /// prior to encountering a Join, TableScan, or a nested subquery (derived table factor). @@ -62,6 +62,28 @@ pub(crate) fn find_agg_node_within_select( } } +/// Recursively searches children of [LogicalPlan] to find Unnest node if exist +pub(crate) fn find_unnest_node_within_select(plan: &LogicalPlan) -> Option<&Unnest> { + // Note that none of the nodes that have a corresponding node can have more + // than 1 input node. E.g. Projection / Filter always have 1 input node. + let input = plan.inputs(); + let input = if input.len() > 1 { + return None; + } else { + input.first()? + }; + + if let LogicalPlan::Unnest(unnest) = input { + Some(unnest) + } else if let LogicalPlan::TableScan(_) = input { + None + } else if let LogicalPlan::Projection(_) = input { + None + } else { + find_unnest_node_within_select(input) + } +} + /// Recursively searches children of [LogicalPlan] to find Window nodes if exist /// prior to encountering a Join, TableScan, or a nested subquery (derived table factor). /// If Window node is not found prior to this or at all before reaching the end @@ -104,18 +126,46 @@ pub(crate) fn find_window_nodes_within_select<'a>( } } +/// Recursively identify Column expressions and transform them into the appropriate unnest expression +/// +/// For example, if expr contains the column expr "unnest_placeholder(make_array(Int64(1),Int64(2),Int64(2),Int64(5),NULL),depth=1)" +/// it will be transformed into an actual unnest expression UNNEST([1, 2, 2, 5, NULL]) +pub(crate) fn unproject_unnest_expr(expr: Expr, unnest: &Unnest) -> Result { + expr.transform(|sub_expr| { + if let Expr::Column(col_ref) = &sub_expr { + // Check if the column is among the columns to run unnest on. + // Currently, only List/Array columns (defined in `list_type_columns`) are supported for unnesting. + if unnest.list_type_columns.iter().any(|e| e.1.output_column.name == col_ref.name) { + if let Ok(idx) = unnest.schema.index_of_column(col_ref) { + if let LogicalPlan::Projection(Projection { expr, .. }) = unnest.input.as_ref() { + if let Some(unprojected_expr) = expr.get(idx) { + let unnest_expr = Expr::Unnest(expr::Unnest::new(unprojected_expr.clone())); + return Ok(Transformed::yes(unnest_expr)); + } + } + } + return internal_err!( + "Tried to unproject unnest expr for column '{}' that was not found in the provided Unnest!", &col_ref.name + ); + } + } + + Ok(Transformed::no(sub_expr)) + + }).map(|e| e.data) +} + /// Recursively identify all Column expressions and transform them into the appropriate /// aggregate expression contained in agg. /// /// For example, if expr contains the column expr "COUNT(*)" it will be transformed /// into an actual aggregate expression COUNT(*) as identified in the aggregate node. pub(crate) fn unproject_agg_exprs( - expr: &Expr, + expr: Expr, agg: &Aggregate, windows: Option<&[&Window]>, ) -> Result { - expr.clone() - .transform(|sub_expr| { + expr.transform(|sub_expr| { if let Expr::Column(c) = sub_expr { if let Some(unprojected_expr) = find_agg_expr(agg, &c)? { Ok(Transformed::yes(unprojected_expr.clone())) @@ -123,7 +173,7 @@ pub(crate) fn unproject_agg_exprs( windows.and_then(|w| find_window_expr(w, &c.name).cloned()) { // Window function can contain an aggregation columns, e.g., 'avg(sum(ss_sales_price)) over ...' that needs to be unprojected - return Ok(Transformed::yes(unproject_agg_exprs(&unprojected_expr, agg, None)?)); + return Ok(Transformed::yes(unproject_agg_exprs(unprojected_expr, agg, None)?)); } else { internal_err!( "Tried to unproject agg expr for column '{}' that was not found in the provided Aggregate!", &c.name @@ -141,20 +191,19 @@ pub(crate) fn unproject_agg_exprs( /// /// For example, if expr contains the column expr "COUNT(*) PARTITION BY id" it will be transformed /// into an actual window expression as identified in the window node. -pub(crate) fn unproject_window_exprs(expr: &Expr, windows: &[&Window]) -> Result { - expr.clone() - .transform(|sub_expr| { - if let Expr::Column(c) = sub_expr { - if let Some(unproj) = find_window_expr(windows, &c.name) { - Ok(Transformed::yes(unproj.clone())) - } else { - Ok(Transformed::no(Expr::Column(c))) - } +pub(crate) fn unproject_window_exprs(expr: Expr, windows: &[&Window]) -> Result { + expr.transform(|sub_expr| { + if let Expr::Column(c) = sub_expr { + if let Some(unproj) = find_window_expr(windows, &c.name) { + Ok(Transformed::yes(unproj.clone())) } else { - Ok(Transformed::no(sub_expr)) + Ok(Transformed::no(Expr::Column(c))) } - }) - .map(|e| e.data) + } else { + Ok(Transformed::no(sub_expr)) + } + }) + .map(|e| e.data) } fn find_agg_expr<'a>(agg: &'a Aggregate, column: &Column) -> Result> { @@ -218,7 +267,7 @@ pub(crate) fn unproject_sort_expr( // In case of aggregation there could be columns containing aggregation functions we need to unproject if let Some(agg) = agg { if agg.schema.is_column_from_schema(col_ref) { - let new_expr = unproject_agg_exprs(&sort_expr.expr, agg, None)?; + let new_expr = unproject_agg_exprs(sort_expr.expr, agg, None)?; sort_expr.expr = new_expr; return Ok(sort_expr); } @@ -239,6 +288,87 @@ pub(crate) fn unproject_sort_expr( Ok(sort_expr) } +/// Iterates through the children of a [LogicalPlan] to find a TableScan node before encountering +/// a Projection or any unexpected node that indicates the presence of a Projection (SELECT) in the plan. +/// If a TableScan node is found, returns the TableScan node without filters, along with the collected filters separately. +/// If the plan contains a Projection, returns None. +/// +/// Note: If a table alias is present, TableScan filters are rewritten to reference the alias. +/// +/// LogicalPlan example: +/// Filter: ta.j1_id < 5 +/// Alias: ta +/// TableScan: j1, j1_id > 10 +/// +/// Will return LogicalPlan below: +/// Alias: ta +/// TableScan: j1 +/// And filters: [ta.j1_id < 5, ta.j1_id > 10] +pub(crate) fn try_transform_to_simple_table_scan_with_filters( + plan: &LogicalPlan, +) -> Result)>> { + let mut filters: Vec = vec![]; + let mut plan_stack = vec![plan]; + let mut table_alias = None; + + while let Some(current_plan) = plan_stack.pop() { + match current_plan { + LogicalPlan::SubqueryAlias(alias) => { + table_alias = Some(alias.alias.clone()); + plan_stack.push(alias.input.as_ref()); + } + LogicalPlan::Filter(filter) => { + filters.push(filter.predicate.clone()); + plan_stack.push(filter.input.as_ref()); + } + LogicalPlan::TableScan(table_scan) => { + let table_schema = table_scan.source.schema(); + // optional rewriter if table has an alias + let mut filter_alias_rewriter = + table_alias.as_ref().map(|alias_name| TableAliasRewriter { + table_schema: &table_schema, + alias_name: alias_name.clone(), + }); + + // rewrite filters to use table alias if present + let table_scan_filters = table_scan + .filters + .iter() + .cloned() + .map(|expr| { + if let Some(ref mut rewriter) = filter_alias_rewriter { + expr.rewrite(rewriter).data() + } else { + Ok(expr) + } + }) + .collect::, DataFusionError>>()?; + + filters.extend(table_scan_filters); + + let mut builder = LogicalPlanBuilder::scan( + table_scan.table_name.clone(), + Arc::clone(&table_scan.source), + None, + )?; + + if let Some(alias) = table_alias.take() { + builder = builder.alias(alias)?; + } + + let plan = builder.build()?; + + return Ok(Some((plan, filters))); + } + _ => { + return Ok(None); + } + } + } + + Ok(None) +} + /// Converts a date_part function to SQL, tailoring it to the supported date field extraction style. pub(crate) fn date_part_to_sql( unparser: &Unparser, diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index 8e25c1c5b1cd7..ea0ccb8e4b43e 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -24,6 +24,7 @@ use datafusion_expr::test::function_stub::{count_udaf, max_udaf, min_udaf, sum_u use datafusion_expr::{col, lit, table_scan, wildcard, LogicalPlanBuilder}; use datafusion_functions::unicode; use datafusion_functions_aggregate::grouping::grouping_udaf; +use datafusion_functions_nested::make_array::make_array_udf; use datafusion_functions_window::rank::rank_udwf; use datafusion_sql::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_sql::unparser::dialect::{ @@ -711,7 +712,8 @@ where .with_aggregate_function(max_udaf()) .with_aggregate_function(grouping_udaf()) .with_window_function(rank_udwf()) - .with_scalar_function(Arc::new(unicode::substr().as_ref().clone())), + .with_scalar_function(Arc::new(unicode::substr().as_ref().clone())) + .with_scalar_function(make_array_udf()), }; let sql_to_rel = SqlToRel::new(&context); let plan = sql_to_rel.sql_statement_to_plan(statement).unwrap(); @@ -968,6 +970,21 @@ fn test_table_scan_pushdown() -> Result<()> { table_scan_with_all.to_string(), "SELECT t1.id, t1.age FROM t1 WHERE (t1.id > t1.age) LIMIT 10" ); + + let table_scan_with_additional_filter = table_scan_with_filters( + Some("t1"), + &schema, + None, + vec![col("id").gt(col("age"))], + )? + .filter(col("id").eq(lit(5)))? + .build()?; + let table_scan_with_filter = plan_to_sql(&table_scan_with_additional_filter)?; + assert_eq!( + table_scan_with_filter.to_string(), + "SELECT * FROM t1 WHERE (t1.id = 5) AND (t1.id > t1.age)" + ); + Ok(()) } @@ -991,6 +1008,93 @@ fn test_sort_with_push_down_fetch() -> Result<()> { Ok(()) } +#[test] +fn test_join_with_table_scan_filters() -> Result<()> { + let schema_left = Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, false), + ]); + + let schema_right = Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("age", DataType::Utf8, false), + ]); + + let left_plan = table_scan_with_filters( + Some("left_table"), + &schema_left, + None, + vec![col("name").like(lit("some_name"))], + )? + .alias("left")? + .build()?; + + let right_plan = table_scan_with_filters( + Some("right_table"), + &schema_right, + None, + vec![col("age").gt(lit(10))], + )? + .build()?; + + let join_plan_with_filter = LogicalPlanBuilder::from(left_plan.clone()) + .join( + right_plan.clone(), + datafusion_expr::JoinType::Inner, + (vec!["left.id"], vec!["right_table.id"]), + Some(col("left.id").gt(lit(5))), + )? + .build()?; + + let sql = plan_to_sql(&join_plan_with_filter)?; + + let expected_sql = r#"SELECT * FROM left_table AS "left" JOIN right_table ON "left".id = right_table.id AND (("left".id > 5) AND ("left"."name" LIKE 'some_name' AND (age > 10)))"#; + + assert_eq!(sql.to_string(), expected_sql); + + let join_plan_no_filter = LogicalPlanBuilder::from(left_plan.clone()) + .join( + right_plan, + datafusion_expr::JoinType::Inner, + (vec!["left.id"], vec!["right_table.id"]), + None, + )? + .build()?; + + let sql = plan_to_sql(&join_plan_no_filter)?; + + let expected_sql = r#"SELECT * FROM left_table AS "left" JOIN right_table ON "left".id = right_table.id AND ("left"."name" LIKE 'some_name' AND (age > 10))"#; + + assert_eq!(sql.to_string(), expected_sql); + + let right_plan_with_filter = table_scan_with_filters( + Some("right_table"), + &schema_right, + None, + vec![col("age").gt(lit(10))], + )? + .filter(col("right_table.name").eq(lit("before_join_filter_val")))? + .build()?; + + let join_plan_multiple_filters = LogicalPlanBuilder::from(left_plan.clone()) + .join( + right_plan_with_filter, + datafusion_expr::JoinType::Inner, + (vec!["left.id"], vec!["right_table.id"]), + Some(col("left.id").gt(lit(5))), + )? + .filter(col("left.name").eq(lit("after_join_filter_val")))? + .build()?; + + let sql = plan_to_sql(&join_plan_multiple_filters)?; + + let expected_sql = r#"SELECT * FROM left_table AS "left" JOIN right_table ON "left".id = right_table.id AND (("left".id > 5) AND (("left"."name" LIKE 'some_name' AND (right_table."name" = 'before_join_filter_val')) AND (age > 10))) WHERE ("left"."name" = 'after_join_filter_val')"#; + + assert_eq!(sql.to_string(), expected_sql); + + Ok(()) +} + #[test] fn test_interval_lhs_eq() { sql_round_trip( @@ -1069,3 +1173,18 @@ FROM person GROUP BY person.id, person.first_name"#.replace("\n", " ").as_str(), ); } + +#[test] +fn test_unnest_to_sql() { + sql_round_trip( + GenericDialect {}, + r#"SELECT unnest(array_col) as u1, struct_col, array_col FROM unnest_table WHERE array_col != NULL ORDER BY struct_col, array_col"#, + r#"SELECT UNNEST(unnest_table.array_col) AS u1, unnest_table.struct_col, unnest_table.array_col FROM unnest_table WHERE (unnest_table.array_col <> NULL) ORDER BY unnest_table.struct_col ASC NULLS LAST, unnest_table.array_col ASC NULLS LAST"#, + ); + + sql_round_trip( + GenericDialect {}, + r#"SELECT unnest(make_array(1, 2, 2, 5, NULL)) as u1"#, + r#"SELECT UNNEST(make_array(1, 2, 2, 5, NULL)) AS u1"#, + ); +} diff --git a/datafusion/sql/tests/common/mod.rs b/datafusion/sql/tests/common/mod.rs index 47caeec78dc76..b0fa170318493 100644 --- a/datafusion/sql/tests/common/mod.rs +++ b/datafusion/sql/tests/common/mod.rs @@ -232,10 +232,7 @@ impl ContextProvider for MockContextProvider { &self.state.config_options } - fn get_file_type( - &self, - _ext: &str, - ) -> Result> { + fn get_file_type(&self, _ext: &str) -> Result> { Ok(Arc::new(MockCsvType {})) } @@ -275,7 +272,7 @@ impl EmptyTable { } impl TableSource for EmptyTable { - fn as_any(&self) -> &dyn std::any::Any { + fn as_any(&self) -> &dyn Any { self } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index edb614493b387..698c408e538f5 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -4209,6 +4209,29 @@ fn test_prepare_statement_to_plan_having() { prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); } +#[test] +fn test_prepare_statement_to_plan_limit() { + let sql = "PREPARE my_plan(BIGINT, BIGINT) AS + SELECT id FROM person \ + OFFSET $1 LIMIT $2"; + + let expected_plan = "Prepare: \"my_plan\" [Int64, Int64] \ + \n Limit: skip=$1, fetch=$2\ + \n Projection: person.id\ + \n TableScan: person"; + + let expected_dt = "[Int64, Int64]"; + + let plan = prepare_stmt_quick_test(sql, expected_plan, expected_dt); + + // replace params with values + let param_values = vec![ScalarValue::Int64(Some(10)), ScalarValue::Int64(Some(200))]; + let expected_plan = "Limit: skip=10, fetch=200\ + \n Projection: person.id\ + \n TableScan: person"; + prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); +} + #[test] fn test_prepare_statement_to_plan_value_list() { let sql = "PREPARE my_plan(STRING, STRING) AS SELECT * FROM (VALUES(1, $1), (2, $2)) AS t (num, letter);"; diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index baa49057e1b97..2479252a7b5b0 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -61,7 +61,16 @@ async fn run_tests() -> Result<()> { // Enable logging (e.g. set RUST_LOG=debug to see debug logs) env_logger::init(); - let options: Options = clap::Parser::parse(); + let options: Options = Parser::parse(); + if options.list { + // nextest parses stdout, so print messages to stderr + eprintln!("NOTICE: --list option unsupported, quitting"); + // return Ok, not error so that tools like nextest which are listing all + // workspace tests (by running `cargo test ... --list --format terse`) + // do not fail when they encounter this binary. Instead, print nothing + // to stdout and return OK so they can continue listing other tests. + return Ok(()); + } options.warn_on_ignored(); // Run all tests in parallel, reporting failures at the end @@ -255,7 +264,7 @@ fn read_dir_recursive>(path: P) -> Result> { /// Append all paths recursively to dst fn read_dir_recursive_impl(dst: &mut Vec, path: &Path) -> Result<()> { - let entries = std::fs::read_dir(path) + let entries = fs::read_dir(path) .map_err(|e| exec_datafusion_err!("Error reading directory {path:?}: {e}"))?; for entry in entries { let path = entry @@ -276,7 +285,7 @@ fn read_dir_recursive_impl(dst: &mut Vec, path: &Path) -> Result<()> { /// Parsed command line options /// -/// This structure attempts to mimic the command line options +/// This structure attempts to mimic the command line options of the built in rust test runner /// accepted by IDEs such as CLion that pass arguments /// /// See for more details @@ -320,6 +329,18 @@ struct Options { help = "IGNORED (for compatibility with built in rust test runner)" )] show_output: bool, + + #[clap( + long, + help = "Quits immediately, not listing anything (for compatibility with built-in rust test runner)" + )] + list: bool, + + #[clap( + long, + help = "IGNORED (for compatibility with built-in rust test runner)" + )] + ignored: bool, } impl Options { @@ -354,15 +375,15 @@ impl Options { /// Logs warning messages to stdout if any ignored options are passed fn warn_on_ignored(&self) { if self.format.is_some() { - println!("WARNING: Ignoring `--format` compatibility option"); + eprintln!("WARNING: Ignoring `--format` compatibility option"); } if self.z_options.is_some() { - println!("WARNING: Ignoring `-Z` compatibility option"); + eprintln!("WARNING: Ignoring `-Z` compatibility option"); } if self.show_output { - println!("WARNING: Ignoring `--show-output` compatibility option"); + eprintln!("WARNING: Ignoring `--show-output` compatibility option"); } } } diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index deeacb1b88190..477f225443e28 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -139,7 +139,7 @@ impl TestContext { } #[cfg(feature = "avro")] -pub async fn register_avro_tables(ctx: &mut crate::TestContext) { +pub async fn register_avro_tables(ctx: &mut TestContext) { use datafusion::prelude::AvroReadOptions; ctx.enable_testdir(); diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 3eb2c3a5bd983..31383b9a979a8 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4255,3 +4255,16 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(c1@0, c1@0)], filter=c2@0 >= c2@1 03)----MemoryExec: partitions=1, partition_sizes=[1] 04)----MemoryExec: partitions=1, partition_sizes=[1] + +# Test Utf8View as Join Key +# Issue: https://github.com/apache/datafusion/issues/12468 +statement ok +CREATE TABLE table1(v1 STRING) AS VALUES ('foo'), (NULL); + +statement ok +CREATE TABLE table1_stringview AS SELECT arrow_cast(v1, 'Utf8View') AS v1 FROM table1; + +query T +select * from table1 as t1 natural join table1_stringview as t2; +---- +foo diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 051cc6dce3d47..f4cc888d6b8e7 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -407,214 +407,201 @@ select t1.* from t1 where exists (select 1 from t2 where t2.a = t1.a and t2.b != statement ok set datafusion.execution.batch_size = 10; -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c union all -# select 11 a, 14 b, 4 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c where false -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 11 c union all -# select 11 a, 14 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 11 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c union all + select 11 a, 14 b, 4 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c where false + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 14 c union all -# select 11 a, 11 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 11 c union all + select 11 a, 14 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 11 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- + + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 14 c union all + select 11 a, 11 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- # Test LEFT ANTI with cross batch data distribution statement ok set datafusion.execution.batch_size = 1; -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c union all -# select 11 a, 14 b, 4 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query III -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b, 1 c union all -# select 11 a, 13 b, 2 c), -#t2 as ( -# select 11 a, 12 b, 3 c where false -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) -#) order by 1, 2; -#---- -#11 12 1 -#11 13 2 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 13 c union all -# select 11 a, 14 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- -#11 12 - -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 11 c union all -# select 11 a, 15 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 -# Uncomment when filtered LEFTANTI moved -#query II -#select * from ( -#with -#t1 as ( -# select 11 a, 12 b), -#t2 as ( -# select 11 a, 12 c union all -# select 11 a, 14 c union all -# select 11 a, 11 c -# ) -#select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) -#) order by 1, 2 -#---- +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c union all + select 11 a, 14 b, 4 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 + +query III +select * from ( +with +t1 as ( + select 11 a, 12 b, 1 c union all + select 11 a, 13 b, 2 c), +t2 as ( + select 11 a, 12 b, 3 c where false + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t2.b != t1.b and t1.c > t2.c) +) order by 1, 2; +---- +11 12 1 +11 13 2 + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 13 c union all + select 11 a, 14 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- +11 12 + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 11 c union all + select 11 a, 15 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- + +query II +select * from ( +with +t1 as ( + select 11 a, 12 b), +t2 as ( + select 11 a, 12 c union all + select 11 a, 14 c union all + select 11 a, 11 c + ) +select t1.* from t1 where not exists (select 1 from t2 where t2.a = t1.a and t1.b > t2.c) +) order by 1, 2 +---- query IIII select * from ( diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 26b5d8b952f6c..36de19f1c3aa7 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -438,7 +438,7 @@ SELECT t1_id, t1_name, t1_int, (select t2_id, t2_name FROM t2 WHERE t2.t2_id = t #subquery_not_allowed #In/Exist Subquery is not allowed in ORDER BY clause. -statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: In/Exist subquery can only be used in Projection, Filter, Window functions, Aggregate and Join plan nodes +statement error DataFusion error: check_analyzed_plan\ncaused by\nError during planning: In/Exist subquery can only be used in Projection, Filter, Window functions, Aggregate and Join plan nodes, but was used in \[Sort: t1.t1_int IN \(\) ASC NULLS LAST\] SELECT t1_id, t1_name, t1_int FROM t1 order by t1_int in (SELECT t2_int FROM t2 WHERE t1.t1_id > t1.t1_int) #non_aggregated_correlated_scalar_subquery diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index d866ec8c94dd3..38c2a66472731 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2780,6 +2780,11 @@ FROM NULL 01:01:2025 23-59-58 +query T +select to_char('2020-01-01 00:10:20.123'::timestamp at time zone 'America/New_York', '%Y-%m-%d %H:%M:%S.%3f'); +---- +2020-01-01 00:10:20.123 + statement ok drop table formats; diff --git a/datafusion/sqllogictest/test_files/type_coercion.slt b/datafusion/sqllogictest/test_files/type_coercion.slt index 0f9399cede2ec..43e7c2f7bc250 100644 --- a/datafusion/sqllogictest/test_files/type_coercion.slt +++ b/datafusion/sqllogictest/test_files/type_coercion.slt @@ -103,11 +103,11 @@ CREATE TABLE orders( ); # union_different_num_columns_error() / UNION -query error Error during planning: Union schemas have different number of fields: query 1 has 1 fields whereas query 2 has 2 fields +query error DataFusion error: Error during planning: UNION queries have different number of columns: left has 1 columns whereas right has 2 columns SELECT order_id FROM orders UNION SELECT customer_id, o_item_id FROM orders # union_different_num_columns_error() / UNION ALL -query error Error during planning: Union schemas have different number of fields: query 1 has 1 fields whereas query 2 has 2 fields +query error DataFusion error: Error during planning: UNION queries have different number of columns: left has 1 columns whereas right has 2 columns SELECT order_id FROM orders UNION ALL SELECT customer_id, o_item_id FROM orders # union_with_different_column_names() diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 3d5d7cce5673f..2aaf8ec0aa06b 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -42,17 +42,18 @@ use crate::variation_const::{ DATE_32_TYPE_VARIATION_REF, DATE_64_TYPE_VARIATION_REF, DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME, LARGE_CONTAINER_TYPE_VARIATION_REF, - UNSIGNED_INTEGER_TYPE_VARIATION_REF, VIEW_CONTAINER_TYPE_VARIATION_REF, + LARGE_CONTAINER_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, }; #[allow(deprecated)] use crate::variation_const::{ - INTERVAL_DAY_TIME_TYPE_REF, INTERVAL_MONTH_DAY_NANO_TYPE_REF, - INTERVAL_YEAR_MONTH_TYPE_REF, TIMESTAMP_MICRO_TYPE_VARIATION_REF, - TIMESTAMP_MILLI_TYPE_VARIATION_REF, TIMESTAMP_NANO_TYPE_VARIATION_REF, - TIMESTAMP_SECOND_TYPE_VARIATION_REF, + INTERVAL_DAY_TIME_TYPE_REF, INTERVAL_MONTH_DAY_NANO_TYPE_NAME, + INTERVAL_MONTH_DAY_NANO_TYPE_REF, INTERVAL_YEAR_MONTH_TYPE_REF, + TIMESTAMP_MICRO_TYPE_VARIATION_REF, TIMESTAMP_MILLI_TYPE_VARIATION_REF, + TIMESTAMP_NANO_TYPE_VARIATION_REF, TIMESTAMP_SECOND_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{new_empty_array, AsArray}; +use datafusion::arrow::temporal_conversions::NANOSECONDS; use datafusion::common::scalar::ScalarStructBuilder; use datafusion::dataframe::DataFrame; use datafusion::logical_expr::expr::InList; @@ -71,13 +72,13 @@ use datafusion::{ use std::collections::HashSet; use std::sync::Arc; use substrait::proto::exchange_rel::ExchangeKind; -use substrait::proto::expression::literal::interval_day_to_second::PrecisionMode; use substrait::proto::expression::literal::user_defined::Val; use substrait::proto::expression::literal::{ - IntervalDayToSecond, IntervalYearToMonth, UserDefined, + interval_day_to_second, IntervalCompound, IntervalDayToSecond, IntervalYearToMonth, + UserDefined, }; use substrait::proto::expression::subquery::SubqueryType; -use substrait::proto::expression::{self, FieldReference, Literal, ScalarFunction}; +use substrait::proto::expression::{FieldReference, Literal, ScalarFunction}; use substrait::proto::read_rel::local_files::file_or_files::PathType::UriFile; use substrait::proto::{ aggregate_function::AggregationInvocation, @@ -118,6 +119,7 @@ pub fn name_to_op(name: &str) -> Option { "multiply" => Some(Operator::Multiply), "divide" => Some(Operator::Divide), "mod" => Some(Operator::Modulo), + "modulus" => Some(Operator::Modulo), "and" => Some(Operator::And), "or" => Some(Operator::Or), "is_distinct_from" => Some(Operator::IsDistinctFrom), @@ -388,7 +390,7 @@ pub async fn from_substrait_extended_expr( pub fn apply_masking( schema: DFSchema, - mask_expression: &::core::option::Option, + mask_expression: &::core::option::Option, ) -> Result { match mask_expression { Some(MaskExpression { select, .. }) => match &select.as_ref() { @@ -713,14 +715,27 @@ pub async fn from_substrait_rel( } _ => false, }; + let order_by = if !f.sorts.is_empty() { + Some( + from_substrait_sorts( + ctx, + &f.sorts, + input.schema(), + extensions, + ) + .await?, + ) + } else { + None + }; + from_substrait_agg_func( ctx, f, input.schema(), extensions, filter, - // TODO: Add parsing of order_by also - None, + order_by, distinct, ) .await @@ -1831,9 +1846,14 @@ fn from_substrait_type( Ok(DataType::Interval(IntervalUnit::YearMonth)) } r#type::Kind::IntervalDay(_) => Ok(DataType::Interval(IntervalUnit::DayTime)), + r#type::Kind::IntervalCompound(_) => { + Ok(DataType::Interval(IntervalUnit::MonthDayNano)) + } r#type::Kind::UserDefined(u) => { if let Some(name) = extensions.types.get(&u.type_reference) { + #[allow(deprecated)] match name.as_ref() { + // Kept for backwards compatibility, producers should use IntervalCompound instead INTERVAL_MONTH_DAY_NANO_TYPE_NAME => Ok(DataType::Interval(IntervalUnit::MonthDayNano)), _ => not_impl_err!( "Unsupported Substrait user defined type with ref {} and variation {}", @@ -1842,18 +1862,17 @@ fn from_substrait_type( ), } } else { - // Kept for backwards compatibility, new plans should include the extension instead #[allow(deprecated)] match u.type_reference { - // Kept for backwards compatibility, use IntervalYear instead + // Kept for backwards compatibility, producers should use IntervalYear instead INTERVAL_YEAR_MONTH_TYPE_REF => { Ok(DataType::Interval(IntervalUnit::YearMonth)) } - // Kept for backwards compatibility, use IntervalDay instead + // Kept for backwards compatibility, producers should use IntervalDay instead INTERVAL_DAY_TIME_TYPE_REF => { Ok(DataType::Interval(IntervalUnit::DayTime)) } - // Not supported yet by Substrait + // Kept for backwards compatibility, producers should use IntervalCompound instead INTERVAL_MONTH_DAY_NANO_TYPE_REF => { Ok(DataType::Interval(IntervalUnit::MonthDayNano)) } @@ -2116,11 +2135,7 @@ fn from_substrait_literal( let s = d.scale.try_into().map_err(|e| { substrait_datafusion_err!("Failed to parse decimal scale: {e}") })?; - ScalarValue::Decimal128( - Some(std::primitive::i128::from_le_bytes(value)), - p, - s, - ) + ScalarValue::Decimal128(Some(i128::from_le_bytes(value)), p, s) } Some(LiteralType::List(l)) => { // Each element should start the name index from the same value, then we increase it @@ -2275,6 +2290,7 @@ fn from_substrait_literal( subseconds, precision_mode, })) => { + use interval_day_to_second::PrecisionMode; // DF only supports millisecond precision, so for any more granular type we lose precision let milliseconds = match precision_mode { Some(PrecisionMode::Microseconds(ms)) => ms / 1000, @@ -2299,6 +2315,35 @@ fn from_substrait_literal( Some(LiteralType::IntervalYearToMonth(IntervalYearToMonth { years, months })) => { ScalarValue::new_interval_ym(*years, *months) } + Some(LiteralType::IntervalCompound(IntervalCompound { + interval_year_to_month, + interval_day_to_second, + })) => match (interval_year_to_month, interval_day_to_second) { + ( + Some(IntervalYearToMonth { years, months }), + Some(IntervalDayToSecond { + days, + seconds, + subseconds, + precision_mode: + Some(interval_day_to_second::PrecisionMode::Precision(p)), + }), + ) => { + if *p < 0 || *p > 9 { + return plan_err!( + "Unsupported Substrait interval day to second precision: {}", + p + ); + } + let nanos = *subseconds * i64::pow(10, (9 - p) as u32); + ScalarValue::new_interval_mdn( + *years * 12 + months, + *days, + *seconds as i64 * NANOSECONDS + nanos, + ) + } + _ => return plan_err!("Substrait compound interval missing components"), + }, Some(LiteralType::FixedChar(c)) => ScalarValue::Utf8(Some(c.clone())), Some(LiteralType::UserDefined(user_defined)) => { // Helper function to prevent duplicating this code - can be inlined once the non-extension path is removed @@ -2329,6 +2374,8 @@ fn from_substrait_literal( if let Some(name) = extensions.types.get(&user_defined.type_reference) { match name.as_ref() { + // Kept for backwards compatibility - producers should use IntervalCompound instead + #[allow(deprecated)] INTERVAL_MONTH_DAY_NANO_TYPE_NAME => { interval_month_day_nano(user_defined)? } @@ -2341,10 +2388,9 @@ fn from_substrait_literal( } } } else { - // Kept for backwards compatibility - new plans should include extension instead #[allow(deprecated)] match user_defined.type_reference { - // Kept for backwards compatibility, use IntervalYearToMonth instead + // Kept for backwards compatibility, producers should useIntervalYearToMonth instead INTERVAL_YEAR_MONTH_TYPE_REF => { let Some(Val::Value(raw_val)) = user_defined.val.as_ref() else { return substrait_err!("Interval year month value is empty"); @@ -2359,7 +2405,7 @@ fn from_substrait_literal( value_slice, ))) } - // Kept for backwards compatibility, use IntervalDayToSecond instead + // Kept for backwards compatibility, producers should useIntervalDayToSecond instead INTERVAL_DAY_TIME_TYPE_REF => { let Some(Val::Value(raw_val)) = user_defined.val.as_ref() else { return substrait_err!("Interval day time value is empty"); @@ -2379,6 +2425,7 @@ fn from_substrait_literal( milliseconds, })) } + // Kept for backwards compatibility, producers should useIntervalCompound instead INTERVAL_MONTH_DAY_NANO_TYPE_REF => { interval_month_day_nano(user_defined)? } @@ -2603,7 +2650,7 @@ impl BuiltinExprBuilder { match name { "not" | "like" | "ilike" | "is_null" | "is_not_null" | "is_true" | "is_false" | "is_not_true" | "is_not_false" | "is_unknown" - | "is_not_unknown" | "negative" => Some(Self { + | "is_not_unknown" | "negative" | "negate" => Some(Self { expr_name: name.to_string(), }), _ => None, @@ -2624,8 +2671,9 @@ impl BuiltinExprBuilder { "ilike" => { Self::build_like_expr(ctx, true, f, input_schema, extensions).await } - "not" | "negative" | "is_null" | "is_not_null" | "is_true" | "is_false" - | "is_not_true" | "is_not_false" | "is_unknown" | "is_not_unknown" => { + "not" | "negative" | "negate" | "is_null" | "is_not_null" | "is_true" + | "is_false" | "is_not_true" | "is_not_false" | "is_unknown" + | "is_not_unknown" => { Self::build_unary_expr(ctx, &self.expr_name, f, input_schema, extensions) .await } @@ -2654,7 +2702,7 @@ impl BuiltinExprBuilder { let expr = match fn_name { "not" => Expr::Not(arg), - "negative" => Expr::Negative(arg), + "negative" | "negate" => Expr::Negative(arg), "is_null" => Expr::IsNull(arg), "is_not_null" => Expr::IsNotNull(arg), "is_true" => Expr::IsTrue(arg), @@ -2727,3 +2775,52 @@ impl BuiltinExprBuilder { })) } } + +#[cfg(test)] +mod test { + use crate::extensions::Extensions; + use crate::logical_plan::consumer::from_substrait_literal_without_names; + use arrow_buffer::IntervalMonthDayNano; + use datafusion::error::Result; + use datafusion::scalar::ScalarValue; + use substrait::proto::expression::literal::{ + interval_day_to_second, IntervalCompound, IntervalDayToSecond, + IntervalYearToMonth, LiteralType, + }; + use substrait::proto::expression::Literal; + + #[test] + fn interval_compound_different_precision() -> Result<()> { + // DF producer (and thus roundtrip) always uses precision = 9, + // this test exists to test with some other value. + let substrait = Literal { + nullable: false, + type_variation_reference: 0, + literal_type: Some(LiteralType::IntervalCompound(IntervalCompound { + interval_year_to_month: Some(IntervalYearToMonth { + years: 1, + months: 2, + }), + interval_day_to_second: Some(IntervalDayToSecond { + days: 3, + seconds: 4, + subseconds: 5, + precision_mode: Some( + interval_day_to_second::PrecisionMode::Precision(6), + ), + }), + })), + }; + + assert_eq!( + from_substrait_literal_without_names(&substrait, &Extensions::default())?, + ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano { + months: 14, + days: 3, + nanoseconds: 4_000_005_000 + })) + ); + + Ok(()) + } +} diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 4855af683b7da..408885f70687f 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -21,7 +21,6 @@ use datafusion::optimizer::AnalyzerRule; use std::sync::Arc; use substrait::proto::expression_reference::ExprType; -use arrow_buffer::ToByteSlice; use datafusion::arrow::datatypes::{Field, IntervalUnit}; use datafusion::logical_expr::{ Distinct, FetchType, Like, Partitioning, SkipType, WindowFrameUnits, @@ -39,10 +38,11 @@ use crate::variation_const::{ DATE_32_TYPE_VARIATION_REF, DATE_64_TYPE_VARIATION_REF, DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME, LARGE_CONTAINER_TYPE_VARIATION_REF, - UNSIGNED_INTEGER_TYPE_VARIATION_REF, VIEW_CONTAINER_TYPE_VARIATION_REF, + LARGE_CONTAINER_TYPE_VARIATION_REF, UNSIGNED_INTEGER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; +use datafusion::arrow::temporal_conversions::NANOSECONDS; use datafusion::common::{ exec_err, internal_err, not_impl_err, plan_err, substrait_datafusion_err, substrait_err, DFSchemaRef, ToDFSchema, @@ -58,8 +58,8 @@ use substrait::proto::exchange_rel::{ExchangeKind, RoundRobin, ScatterFields}; use substrait::proto::expression::literal::interval_day_to_second::PrecisionMode; use substrait::proto::expression::literal::map::KeyValue; use substrait::proto::expression::literal::{ - user_defined, IntervalDayToSecond, IntervalYearToMonth, List, Map, - PrecisionTimestamp, Struct, UserDefined, + IntervalCompound, IntervalDayToSecond, IntervalYearToMonth, List, Map, + PrecisionTimestamp, Struct, }; use substrait::proto::expression::subquery::InPredicate; use substrait::proto::expression::window_function::BoundsType; @@ -114,7 +114,7 @@ pub fn to_substrait_plan(plan: &LogicalPlan, ctx: &SessionContext) -> Result>>()?; - let substrait_schema = to_substrait_named_struct(schema, &mut extensions)?; + let substrait_schema = to_substrait_named_struct(schema)?; Ok(Box::new(ExtendedExpression { advanced_extensions: None, @@ -203,7 +203,7 @@ pub fn to_substrait_rel( }); let table_schema = scan.source.schema().to_dfschema_ref()?; - let base_schema = to_substrait_named_struct(&table_schema, extensions)?; + let base_schema = to_substrait_named_struct(&table_schema)?; Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { @@ -229,7 +229,7 @@ pub fn to_substrait_rel( Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { common: None, - base_schema: Some(to_substrait_named_struct(&e.schema, extensions)?), + base_schema: Some(to_substrait_named_struct(&e.schema)?), filter: None, best_effort_filter: None, projection: None, @@ -268,7 +268,7 @@ pub fn to_substrait_rel( Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { common: None, - base_schema: Some(to_substrait_named_struct(&v.schema, extensions)?), + base_schema: Some(to_substrait_named_struct(&v.schema)?), filter: None, best_effort_filter: None, projection: None, @@ -491,7 +491,7 @@ pub fn to_substrait_rel( .map(|ptr| *ptr) .collect(); Ok(Box::new(Rel { - rel_type: Some(substrait::proto::rel::RelType::Set(SetRel { + rel_type: Some(RelType::Set(SetRel { common: None, inputs: input_rels, op: set_rel::SetOp::UnionAll as i32, // UNION DISTINCT gets translated to AGGREGATION + UNION ALL @@ -664,10 +664,7 @@ fn flatten_names(field: &Field, skip_self: bool, names: &mut Vec) -> Res Ok(()) } -fn to_substrait_named_struct( - schema: &DFSchemaRef, - extensions: &mut Extensions, -) -> Result { +fn to_substrait_named_struct(schema: &DFSchemaRef) -> Result { let mut names = Vec::with_capacity(schema.fields().len()); for field in schema.fields() { flatten_names(field, false, &mut names)?; @@ -677,7 +674,7 @@ fn to_substrait_named_struct( types: schema .fields() .iter() - .map(|f| to_substrait_type(f.data_type(), f.is_nullable(), extensions)) + .map(|f| to_substrait_type(f.data_type(), f.is_nullable())) .collect::>()?, type_variation_reference: DEFAULT_TYPE_VARIATION_REF, nullability: r#type::Nullability::Unspecified as i32, @@ -744,7 +741,7 @@ pub fn operator_to_name(op: Operator) -> &'static str { Operator::Minus => "subtract", Operator::Multiply => "multiply", Operator::Divide => "divide", - Operator::Modulo => "mod", + Operator::Modulo => "modulus", Operator::And => "and", Operator::Or => "or", Operator::IsDistinctFrom => "is_distinct_from", @@ -1150,7 +1147,7 @@ pub fn to_substrait_rex( Ok(Expression { rex_type: Some(RexType::Cast(Box::new( substrait::proto::expression::Cast { - r#type: Some(to_substrait_type(data_type, true, extensions)?), + r#type: Some(to_substrait_type(data_type, true)?), input: Some(Box::new(to_substrait_rex( ctx, expr, @@ -1344,7 +1341,7 @@ pub fn to_substrait_rex( ), Expr::Negative(arg) => to_substrait_unary_scalar_fn( ctx, - "negative", + "negate", arg, schema, col_ref_offset, @@ -1356,11 +1353,7 @@ pub fn to_substrait_rex( } } -fn to_substrait_type( - dt: &DataType, - nullable: bool, - extensions: &mut Extensions, -) -> Result { +fn to_substrait_type(dt: &DataType, nullable: bool) -> Result { let nullability = if nullable { r#type::Nullability::Nullable as i32 } else { @@ -1489,16 +1482,14 @@ fn to_substrait_type( })), }), IntervalUnit::MonthDayNano => { - // Substrait doesn't currently support this type, so we represent it as a UDT Ok(substrait::proto::Type { - kind: Some(r#type::Kind::UserDefined(r#type::UserDefined { - type_reference: extensions.register_type( - INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string(), - ), - type_variation_reference: DEFAULT_TYPE_VARIATION_REF, - nullability, - type_parameters: vec![], - })), + kind: Some(r#type::Kind::IntervalCompound( + r#type::IntervalCompound { + type_variation_reference: DEFAULT_TYPE_VARIATION_REF, + nullability, + precision: 9, // nanos + }, + )), }) } } @@ -1547,8 +1538,7 @@ fn to_substrait_type( })), }), DataType::List(inner) => { - let inner_type = - to_substrait_type(inner.data_type(), inner.is_nullable(), extensions)?; + let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { r#type: Some(Box::new(inner_type)), @@ -1558,8 +1548,7 @@ fn to_substrait_type( }) } DataType::LargeList(inner) => { - let inner_type = - to_substrait_type(inner.data_type(), inner.is_nullable(), extensions)?; + let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable())?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::List(Box::new(r#type::List { r#type: Some(Box::new(inner_type)), @@ -1573,12 +1562,10 @@ fn to_substrait_type( let key_type = to_substrait_type( key_and_value[0].data_type(), key_and_value[0].is_nullable(), - extensions, )?; let value_type = to_substrait_type( key_and_value[1].data_type(), key_and_value[1].is_nullable(), - extensions, )?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::Map(Box::new(r#type::Map { @@ -1594,9 +1581,7 @@ fn to_substrait_type( DataType::Struct(fields) => { let field_types = fields .iter() - .map(|field| { - to_substrait_type(field.data_type(), field.is_nullable(), extensions) - }) + .map(|field| to_substrait_type(field.data_type(), field.is_nullable())) .collect::>>()?; Ok(substrait::proto::Type { kind: Some(r#type::Kind::Struct(r#type::Struct { @@ -1783,7 +1768,6 @@ fn to_substrait_literal( literal_type: Some(LiteralType::Null(to_substrait_type( &value.data_type(), true, - extensions, )?)), }); } @@ -1892,23 +1876,21 @@ fn to_substrait_literal( }), DEFAULT_TYPE_VARIATION_REF, ), - ScalarValue::IntervalMonthDayNano(Some(i)) => { - // IntervalMonthDayNano is internally represented as a 128-bit integer, containing - // months (32bit), days (32bit), and nanoseconds (64bit) - let bytes = i.to_byte_slice(); - ( - LiteralType::UserDefined(UserDefined { - type_reference: extensions - .register_type(INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string()), - type_parameters: vec![], - val: Some(user_defined::Val::Value(ProtoAny { - type_url: INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string(), - value: bytes.to_vec().into(), - })), + ScalarValue::IntervalMonthDayNano(Some(i)) => ( + LiteralType::IntervalCompound(IntervalCompound { + interval_year_to_month: Some(IntervalYearToMonth { + years: i.months / 12, + months: i.months % 12, }), - DEFAULT_TYPE_VARIATION_REF, - ) - } + interval_day_to_second: Some(IntervalDayToSecond { + days: i.days, + seconds: (i.nanoseconds / NANOSECONDS) as i32, + subseconds: i.nanoseconds % NANOSECONDS, + precision_mode: Some(PrecisionMode::Precision(9)), // nanoseconds + }), + }), + DEFAULT_TYPE_VARIATION_REF, + ), ScalarValue::IntervalDayTime(Some(i)) => ( LiteralType::IntervalDayToSecond(IntervalDayToSecond { days: i.days, @@ -1964,7 +1946,7 @@ fn to_substrait_literal( ), ScalarValue::Map(m) => { let map = if m.is_empty() || m.value(0).is_empty() { - let mt = to_substrait_type(m.data_type(), m.is_nullable(), extensions)?; + let mt = to_substrait_type(m.data_type(), m.is_nullable())?; let mt = match mt { substrait::proto::Type { kind: Some(r#type::Kind::Map(mt)), @@ -2049,11 +2031,7 @@ fn convert_array_to_literal_list( .collect::>>()?; if values.is_empty() { - let lt = match to_substrait_type( - array.data_type(), - array.is_nullable(), - extensions, - )? { + let lt = match to_substrait_type(array.data_type(), array.is_nullable())? { substrait::proto::Type { kind: Some(r#type::Kind::List(lt)), } => lt.as_ref().to_owned(), @@ -2179,7 +2157,6 @@ mod test { use datafusion::arrow::datatypes::{Field, Fields, Schema}; use datafusion::common::scalar::ScalarStructBuilder; use datafusion::common::DFSchema; - use std::collections::HashMap; #[test] fn round_trip_literals() -> Result<()> { @@ -2310,39 +2287,6 @@ mod test { Ok(()) } - #[test] - fn custom_type_literal_extensions() -> Result<()> { - let mut extensions = Extensions::default(); - // IntervalMonthDayNano is represented as a custom type in Substrait - let scalar = ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano::new( - 17, 25, 1234567890, - ))); - let substrait_literal = to_substrait_literal(&scalar, &mut extensions)?; - let roundtrip_scalar = - from_substrait_literal_without_names(&substrait_literal, &extensions)?; - assert_eq!(scalar, roundtrip_scalar); - - assert_eq!( - extensions, - Extensions { - functions: HashMap::new(), - types: HashMap::from([( - 0, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string() - )]), - type_variations: HashMap::new(), - } - ); - - // Check we fail if we don't propagate extensions - assert!(from_substrait_literal_without_names( - &substrait_literal, - &Extensions::default() - ) - .is_err()); - Ok(()) - } - #[test] fn round_trip_types() -> Result<()> { round_trip_type(DataType::Boolean)?; @@ -2414,50 +2358,17 @@ mod test { fn round_trip_type(dt: DataType) -> Result<()> { println!("Checking round trip of {dt:?}"); - let mut extensions = Extensions::default(); - // As DataFusion doesn't consider nullability as a property of the type, but field, // it doesn't matter if we set nullability to true or false here. - let substrait = to_substrait_type(&dt, true, &mut extensions)?; - let roundtrip_dt = from_substrait_type_without_names(&substrait, &extensions)?; + let substrait = to_substrait_type(&dt, true)?; + let roundtrip_dt = + from_substrait_type_without_names(&substrait, &Extensions::default())?; assert_eq!(dt, roundtrip_dt); Ok(()) } - #[test] - fn custom_type_extensions() -> Result<()> { - let mut extensions = Extensions::default(); - // IntervalMonthDayNano is represented as a custom type in Substrait - let dt = DataType::Interval(IntervalUnit::MonthDayNano); - - let substrait = to_substrait_type(&dt, true, &mut extensions)?; - let roundtrip_dt = from_substrait_type_without_names(&substrait, &extensions)?; - assert_eq!(dt, roundtrip_dt); - - assert_eq!( - extensions, - Extensions { - functions: HashMap::new(), - types: HashMap::from([( - 0, - INTERVAL_MONTH_DAY_NANO_TYPE_NAME.to_string() - )]), - type_variations: HashMap::new(), - } - ); - - // Check we fail if we don't propagate extensions - assert!( - from_substrait_type_without_names(&substrait, &Extensions::default()) - .is_err() - ); - - Ok(()) - } - #[test] fn named_struct_names() -> Result<()> { - let mut extensions = Extensions::default(); let schema = DFSchemaRef::new(DFSchema::try_from(Schema::new(vec![ Field::new("int", DataType::Int32, true), Field::new( @@ -2472,7 +2383,7 @@ mod test { Field::new("trailer", DataType::Float64, true), ]))?); - let named_struct = to_substrait_named_struct(&schema, &mut extensions)?; + let named_struct = to_substrait_named_struct(&schema)?; // Struct field names should be flattened DFS style // List field names should be omitted @@ -2481,7 +2392,8 @@ mod test { vec!["int", "struct", "inner", "trailer"] ); - let roundtrip_schema = from_substrait_named_struct(&named_struct, &extensions)?; + let roundtrip_schema = + from_substrait_named_struct(&named_struct, &Extensions::default())?; assert_eq!(schema.as_ref(), &roundtrip_schema); Ok(()) } diff --git a/datafusion/substrait/src/variation_const.rs b/datafusion/substrait/src/variation_const.rs index a3e76389d5104..58774db424da1 100644 --- a/datafusion/substrait/src/variation_const.rs +++ b/datafusion/substrait/src/variation_const.rs @@ -96,7 +96,7 @@ pub const INTERVAL_DAY_TIME_TYPE_REF: u32 = 2; /// [`ScalarValue::IntervalMonthDayNano`]: datafusion::common::ScalarValue::IntervalMonthDayNano #[deprecated( since = "41.0.0", - note = "Use Substrait `UserDefinedType` with name `INTERVAL_MONTH_DAY_NANO_TYPE_NAME` instead" + note = "Use Substrait `IntervalCompund` type instead" )] pub const INTERVAL_MONTH_DAY_NANO_TYPE_REF: u32 = 3; @@ -104,4 +104,8 @@ pub const INTERVAL_MONTH_DAY_NANO_TYPE_REF: u32 = 3; /// /// [`DataType::Interval`]: datafusion::arrow::datatypes::DataType::Interval /// [`IntervalUnit::MonthDayNano`]: datafusion::arrow::datatypes::IntervalUnit::MonthDayNano +#[deprecated( + since = "43.0.0", + note = "Use Substrait `IntervalCompund` type instead" +)] pub const INTERVAL_MONTH_DAY_NANO_TYPE_NAME: &str = "interval-month-day-nano"; diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 06a047b108bd3..04530dd34d4bf 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -39,10 +39,7 @@ use std::hash::Hash; use std::sync::Arc; use datafusion::execution::session_state::SessionStateBuilder; -use substrait::proto::extensions::simple_extension_declaration::{ - ExtensionType, MappingType, -}; -use substrait::proto::extensions::SimpleExtensionDeclaration; +use substrait::proto::extensions::simple_extension_declaration::MappingType; use substrait::proto::rel::RelType; use substrait::proto::{plan_rel, Plan, Rel}; @@ -69,7 +66,7 @@ impl SerializerRegistry for MockSerializerRegistry { &self, name: &str, bytes: &[u8], - ) -> Result> { + ) -> Result> { if name == "MockUserDefinedLogicalPlan" { MockUserDefinedLogicalPlan::deserialize(bytes) } else { @@ -230,23 +227,6 @@ async fn select_with_reused_functions() -> Result<()> { Ok(()) } -#[tokio::test] -async fn roundtrip_udt_extensions() -> Result<()> { - let ctx = create_context().await?; - let proto = - roundtrip_with_ctx("SELECT INTERVAL '1 YEAR 1 DAY 1 SECOND' FROM data", ctx) - .await?; - let expected_type = SimpleExtensionDeclaration { - mapping_type: Some(MappingType::ExtensionType(ExtensionType { - extension_uri_reference: u32::MAX, - type_anchor: 0, - name: "interval-month-day-nano".to_string(), - })), - }; - assert_eq!(proto.extensions, vec![expected_type]); - Ok(()) -} - #[tokio::test] async fn select_with_filter_date() -> Result<()> { roundtrip("SELECT * FROM data WHERE c > CAST('2020-01-01' AS DATE)").await @@ -593,6 +573,11 @@ async fn roundtrip_ilike() -> Result<()> { roundtrip("SELECT f FROM data WHERE f ILIKE 'a%b'").await } +#[tokio::test] +async fn roundtrip_modulus() -> Result<()> { + roundtrip("SELECT a%3 from data").await +} + #[tokio::test] async fn roundtrip_not() -> Result<()> { roundtrip("SELECT * FROM data WHERE NOT d").await @@ -680,6 +665,19 @@ async fn aggregate_wo_projection_consume() -> Result<()> { .await } +#[tokio::test] +async fn aggregate_wo_projection_sorted_consume() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/aggregate_sorted_no_project.substrait.json"); + + assert_expected_plan_substrait( + proto_plan, + "Aggregate: groupBy=[[data.a]], aggr=[[count(data.a) ORDER BY [data.a DESC NULLS FIRST] AS countA]]\ + \n TableScan: data projection=[a]", + ) + .await +} + #[tokio::test] async fn simple_intersect_consume() -> Result<()> { let proto_plan = read_json("tests/testdata/test_plans/intersect.substrait.json"); @@ -1000,7 +998,7 @@ async fn roundtrip_aggregate_udf() -> Result<()> { } fn size(&self) -> usize { - std::mem::size_of_val(self) + size_of_val(self) } } @@ -1020,8 +1018,9 @@ async fn roundtrip_aggregate_udf() -> Result<()> { let ctx = create_context().await?; ctx.register_udaf(dummy_agg); + roundtrip_with_ctx("select dummy_agg(a) from data", ctx.clone()).await?; + roundtrip_with_ctx("select dummy_agg(a order by a) from data", ctx.clone()).await?; - roundtrip_with_ctx("select dummy_agg(a) from data", ctx).await?; Ok(()) } diff --git a/datafusion/substrait/tests/cases/serialize.rs b/datafusion/substrait/tests/cases/serialize.rs index 72d685817d7d4..54d55d1b6f10e 100644 --- a/datafusion/substrait/tests/cases/serialize.rs +++ b/datafusion/substrait/tests/cases/serialize.rs @@ -20,13 +20,12 @@ mod tests { use datafusion::datasource::provider_as_source; use datafusion::logical_expr::LogicalPlanBuilder; use datafusion_substrait::logical_plan::consumer::from_substrait_plan; - use datafusion_substrait::logical_plan::producer; + use datafusion_substrait::logical_plan::producer::to_substrait_plan; use datafusion_substrait::serializer; use datafusion::error::Result; use datafusion::prelude::*; - use datafusion_substrait::logical_plan::producer::to_substrait_plan; use std::fs; use substrait::proto::plan_rel::RelType; use substrait::proto::rel_common::{Emit, EmitKind}; @@ -61,7 +60,7 @@ mod tests { let ctx = create_context().await?; let table = provider_as_source(ctx.table_provider("data").await?); let table_scan = LogicalPlanBuilder::scan("data", table, None)?.build()?; - let convert_result = producer::to_substrait_plan(&table_scan, &ctx); + let convert_result = to_substrait_plan(&table_scan, &ctx); assert!(convert_result.is_ok()); Ok(()) diff --git a/datafusion/substrait/tests/testdata/test_plans/aggregate_sorted_no_project.substrait.json b/datafusion/substrait/tests/testdata/test_plans/aggregate_sorted_no_project.substrait.json new file mode 100644 index 0000000000000..d5170223cd65b --- /dev/null +++ b/datafusion/substrait/tests/testdata/test_plans/aggregate_sorted_no_project.substrait.json @@ -0,0 +1,113 @@ +{ + "extensionUris": [ + { + "uri": "https://github.com/substrait-io/substrait/blob/main/extensions/functions_aggregate_generic.yaml" + } + ], + "extensions": [ + { + "extensionFunction": { + "functionAnchor": 185, + "name": "count:any" + } + } + ], + "relations": [ + { + "root": { + "input": { + "aggregate": { + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "a" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_NULLABLE" + } + }, + "namedTable": { + "names": [ + "data" + ] + } + } + }, + "groupings": [ + { + "groupingExpressions": [ + { + "selection": { + "directReference": { + "structField": {} + }, + "rootReference": {} + } + } + ] + } + ], + "measures": [ + { + "measure": { + "functionReference": 185, + "phase": "AGGREGATION_PHASE_INITIAL_TO_RESULT", + "outputType": { + "i64": {} + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": {} + }, + "rootReference": {} + } + } + } + ], + "sorts": [ + { + "expr": { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": { + } + } + }, + "direction": "SORT_DIRECTION_DESC_NULLS_FIRST" + } + ] + } + } + ] + } + }, + "names": [ + "a", + "countA" + ] + } + } + ], + "version": { + "minorNumber": 54, + "producer": "manual" + } +} \ No newline at end of file diff --git a/dev/update_function_docs.sh b/dev/update_function_docs.sh index 13bc22afcc135..ad3bc9c7f69ca 100755 --- a/dev/update_function_docs.sh +++ b/dev/update_function_docs.sh @@ -24,7 +24,7 @@ SOURCE_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" cd "${SOURCE_DIR}/../" && pwd -TARGET_FILE="docs/source/user-guide/sql/aggregate_functions_new.md" +TARGET_FILE="docs/source/user-guide/sql/aggregate_functions.md" PRINT_AGGREGATE_FUNCTION_DOCS_COMMAND="cargo run --manifest-path datafusion/core/Cargo.toml --bin print_functions_docs -- aggregate" echo "Inserting header" @@ -56,13 +56,7 @@ update documentation for an individual UDF or the dev/update_function_docs.sh file for updating surrounding text. --> -# Aggregate Functions (NEW) - -Note: this documentation is in the process of being migrated to be [automatically created from the codebase]. -Please see the [Aggregate Functions (old)](aggregate_functions.md) page for -the rest of the documentation. - -[automatically created from the codebase]: https://github.com/apache/datafusion/issues/12740 +# Aggregate Functions Aggregate functions operate on a set of values to compute a single result. EOF @@ -75,7 +69,7 @@ npx prettier@2.3.2 --write "$TARGET_FILE" echo "'$TARGET_FILE' successfully updated!" -TARGET_FILE="docs/source/user-guide/sql/scalar_functions_new.md" +TARGET_FILE="docs/source/user-guide/sql/scalar_functions.md" PRINT_SCALAR_FUNCTION_DOCS_COMMAND="cargo run --manifest-path datafusion/core/Cargo.toml --bin print_functions_docs -- scalar" echo "Inserting header" @@ -107,13 +101,7 @@ update documentation for an individual UDF or the dev/update_function_docs.sh file for updating surrounding text. --> -# Scalar Functions (NEW) - -Note: this documentation is in the process of being migrated to be [automatically created from the codebase]. -Please see the [Scalar Functions (old)](aggregate_functions.md) page for -the rest of the documentation. - -[automatically created from the codebase]: https://github.com/apache/datafusion/issues/12740 +# Scalar Functions EOF diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index 77f527c92cdae..d9fc28a81772d 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -17,6 +17,843 @@ under the License. --> + + # Aggregate Functions -Note: this documentation has been migrated to [Aggregate Functions (new)](aggregate_functions_new.md) +Aggregate functions operate on a set of values to compute a single result. + +## General Functions + +- [array_agg](#array_agg) +- [avg](#avg) +- [bit_and](#bit_and) +- [bit_or](#bit_or) +- [bit_xor](#bit_xor) +- [bool_and](#bool_and) +- [bool_or](#bool_or) +- [count](#count) +- [first_value](#first_value) +- [grouping](#grouping) +- [last_value](#last_value) +- [max](#max) +- [mean](#mean) +- [median](#median) +- [min](#min) +- [string_agg](#string_agg) +- [sum](#sum) +- [var](#var) +- [var_pop](#var_pop) +- [var_population](#var_population) +- [var_samp](#var_samp) +- [var_sample](#var_sample) + +### `array_agg` + +Returns an array created from the expression elements. If ordering is required, elements are inserted in the specified order. + +``` +array_agg(expression [ORDER BY expression]) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT array_agg(column_name ORDER BY other_column) FROM table_name; ++-----------------------------------------------+ +| array_agg(column_name ORDER BY other_column) | ++-----------------------------------------------+ +| [element1, element2, element3] | ++-----------------------------------------------+ +``` + +### `avg` + +Returns the average of numeric values in the specified column. + +``` +avg(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT avg(column_name) FROM table_name; ++---------------------------+ +| avg(column_name) | ++---------------------------+ +| 42.75 | ++---------------------------+ +``` + +#### Aliases + +- mean + +### `bit_and` + +Computes the bitwise AND of all non-null input values. + +``` +bit_and(expression) +``` + +#### Arguments + +- **expression**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `bit_or` + +Computes the bitwise OR of all non-null input values. + +``` +bit_or(expression) +``` + +#### Arguments + +- **expression**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `bit_xor` + +Computes the bitwise exclusive OR of all non-null input values. + +``` +bit_xor(expression) +``` + +#### Arguments + +- **expression**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `bool_and` + +Returns true if all non-null input values are true, otherwise false. + +``` +bool_and(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT bool_and(column_name) FROM table_name; ++----------------------------+ +| bool_and(column_name) | ++----------------------------+ +| true | ++----------------------------+ +``` + +### `bool_or` + +Returns true if all non-null input values are true, otherwise false. + +``` +bool_and(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT bool_and(column_name) FROM table_name; ++----------------------------+ +| bool_and(column_name) | ++----------------------------+ +| true | ++----------------------------+ +``` + +### `count` + +Returns the number of non-null values in the specified column. To include null values in the total count, use `count(*)`. + +``` +count(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT count(column_name) FROM table_name; ++-----------------------+ +| count(column_name) | ++-----------------------+ +| 100 | ++-----------------------+ + +> SELECT count(*) FROM table_name; ++------------------+ +| count(*) | ++------------------+ +| 120 | ++------------------+ +``` + +### `first_value` + +Returns the first element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group. + +``` +first_value(expression [ORDER BY expression]) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT first_value(column_name ORDER BY other_column) FROM table_name; ++-----------------------------------------------+ +| first_value(column_name ORDER BY other_column)| ++-----------------------------------------------+ +| first_element | ++-----------------------------------------------+ +``` + +### `grouping` + +Returns 1 if the data is aggregated across the specified column, or 0 if it is not aggregated in the result set. + +``` +grouping(expression) +``` + +#### Arguments + +- **expression**: Expression to evaluate whether data is aggregated across the specified column. Can be a constant, column, or function. + +#### Example + +```sql +> SELECT column_name, GROUPING(column_name) AS group_column + FROM table_name + GROUP BY GROUPING SETS ((column_name), ()); ++-------------+-------------+ +| column_name | group_column | ++-------------+-------------+ +| value1 | 0 | +| value2 | 0 | +| NULL | 1 | ++-------------+-------------+ +``` + +### `last_value` + +Returns the first element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group. + +``` +first_value(expression [ORDER BY expression]) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT first_value(column_name ORDER BY other_column) FROM table_name; ++-----------------------------------------------+ +| first_value(column_name ORDER BY other_column)| ++-----------------------------------------------+ +| first_element | ++-----------------------------------------------+ +``` + +### `max` + +Returns the maximum value in the specified column. + +``` +max(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT max(column_name) FROM table_name; ++----------------------+ +| max(column_name) | ++----------------------+ +| 150 | ++----------------------+ +``` + +### `mean` + +_Alias of [avg](#avg)._ + +### `median` + +Returns the median value in the specified column. + +``` +median(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT median(column_name) FROM table_name; ++----------------------+ +| median(column_name) | ++----------------------+ +| 45.5 | ++----------------------+ +``` + +### `min` + +Returns the maximum value in the specified column. + +``` +max(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT max(column_name) FROM table_name; ++----------------------+ +| max(column_name) | ++----------------------+ +| 150 | ++----------------------+ +``` + +### `string_agg` + +Concatenates the values of string expressions and places separator values between them. + +``` +string_agg(expression, delimiter) +``` + +#### Arguments + +- **expression**: The string expression to concatenate. Can be a column or any valid string expression. +- **delimiter**: A literal string used as a separator between the concatenated values. + +#### Example + +```sql +> SELECT string_agg(name, ', ') AS names_list + FROM employee; ++--------------------------+ +| names_list | ++--------------------------+ +| Alice, Bob, Charlie | ++--------------------------+ +``` + +### `sum` + +Returns the sum of all values in the specified column. + +``` +sum(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT sum(column_name) FROM table_name; ++-----------------------+ +| sum(column_name) | ++-----------------------+ +| 12345 | ++-----------------------+ +``` + +### `var` + +Returns the statistical sample variance of a set of numbers. + +``` +var(expression) +``` + +#### Arguments + +- **expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Aliases + +- var_sample +- var_samp + +### `var_pop` + +Returns the statistical population variance of a set of numbers. + +``` +var_pop(expression) +``` + +#### Arguments + +- **expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Aliases + +- var_population + +### `var_population` + +_Alias of [var_pop](#var_pop)._ + +### `var_samp` + +_Alias of [var](#var)._ + +### `var_sample` + +_Alias of [var](#var)._ + +## Statistical Functions + +- [corr](#corr) +- [covar](#covar) +- [covar_pop](#covar_pop) +- [covar_samp](#covar_samp) +- [nth_value](#nth_value) +- [regr_avgx](#regr_avgx) +- [regr_avgy](#regr_avgy) +- [regr_count](#regr_count) +- [regr_intercept](#regr_intercept) +- [regr_r2](#regr_r2) +- [regr_slope](#regr_slope) +- [regr_sxx](#regr_sxx) +- [regr_sxy](#regr_sxy) +- [regr_syy](#regr_syy) +- [stddev](#stddev) +- [stddev_pop](#stddev_pop) +- [stddev_samp](#stddev_samp) + +### `corr` + +Returns the coefficient of correlation between two numeric values. + +``` +corr(expression1, expression2) +``` + +#### Arguments + +- **expression1**: First expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression2**: Second expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT corr(column1, column2) FROM table_name; ++--------------------------------+ +| corr(column1, column2) | ++--------------------------------+ +| 0.85 | ++--------------------------------+ +``` + +### `covar` + +_Alias of [covar_samp](#covar_samp)._ + +### `covar_pop` + +Returns the sample covariance of a set of number pairs. + +``` +covar_samp(expression1, expression2) +``` + +#### Arguments + +- **expression1**: First expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression2**: Second expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT covar_samp(column1, column2) FROM table_name; ++-----------------------------------+ +| covar_samp(column1, column2) | ++-----------------------------------+ +| 8.25 | ++-----------------------------------+ +``` + +### `covar_samp` + +Returns the sample covariance of a set of number pairs. + +``` +covar_samp(expression1, expression2) +``` + +#### Arguments + +- **expression1**: First expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression2**: Second expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT covar_samp(column1, column2) FROM table_name; ++-----------------------------------+ +| covar_samp(column1, column2) | ++-----------------------------------+ +| 8.25 | ++-----------------------------------+ +``` + +#### Aliases + +- covar + +### `nth_value` + +Returns the nth value in a group of values. + +``` +nth_value(expression, n ORDER BY expression) +``` + +#### Arguments + +- **expression**: The column or expression to retrieve the nth value from. +- **n**: The position (nth) of the value to retrieve, based on the ordering. + +#### Example + +```sql +> SELECT dept_id, salary, NTH_VALUE(salary, 2) OVER (PARTITION BY dept_id ORDER BY salary ASC) AS second_salary_by_dept + FROM employee; ++---------+--------+-------------------------+ +| dept_id | salary | second_salary_by_dept | ++---------+--------+-------------------------+ +| 1 | 30000 | NULL | +| 1 | 40000 | 40000 | +| 1 | 50000 | 40000 | +| 2 | 35000 | NULL | +| 2 | 45000 | 45000 | ++---------+--------+-------------------------+ +``` + +### `regr_avgx` + +Computes the average of the independent variable (input) expression_x for the non-null paired data points. + +``` +regr_avgx(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_avgy` + +Computes the average of the dependent variable (output) expression_y for the non-null paired data points. + +``` +regr_avgy(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_count` + +Counts the number of non-null paired data points. + +``` +regr_count(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_intercept` + +Computes the y-intercept of the linear regression line. For the equation (y = kx + b), this function returns b. + +``` +regr_intercept(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_r2` + +Computes the square of the correlation coefficient between the independent and dependent variables. + +``` +regr_r2(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_slope` + +Returns the slope of the linear regression line for non-null pairs in aggregate columns. Given input column Y and X: regr_slope(Y, X) returns the slope (k in Y = k\*X + b) using minimal RSS fitting. + +``` +regr_slope(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_sxx` + +Computes the sum of squares of the independent variable. + +``` +regr_sxx(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_sxy` + +Computes the sum of products of paired data points. + +``` +regr_sxy(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `regr_syy` + +Computes the sum of squares of the dependent variable. + +``` +regr_syy(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `stddev` + +Returns the standard deviation of a set of numbers. + +``` +stddev(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT stddev(column_name) FROM table_name; ++----------------------+ +| stddev(column_name) | ++----------------------+ +| 12.34 | ++----------------------+ +``` + +#### Aliases + +- stddev_samp + +### `stddev_pop` + +Returns the standard deviation of a set of numbers. + +``` +stddev(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT stddev(column_name) FROM table_name; ++----------------------+ +| stddev(column_name) | ++----------------------+ +| 12.34 | ++----------------------+ +``` + +### `stddev_samp` + +_Alias of [stddev](#stddev)._ + +## Approximate Functions + +- [approx_distinct](#approx_distinct) +- [approx_median](#approx_median) +- [approx_percentile_cont](#approx_percentile_cont) +- [approx_percentile_cont_with_weight](#approx_percentile_cont_with_weight) + +### `approx_distinct` + +Returns the approximate number of distinct input values calculated using the HyperLogLog algorithm. + +``` +approx_distinct(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT approx_distinct(column_name) FROM table_name; ++-----------------------------------+ +| approx_distinct(column_name) | ++-----------------------------------+ +| 42 | ++-----------------------------------+ +``` + +### `approx_median` + +Returns the approximate median (50th percentile) of input values. It is an alias of `approx_percentile_cont(x, 0.5)`. + +``` +approx_median(expression) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> SELECT approx_median(column_name) FROM table_name; ++-----------------------------------+ +| approx_median(column_name) | ++-----------------------------------+ +| 23.5 | ++-----------------------------------+ +``` + +### `approx_percentile_cont` + +Returns the approximate percentile of input values using the t-digest algorithm. + +``` +approx_percentile_cont(expression, percentile, centroids) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **percentile**: Percentile to compute. Must be a float value between 0 and 1 (inclusive). +- **centroids**: Number of centroids to use in the t-digest algorithm. _Default is 100_. A higher number results in more accurate approximation but requires more memory. + +#### Example + +```sql +> SELECT approx_percentile_cont(column_name, 0.75, 100) FROM table_name; ++-------------------------------------------------+ +| approx_percentile_cont(column_name, 0.75, 100) | ++-------------------------------------------------+ +| 65.0 | ++-------------------------------------------------+ +``` + +### `approx_percentile_cont_with_weight` + +Returns the weighted approximate percentile of input values using the t-digest algorithm. + +``` +approx_percentile_cont_with_weight(expression, weight, percentile) +``` + +#### Arguments + +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **weight**: Expression to use as weight. Can be a constant, column, or function, and any combination of arithmetic operators. +- **percentile**: Percentile to compute. Must be a float value between 0 and 1 (inclusive). + +#### Example + +```sql +> SELECT approx_percentile_cont_with_weight(column_name, weight_column, 0.90) FROM table_name; ++----------------------------------------------------------------------+ +| approx_percentile_cont_with_weight(column_name, weight_column, 0.90) | ++----------------------------------------------------------------------+ +| 78.5 | ++----------------------------------------------------------------------+ +``` diff --git a/docs/source/user-guide/sql/aggregate_functions_new.md b/docs/source/user-guide/sql/aggregate_functions_new.md deleted file mode 100644 index ad6d15b94ee53..0000000000000 --- a/docs/source/user-guide/sql/aggregate_functions_new.md +++ /dev/null @@ -1,865 +0,0 @@ - - - - -# Aggregate Functions (NEW) - -Note: this documentation is in the process of being migrated to be [automatically created from the codebase]. -Please see the [Aggregate Functions (old)](aggregate_functions.md) page for -the rest of the documentation. - -[automatically created from the codebase]: https://github.com/apache/datafusion/issues/12740 - -Aggregate functions operate on a set of values to compute a single result. - -## General Functions - -- [array_agg](#array_agg) -- [avg](#avg) -- [bit_and](#bit_and) -- [bit_or](#bit_or) -- [bit_xor](#bit_xor) -- [bool_and](#bool_and) -- [bool_or](#bool_or) -- [count](#count) -- [first_value](#first_value) -- [grouping](#grouping) -- [last_value](#last_value) -- [max](#max) -- [mean](#mean) -- [median](#median) -- [min](#min) -- [string_agg](#string_agg) -- [sum](#sum) -- [var](#var) -- [var_pop](#var_pop) -- [var_population](#var_population) -- [var_samp](#var_samp) -- [var_sample](#var_sample) - -### `array_agg` - -Returns an array created from the expression elements. If ordering is required, elements are inserted in the specified order. - -``` -array_agg(expression [ORDER BY expression]) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT array_agg(column_name ORDER BY other_column) FROM table_name; -+-----------------------------------------------+ -| array_agg(column_name ORDER BY other_column) | -+-----------------------------------------------+ -| [element1, element2, element3] | -+-----------------------------------------------+ -``` - -### `avg` - -Returns the average of numeric values in the specified column. - -``` -avg(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT avg(column_name) FROM table_name; -+---------------------------+ -| avg(column_name) | -+---------------------------+ -| 42.75 | -+---------------------------+ -``` - -#### Aliases - -- mean - -### `bit_and` - -Computes the bitwise AND of all non-null input values. - -``` -bit_and(expression) -``` - -#### Arguments - -- **expression**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `bit_or` - -Computes the bitwise OR of all non-null input values. - -``` -bit_or(expression) -``` - -#### Arguments - -- **expression**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `bit_xor` - -Computes the bitwise exclusive OR of all non-null input values. - -``` -bit_xor(expression) -``` - -#### Arguments - -- **expression**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `bool_and` - -Returns true if all non-null input values are true, otherwise false. - -``` -bool_and(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT bool_and(column_name) FROM table_name; -+----------------------------+ -| bool_and(column_name) | -+----------------------------+ -| true | -+----------------------------+ -``` - -### `bool_or` - -Returns true if all non-null input values are true, otherwise false. - -``` -bool_and(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT bool_and(column_name) FROM table_name; -+----------------------------+ -| bool_and(column_name) | -+----------------------------+ -| true | -+----------------------------+ -``` - -### `count` - -Returns the number of non-null values in the specified column. To include null values in the total count, use `count(*)`. - -``` -count(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT count(column_name) FROM table_name; -+-----------------------+ -| count(column_name) | -+-----------------------+ -| 100 | -+-----------------------+ - -> SELECT count(*) FROM table_name; -+------------------+ -| count(*) | -+------------------+ -| 120 | -+------------------+ -``` - -### `first_value` - -Returns the first element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group. - -``` -first_value(expression [ORDER BY expression]) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT first_value(column_name ORDER BY other_column) FROM table_name; -+-----------------------------------------------+ -| first_value(column_name ORDER BY other_column)| -+-----------------------------------------------+ -| first_element | -+-----------------------------------------------+ -``` - -### `grouping` - -Returns 1 if the data is aggregated across the specified column, or 0 if it is not aggregated in the result set. - -``` -grouping(expression) -``` - -#### Arguments - -- **expression**: Expression to evaluate whether data is aggregated across the specified column. Can be a constant, column, or function. - -#### Example - -```sql -> SELECT column_name, GROUPING(column_name) AS group_column - FROM table_name - GROUP BY GROUPING SETS ((column_name), ()); -+-------------+-------------+ -| column_name | group_column | -+-------------+-------------+ -| value1 | 0 | -| value2 | 0 | -| NULL | 1 | -+-------------+-------------+ -``` - -### `last_value` - -Returns the first element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group. - -``` -first_value(expression [ORDER BY expression]) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT first_value(column_name ORDER BY other_column) FROM table_name; -+-----------------------------------------------+ -| first_value(column_name ORDER BY other_column)| -+-----------------------------------------------+ -| first_element | -+-----------------------------------------------+ -``` - -### `max` - -Returns the maximum value in the specified column. - -``` -max(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT max(column_name) FROM table_name; -+----------------------+ -| max(column_name) | -+----------------------+ -| 150 | -+----------------------+ -``` - -### `mean` - -_Alias of [avg](#avg)._ - -### `median` - -Returns the median value in the specified column. - -``` -median(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT median(column_name) FROM table_name; -+----------------------+ -| median(column_name) | -+----------------------+ -| 45.5 | -+----------------------+ -``` - -### `min` - -Returns the maximum value in the specified column. - -``` -max(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT max(column_name) FROM table_name; -+----------------------+ -| max(column_name) | -+----------------------+ -| 150 | -+----------------------+ -``` - -### `string_agg` - -Concatenates the values of string expressions and places separator values between them. - -``` -string_agg(expression, delimiter) -``` - -#### Arguments - -- **expression**: The string expression to concatenate. Can be a column or any valid string expression. -- **delimiter**: A literal string used as a separator between the concatenated values. - -#### Example - -```sql -> SELECT string_agg(name, ', ') AS names_list - FROM employee; -+--------------------------+ -| names_list | -+--------------------------+ -| Alice, Bob, Charlie | -+--------------------------+ -``` - -### `sum` - -Returns the sum of all values in the specified column. - -``` -sum(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT sum(column_name) FROM table_name; -+-----------------------+ -| sum(column_name) | -+-----------------------+ -| 12345 | -+-----------------------+ -``` - -### `var` - -Returns the statistical sample variance of a set of numbers. - -``` -var(expression) -``` - -#### Arguments - -- **expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Aliases - -- var_sample -- var_samp - -### `var_pop` - -Returns the statistical population variance of a set of numbers. - -``` -var_pop(expression) -``` - -#### Arguments - -- **expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Aliases - -- var_population - -### `var_population` - -_Alias of [var_pop](#var_pop)._ - -### `var_samp` - -_Alias of [var](#var)._ - -### `var_sample` - -_Alias of [var](#var)._ - -## Statistical Functions - -- [corr](#corr) -- [covar](#covar) -- [covar_pop](#covar_pop) -- [covar_samp](#covar_samp) -- [nth_value](#nth_value) -- [regr_avgx](#regr_avgx) -- [regr_avgy](#regr_avgy) -- [regr_count](#regr_count) -- [regr_intercept](#regr_intercept) -- [regr_r2](#regr_r2) -- [regr_slope](#regr_slope) -- [regr_sxx](#regr_sxx) -- [regr_sxy](#regr_sxy) -- [regr_syy](#regr_syy) -- [stddev](#stddev) -- [stddev_pop](#stddev_pop) -- [stddev_samp](#stddev_samp) - -### `corr` - -Returns the coefficient of correlation between two numeric values. - -``` -corr(expression1, expression2) -``` - -#### Arguments - -- **expression1**: First expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression2**: Second expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT corr(column1, column2) FROM table_name; -+--------------------------------+ -| corr(column1, column2) | -+--------------------------------+ -| 0.85 | -+--------------------------------+ -``` - -### `covar` - -_Alias of [covar_samp](#covar_samp)._ - -### `covar_pop` - -Returns the sample covariance of a set of number pairs. - -``` -covar_samp(expression1, expression2) -``` - -#### Arguments - -- **expression1**: First expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression2**: Second expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT covar_samp(column1, column2) FROM table_name; -+-----------------------------------+ -| covar_samp(column1, column2) | -+-----------------------------------+ -| 8.25 | -+-----------------------------------+ -``` - -### `covar_samp` - -Returns the sample covariance of a set of number pairs. - -``` -covar_samp(expression1, expression2) -``` - -#### Arguments - -- **expression1**: First expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression2**: Second expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT covar_samp(column1, column2) FROM table_name; -+-----------------------------------+ -| covar_samp(column1, column2) | -+-----------------------------------+ -| 8.25 | -+-----------------------------------+ -``` - -#### Aliases - -- covar - -### `nth_value` - -Returns the nth value in a group of values. - -``` -nth_value(expression, n ORDER BY expression) -``` - -#### Arguments - -- **expression**: The column or expression to retrieve the nth value from. -- **n**: The position (nth) of the value to retrieve, based on the ordering. - -#### Example - -```sql -> SELECT dept_id, salary, NTH_VALUE(salary, 2) OVER (PARTITION BY dept_id ORDER BY salary ASC) AS second_salary_by_dept - FROM employee; -+---------+--------+-------------------------+ -| dept_id | salary | second_salary_by_dept | -+---------+--------+-------------------------+ -| 1 | 30000 | NULL | -| 1 | 40000 | 40000 | -| 1 | 50000 | 40000 | -| 2 | 35000 | NULL | -| 2 | 45000 | 45000 | -+---------+--------+-------------------------+ -``` - -### `regr_avgx` - -Computes the average of the independent variable (input) expression_x for the non-null paired data points. - -``` -regr_avgx(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_avgy` - -Computes the average of the dependent variable (output) expression_y for the non-null paired data points. - -``` -regr_avgy(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_count` - -Counts the number of non-null paired data points. - -``` -regr_count(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_intercept` - -Computes the y-intercept of the linear regression line. For the equation (y = kx + b), this function returns b. - -``` -regr_intercept(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_r2` - -Computes the square of the correlation coefficient between the independent and dependent variables. - -``` -regr_r2(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_slope` - -Returns the slope of the linear regression line for non-null pairs in aggregate columns. Given input column Y and X: regr_slope(Y, X) returns the slope (k in Y = k\*X + b) using minimal RSS fitting. - -``` -regr_slope(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_sxx` - -Computes the sum of squares of the independent variable. - -``` -regr_sxx(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_sxy` - -Computes the sum of products of paired data points. - -``` -regr_sxy(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `regr_syy` - -Computes the sum of squares of the dependent variable. - -``` -regr_syy(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: Dependent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_x**: Independent variable expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `stddev` - -Returns the standard deviation of a set of numbers. - -``` -stddev(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT stddev(column_name) FROM table_name; -+----------------------+ -| stddev(column_name) | -+----------------------+ -| 12.34 | -+----------------------+ -``` - -#### Aliases - -- stddev_samp - -### `stddev_pop` - -Returns the standard deviation of a set of numbers. - -``` -stddev(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT stddev(column_name) FROM table_name; -+----------------------+ -| stddev(column_name) | -+----------------------+ -| 12.34 | -+----------------------+ -``` - -### `stddev_samp` - -_Alias of [stddev](#stddev)._ - -## Approximate Functions - -- [approx_distinct](#approx_distinct) -- [approx_median](#approx_median) -- [approx_percentile_cont](#approx_percentile_cont) -- [approx_percentile_cont_with_weight](#approx_percentile_cont_with_weight) - -### `approx_distinct` - -Returns the approximate number of distinct input values calculated using the HyperLogLog algorithm. - -``` -approx_distinct(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT approx_distinct(column_name) FROM table_name; -+-----------------------------------+ -| approx_distinct(column_name) | -+-----------------------------------+ -| 42 | -+-----------------------------------+ -``` - -### `approx_median` - -Returns the approximate median (50th percentile) of input values. It is an alias of `approx_percentile_cont(x, 0.5)`. - -``` -approx_median(expression) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> SELECT approx_median(column_name) FROM table_name; -+-----------------------------------+ -| approx_median(column_name) | -+-----------------------------------+ -| 23.5 | -+-----------------------------------+ -``` - -### `approx_percentile_cont` - -Returns the approximate percentile of input values using the t-digest algorithm. - -``` -approx_percentile_cont(expression, percentile, centroids) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **percentile**: Percentile to compute. Must be a float value between 0 and 1 (inclusive). -- **centroids**: Number of centroids to use in the t-digest algorithm. _Default is 100_. A higher number results in more accurate approximation but requires more memory. - -#### Example - -```sql -> SELECT approx_percentile_cont(column_name, 0.75, 100) FROM table_name; -+-------------------------------------------------+ -| approx_percentile_cont(column_name, 0.75, 100) | -+-------------------------------------------------+ -| 65.0 | -+-------------------------------------------------+ -``` - -### `approx_percentile_cont_with_weight` - -Returns the weighted approximate percentile of input values using the t-digest algorithm. - -``` -approx_percentile_cont_with_weight(expression, weight, percentile) -``` - -#### Arguments - -- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **weight**: Expression to use as weight. Can be a constant, column, or function, and any combination of arithmetic operators. -- **percentile**: Percentile to compute. Must be a float value between 0 and 1 (inclusive). - -#### Example - -```sql -> SELECT approx_percentile_cont_with_weight(column_name, weight_column, 0.90) FROM table_name; -+----------------------------------------------------------------------+ -| approx_percentile_cont_with_weight(column_name, weight_column, 0.90) | -+----------------------------------------------------------------------+ -| 78.5 | -+----------------------------------------------------------------------+ -``` diff --git a/docs/source/user-guide/sql/index.rst b/docs/source/user-guide/sql/index.rst index 6eb451c83b964..4499aac536114 100644 --- a/docs/source/user-guide/sql/index.rst +++ b/docs/source/user-guide/sql/index.rst @@ -30,10 +30,9 @@ SQL Reference information_schema operators aggregate_functions - aggregate_functions_new window_functions window_functions_new scalar_functions - scalar_functions_new + special_functions sql_status write_options diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 203411428777e..98c44cbd981d4 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -17,188 +17,4343 @@ under the License. --> + + # Scalar Functions -Scalar functions operate on a single row at a time and return a single value. +## Math Functions + +- [abs](#abs) +- [acos](#acos) +- [acosh](#acosh) +- [asin](#asin) +- [asinh](#asinh) +- [atan](#atan) +- [atan2](#atan2) +- [atanh](#atanh) +- [cbrt](#cbrt) +- [ceil](#ceil) +- [cos](#cos) +- [cosh](#cosh) +- [cot](#cot) +- [degrees](#degrees) +- [exp](#exp) +- [factorial](#factorial) +- [floor](#floor) +- [gcd](#gcd) +- [isnan](#isnan) +- [iszero](#iszero) +- [lcm](#lcm) +- [ln](#ln) +- [log](#log) +- [log10](#log10) +- [log2](#log2) +- [nanvl](#nanvl) +- [pi](#pi) +- [pow](#pow) +- [power](#power) +- [radians](#radians) +- [random](#random) +- [round](#round) +- [signum](#signum) +- [sin](#sin) +- [sinh](#sinh) +- [sqrt](#sqrt) +- [tan](#tan) +- [tanh](#tanh) +- [trunc](#trunc) + +### `abs` + +Returns the absolute value of a number. + +``` +abs(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `acos` + +Returns the arc cosine or inverse cosine of a number. + +``` +acos(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `acosh` + +Returns the area hyperbolic cosine or inverse hyperbolic cosine of a number. + +``` +acosh(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `asin` + +Returns the arc sine or inverse sine of a number. + +``` +asin(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `asinh` + +Returns the area hyperbolic sine or inverse hyperbolic sine of a number. + +``` +asinh(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `atan` + +Returns the arc tangent or inverse tangent of a number. + +``` +atan(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `atan2` + +Returns the arc tangent or inverse tangent of `expression_y / expression_x`. + +``` +atan2(expression_y, expression_x) +``` + +#### Arguments + +- **expression_y**: First numeric expression to operate on. + Can be a constant, column, or function, and any combination of arithmetic operators. +- **expression_x**: Second numeric expression to operate on. + Can be a constant, column, or function, and any combination of arithmetic operators. + +### `atanh` + +Returns the area hyperbolic tangent or inverse hyperbolic tangent of a number. + +``` +atanh(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `cbrt` + +Returns the cube root of a number. + +``` +cbrt(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `ceil` + +Returns the nearest integer greater than or equal to a number. + +``` +ceil(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `cos` + +Returns the cosine of a number. + +``` +cos(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `cosh` + +Returns the hyperbolic cosine of a number. + +``` +cosh(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `cot` + +Returns the cotangent of a number. + +``` +cot(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `degrees` + +Converts radians to degrees. + +``` +degrees(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `exp` + +Returns the base-e exponential of a number. + +``` +exp(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `factorial` + +Factorial. Returns 1 if value is less than 2. + +``` +factorial(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `floor` + +Returns the nearest integer less than or equal to a number. + +``` +floor(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `gcd` + +Returns the greatest common divisor of `expression_x` and `expression_y`. Returns 0 if both inputs are zero. + +``` +gcd(expression_x, expression_y) +``` + +#### Arguments + +- **expression_x**: First numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_y**: Second numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `isnan` + +Returns true if a given number is +NaN or -NaN otherwise returns false. + +``` +isnan(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `iszero` + +Returns true if a given number is +0.0 or -0.0 otherwise returns false. + +``` +iszero(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `lcm` + +Returns the least common multiple of `expression_x` and `expression_y`. Returns 0 if either input is zero. + +``` +lcm(expression_x, expression_y) +``` + +#### Arguments + +- **expression_x**: First numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **expression_y**: Second numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `ln` + +Returns the natural logarithm of a number. + +``` +ln(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `log` + +Returns the base-x logarithm of a number. Can either provide a specified base, or if omitted then takes the base-10 of a number. + +``` +log(base, numeric_expression) +log(numeric_expression) +``` + +#### Arguments + +- **base**: Base numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `log10` + +Returns the base-10 logarithm of a number. + +``` +log10(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `log2` + +Returns the base-2 logarithm of a number. + +``` +log2(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `nanvl` + +Returns the first argument if it's not _NaN_. +Returns the second argument otherwise. + +``` +nanvl(expression_x, expression_y) +``` + +#### Arguments + +- **expression_x**: Numeric expression to return if it's not _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators. +- **expression_y**: Numeric expression to return if the first expression is _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators. + +### `pi` + +Returns an approximate value of π. + +``` +pi() +``` + +### `pow` + +_Alias of [power](#power)._ + +### `power` + +Returns a base expression raised to the power of an exponent. + +``` +power(base, exponent) +``` + +#### Arguments + +- **base**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **exponent**: Exponent numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Aliases + +- pow + +### `radians` + +Converts degrees to radians. + +``` +radians(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `random` + +Returns a random float value in the range [0, 1). +The random seed is unique to each row. + +``` +random() +``` + +### `round` + +Rounds a number to the nearest integer. + +``` +round(numeric_expression[, decimal_places]) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **decimal_places**: Optional. The number of decimal places to round to. Defaults to 0. + +### `signum` + +Returns the sign of a number. +Negative numbers return `-1`. +Zero and positive numbers return `1`. + +``` +signum(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `sin` + +Returns the sine of a number. + +``` +sin(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `sinh` + +Returns the hyperbolic sine of a number. + +``` +sinh(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `sqrt` + +Returns the square root of a number. + +``` +sqrt(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `tan` + +Returns the tangent of a number. + +``` +tan(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `tanh` + +Returns the hyperbolic tangent of a number. + +``` +tanh(numeric_expression) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. + +### `trunc` + +Truncates a number to a whole number or truncated to the specified decimal places. + +``` +trunc(numeric_expression[, decimal_places]) +``` + +#### Arguments + +- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **decimal_places**: Optional. The number of decimal places to + truncate to. Defaults to 0 (truncate to a whole number). If + `decimal_places` is a positive integer, truncates digits to the + right of the decimal point. If `decimal_places` is a negative + integer, replaces digits to the left of the decimal point with `0`. + +## Conditional Functions + +- [coalesce](#coalesce) +- [ifnull](#ifnull) +- [nullif](#nullif) +- [nvl](#nvl) +- [nvl2](#nvl2) + +### `coalesce` + +Returns the first of its arguments that is not _null_. Returns _null_ if all arguments are _null_. This function is often used to substitute a default value for _null_ values. + +``` +coalesce(expression1[, ..., expression_n]) +``` + +#### Arguments + +- **expression1, expression_n**: Expression to use if previous expressions are _null_. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary. + +#### Example + +```sql +> select coalesce(null, null, 'datafusion'); ++----------------------------------------+ +| coalesce(NULL,NULL,Utf8("datafusion")) | ++----------------------------------------+ +| datafusion | ++----------------------------------------+ +``` + +### `ifnull` + +_Alias of [nvl](#nvl)._ + +### `nullif` + +Returns _null_ if _expression1_ equals _expression2_; otherwise it returns _expression1_. +This can be used to perform the inverse operation of [`coalesce`](#coalesce). + +``` +nullif(expression1, expression2) +``` + +#### Arguments + +- **expression1**: Expression to compare and return if equal to expression2. Can be a constant, column, or function, and any combination of operators. +- **expression2**: Expression to compare to expression1. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select nullif('datafusion', 'data'); ++-----------------------------------------+ +| nullif(Utf8("datafusion"),Utf8("data")) | ++-----------------------------------------+ +| datafusion | ++-----------------------------------------+ +> select nullif('datafusion', 'datafusion'); ++-----------------------------------------------+ +| nullif(Utf8("datafusion"),Utf8("datafusion")) | ++-----------------------------------------------+ +| | ++-----------------------------------------------+ +``` + +### `nvl` + +Returns _expression2_ if _expression1_ is NULL otherwise it returns _expression1_. + +``` +nvl(expression1, expression2) +``` + +#### Arguments + +- **expression1**: Expression to return if not null. Can be a constant, column, or function, and any combination of operators. +- **expression2**: Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select nvl(null, 'a'); ++---------------------+ +| nvl(NULL,Utf8("a")) | ++---------------------+ +| a | ++---------------------+\ +> select nvl('b', 'a'); ++--------------------------+ +| nvl(Utf8("b"),Utf8("a")) | ++--------------------------+ +| b | ++--------------------------+ +``` + +#### Aliases + +- ifnull + +### `nvl2` + +Returns _expression2_ if _expression1_ is not NULL; otherwise it returns _expression3_. + +``` +nvl2(expression1, expression2, expression3) +``` + +#### Arguments + +- **expression1**: Expression to test for null. Can be a constant, column, or function, and any combination of operators. +- **expression2**: Expression to return if expr1 is not null. Can be a constant, column, or function, and any combination of operators. +- **expression3**: Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select nvl2(null, 'a', 'b'); ++--------------------------------+ +| nvl2(NULL,Utf8("a"),Utf8("b")) | ++--------------------------------+ +| b | ++--------------------------------+ +> select nvl2('data', 'a', 'b'); ++----------------------------------------+ +| nvl2(Utf8("data"),Utf8("a"),Utf8("b")) | ++----------------------------------------+ +| a | ++----------------------------------------+ +``` + +## String Functions + +- [ascii](#ascii) +- [bit_length](#bit_length) +- [btrim](#btrim) +- [char_length](#char_length) +- [character_length](#character_length) +- [chr](#chr) +- [concat](#concat) +- [concat_ws](#concat_ws) +- [contains](#contains) +- [ends_with](#ends_with) +- [find_in_set](#find_in_set) +- [initcap](#initcap) +- [instr](#instr) +- [left](#left) +- [length](#length) +- [levenshtein](#levenshtein) +- [lower](#lower) +- [lpad](#lpad) +- [ltrim](#ltrim) +- [octet_length](#octet_length) +- [position](#position) +- [repeat](#repeat) +- [replace](#replace) +- [reverse](#reverse) +- [right](#right) +- [rpad](#rpad) +- [rtrim](#rtrim) +- [split_part](#split_part) +- [starts_with](#starts_with) +- [strpos](#strpos) +- [substr](#substr) +- [substr_index](#substr_index) +- [substring](#substring) +- [substring_index](#substring_index) +- [to_hex](#to_hex) +- [translate](#translate) +- [trim](#trim) +- [upper](#upper) +- [uuid](#uuid) + +### `ascii` + +Returns the Unicode character code of the first character in a string. + +``` +ascii(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select ascii('abc'); ++--------------------+ +| ascii(Utf8("abc")) | ++--------------------+ +| 97 | ++--------------------+ +> select ascii('🚀'); ++-------------------+ +| ascii(Utf8("🚀")) | ++-------------------+ +| 128640 | ++-------------------+ +``` + +**Related functions**: + +- [chr](#chr) + +### `bit_length` + +Returns the bit length of a string. + +``` +bit_length(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select bit_length('datafusion'); ++--------------------------------+ +| bit_length(Utf8("datafusion")) | ++--------------------------------+ +| 80 | ++--------------------------------+ +``` + +**Related functions**: + +- [length](#length) +- [octet_length](#octet_length) + +### `btrim` + +Trims the specified trim string from the start and end of a string. If no trim string is provided, all whitespace is removed from the start and end of the input string. + +``` +btrim(str[, trim_str]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **trim_str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. _Default is whitespace characters._ + +#### Example + +```sql +> select btrim('__datafusion____', '_'); ++-------------------------------------------+ +| btrim(Utf8("__datafusion____"),Utf8("_")) | ++-------------------------------------------+ +| datafusion | ++-------------------------------------------+ +``` + +#### Alternative Syntax + +```sql +trim(BOTH trim_str FROM str) +``` + +```sql +trim(trim_str FROM str) +``` + +#### Aliases + +- trim + +**Related functions**: + +- [ltrim](#ltrim) +- [rtrim](#rtrim) + +### `char_length` + +_Alias of [character_length](#character_length)._ + +### `character_length` + +Returns the number of characters in a string. + +``` +character_length(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select character_length('Ångström'); ++------------------------------------+ +| character_length(Utf8("Ångström")) | ++------------------------------------+ +| 8 | ++------------------------------------+ +``` + +#### Aliases + +- length +- char_length + +**Related functions**: + +- [bit_length](#bit_length) +- [octet_length](#octet_length) + +### `chr` + +Returns the character with the specified ASCII or Unicode code value. + +``` +chr(expression) +``` + +#### Arguments + +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select chr(128640); ++--------------------+ +| chr(Int64(128640)) | ++--------------------+ +| 🚀 | ++--------------------+ +``` + +**Related functions**: + +- [ascii](#ascii) + +### `concat` + +Concatenates multiple strings together. + +``` +concat(str[, ..., str_n]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **str_n**: Subsequent string expressions to concatenate. + +#### Example + +```sql +> select concat('data', 'f', 'us', 'ion'); ++-------------------------------------------------------+ +| concat(Utf8("data"),Utf8("f"),Utf8("us"),Utf8("ion")) | ++-------------------------------------------------------+ +| datafusion | ++-------------------------------------------------------+ +``` + +**Related functions**: + +- [concat_ws](#concat_ws) + +### `concat_ws` + +Concatenates multiple strings together with a specified separator. + +``` +concat_ws(separator, str[, ..., str_n]) +``` + +#### Arguments + +- **separator**: Separator to insert between concatenated strings. +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **str_n**: Subsequent string expressions to concatenate. + +#### Example + +```sql +> select concat_ws('_', 'data', 'fusion'); ++--------------------------------------------------+ +| concat_ws(Utf8("_"),Utf8("data"),Utf8("fusion")) | ++--------------------------------------------------+ +| data_fusion | ++--------------------------------------------------+ +``` + +**Related functions**: + +- [concat](#concat) + +### `contains` + +Return true if search_str is found within string (case-sensitive). + +``` +contains(str, search_str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **search_str**: The string to search for in str. + +#### Example + +```sql +> select contains('the quick brown fox', 'row'); ++---------------------------------------------------+ +| contains(Utf8("the quick brown fox"),Utf8("row")) | ++---------------------------------------------------+ +| true | ++---------------------------------------------------+ +``` + +### `ends_with` + +Tests if a string ends with a substring. + +``` +ends_with(str, substr) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **substr**: Substring to test for. + +#### Example + +```sql +> select ends_with('datafusion', 'soin'); ++--------------------------------------------+ +| ends_with(Utf8("datafusion"),Utf8("soin")) | ++--------------------------------------------+ +| false | ++--------------------------------------------+ +> select ends_with('datafusion', 'sion'); ++--------------------------------------------+ +| ends_with(Utf8("datafusion"),Utf8("sion")) | ++--------------------------------------------+ +| true | ++--------------------------------------------+ +``` + +### `find_in_set` + +Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings. + +``` +find_in_set(str, strlist) +``` + +#### Arguments + +- **str**: String expression to find in strlist. +- **strlist**: A string list is a string composed of substrings separated by , characters. + +#### Example + +```sql +> select find_in_set('b', 'a,b,c,d'); ++----------------------------------------+ +| find_in_set(Utf8("b"),Utf8("a,b,c,d")) | ++----------------------------------------+ +| 2 | ++----------------------------------------+ +``` + +### `initcap` + +Capitalizes the first character in each word in the input string. Words are delimited by non-alphanumeric characters. + +``` +initcap(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select initcap('apache datafusion'); ++------------------------------------+ +| initcap(Utf8("apache datafusion")) | ++------------------------------------+ +| Apache Datafusion | ++------------------------------------+ +``` + +**Related functions**: + +- [lower](#lower) +- [upper](#upper) + +### `instr` + +_Alias of [strpos](#strpos)._ + +### `left` + +Returns a specified number of characters from the left side of a string. + +``` +left(str, n) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **n**: Number of characters to return. + +#### Example + +```sql +> select left('datafusion', 4); ++-----------------------------------+ +| left(Utf8("datafusion"),Int64(4)) | ++-----------------------------------+ +| data | ++-----------------------------------+ +``` + +**Related functions**: + +- [right](#right) + +### `length` + +_Alias of [character_length](#character_length)._ + +### `levenshtein` + +Returns the [`Levenshtein distance`](https://en.wikipedia.org/wiki/Levenshtein_distance) between the two given strings. + +``` +levenshtein(str1, str2) +``` + +#### Arguments + +- **str1**: String expression to compute Levenshtein distance with str2. +- **str2**: String expression to compute Levenshtein distance with str1. + +#### Example + +```sql +> select levenshtein('kitten', 'sitting'); ++---------------------------------------------+ +| levenshtein(Utf8("kitten"),Utf8("sitting")) | ++---------------------------------------------+ +| 3 | ++---------------------------------------------+ +``` + +### `lower` + +Converts a string to lower-case. + +``` +lower(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select lower('Ångström'); ++-------------------------+ +| lower(Utf8("Ångström")) | ++-------------------------+ +| ångström | ++-------------------------+ +``` + +**Related functions**: + +- [initcap](#initcap) +- [upper](#upper) + +### `lpad` + +Pads the left side of a string with another string to a specified string length. + +``` +lpad(str, n[, padding_str]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **n**: String length to pad to. +- **padding_str**: Optional string expression to pad with. Can be a constant, column, or function, and any combination of string operators. _Default is a space._ + +#### Example + +```sql +> select lpad('Dolly', 10, 'hello'); ++---------------------------------------------+ +| lpad(Utf8("Dolly"),Int64(10),Utf8("hello")) | ++---------------------------------------------+ +| helloDolly | ++---------------------------------------------+ +``` + +**Related functions**: + +- [rpad](#rpad) + +### `ltrim` + +Trims the specified trim string from the beginning of a string. If no trim string is provided, all whitespace is removed from the start of the input string. + +``` +ltrim(str[, trim_str]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **trim_str**: String expression to trim from the beginning of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._ + +#### Example + +```sql +> select ltrim(' datafusion '); ++-------------------------------+ +| ltrim(Utf8(" datafusion ")) | ++-------------------------------+ +| datafusion | ++-------------------------------+ +> select ltrim('___datafusion___', '_'); ++-------------------------------------------+ +| ltrim(Utf8("___datafusion___"),Utf8("_")) | ++-------------------------------------------+ +| datafusion___ | ++-------------------------------------------+ +``` + +#### Alternative Syntax + +```sql +trim(LEADING trim_str FROM str) +``` + +**Related functions**: + +- [btrim](#btrim) +- [rtrim](#rtrim) + +### `octet_length` + +Returns the length of a string in bytes. + +``` +octet_length(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select octet_length('Ångström'); ++--------------------------------+ +| octet_length(Utf8("Ångström")) | ++--------------------------------+ +| 10 | ++--------------------------------+ +``` + +**Related functions**: + +- [bit_length](#bit_length) +- [length](#length) + +### `position` + +_Alias of [strpos](#strpos)._ + +### `repeat` + +Returns a string with an input string repeated a specified number. + +``` +repeat(str, n) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **n**: Number of times to repeat the input string. + +#### Example + +```sql +> select repeat('data', 3); ++-------------------------------+ +| repeat(Utf8("data"),Int64(3)) | ++-------------------------------+ +| datadatadata | ++-------------------------------+ +``` + +### `replace` + +Replaces all occurrences of a specified substring in a string with a new substring. + +``` +replace(str, substr, replacement) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **substr**: Substring expression to replace in the input string. Substring expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **replacement**: Replacement substring expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select replace('ABabbaBA', 'ab', 'cd'); ++-------------------------------------------------+ +| replace(Utf8("ABabbaBA"),Utf8("ab"),Utf8("cd")) | ++-------------------------------------------------+ +| ABcdbaBA | ++-------------------------------------------------+ +``` + +### `reverse` + +Reverses the character order of a string. + +``` +reverse(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select reverse('datafusion'); ++-----------------------------+ +| reverse(Utf8("datafusion")) | ++-----------------------------+ +| noisufatad | ++-----------------------------+ +``` + +### `right` + +Returns a specified number of characters from the right side of a string. + +``` +right(str, n) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **n**: Number of characters to return + +#### Example + +```sql +> select right('datafusion', 6); ++------------------------------------+ +| right(Utf8("datafusion"),Int64(6)) | ++------------------------------------+ +| fusion | ++------------------------------------+ +``` + +**Related functions**: + +- [left](#left) + +### `rpad` + +Pads the right side of a string with another string to a specified string length. + +``` +rpad(str, n[, padding_str]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **n**: String length to pad to. +- **padding_str**: String expression to pad with. Can be a constant, column, or function, and any combination of string operators. _Default is a space._ + +#### Example + +```sql +> select rpad('datafusion', 20, '_-'); ++-----------------------------------------------+ +| rpad(Utf8("datafusion"),Int64(20),Utf8("_-")) | ++-----------------------------------------------+ +| datafusion_-_-_-_-_- | ++-----------------------------------------------+ +``` + +**Related functions**: + +- [lpad](#lpad) + +### `rtrim` + +Trims the specified trim string from the end of a string. If no trim string is provided, all whitespace is removed from the end of the input string. + +``` +rtrim(str[, trim_str]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **trim_str**: String expression to trim from the end of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._ + +#### Example + +```sql +> select rtrim(' datafusion '); ++-------------------------------+ +| rtrim(Utf8(" datafusion ")) | ++-------------------------------+ +| datafusion | ++-------------------------------+ +> select rtrim('___datafusion___', '_'); ++-------------------------------------------+ +| rtrim(Utf8("___datafusion___"),Utf8("_")) | ++-------------------------------------------+ +| ___datafusion | ++-------------------------------------------+ +``` + +#### Alternative Syntax + +```sql +trim(TRAILING trim_str FROM str) +``` + +**Related functions**: + +- [btrim](#btrim) +- [ltrim](#ltrim) + +### `split_part` + +Splits a string based on a specified delimiter and returns the substring in the specified position. + +``` +split_part(str, delimiter, pos) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **delimiter**: String or character to split on. +- **pos**: Position of the part to return. + +#### Example + +```sql +> select split_part('1.2.3.4.5', '.', 3); ++--------------------------------------------------+ +| split_part(Utf8("1.2.3.4.5"),Utf8("."),Int64(3)) | ++--------------------------------------------------+ +| 3 | ++--------------------------------------------------+ +``` + +### `starts_with` + +Tests if a string starts with a substring. + +``` +starts_with(str, substr) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **substr**: Substring to test for. + +#### Example + +```sql +> select starts_with('datafusion','data'); ++----------------------------------------------+ +| starts_with(Utf8("datafusion"),Utf8("data")) | ++----------------------------------------------+ +| true | ++----------------------------------------------+ +``` + +### `strpos` + +Returns the starting position of a specified substring in a string. Positions begin at 1. If the substring does not exist in the string, the function returns 0. + +``` +strpos(str, substr) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **substr**: Substring expression to search for. + +#### Example + +```sql +> select strpos('datafusion', 'fus'); ++----------------------------------------+ +| strpos(Utf8("datafusion"),Utf8("fus")) | ++----------------------------------------+ +| 5 | ++----------------------------------------+ +``` + +#### Alternative Syntax + +```sql +position(substr in origstr) +``` + +#### Aliases + +- instr +- position + +### `substr` + +Extracts a substring of a specified number of characters from a specific starting position in a string. + +``` +substr(str, start_pos[, length]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **start_pos**: Character position to start the substring at. The first character in the string has a position of 1. +- **length**: Number of characters to extract. If not specified, returns the rest of the string after the start position. + +#### Example + +```sql +> select substr('datafusion', 5, 3); ++----------------------------------------------+ +| substr(Utf8("datafusion"),Int64(5),Int64(3)) | ++----------------------------------------------+ +| fus | ++----------------------------------------------+ +``` + +#### Alternative Syntax + +```sql +substring(str from start_pos for length) +``` + +#### Aliases + +- substring + +### `substr_index` + +Returns the substring from str before count occurrences of the delimiter delim. +If count is positive, everything to the left of the final delimiter (counting from the left) is returned. +If count is negative, everything to the right of the final delimiter (counting from the right) is returned. + +``` +substr_index(str, delim, count) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **delim**: The string to find in str to split str. +- **count**: The number of times to search for the delimiter. Can be either a positive or negative number. + +#### Example + +```sql +> select substr_index('www.apache.org', '.', 1); ++---------------------------------------------------------+ +| substr_index(Utf8("www.apache.org"),Utf8("."),Int64(1)) | ++---------------------------------------------------------+ +| www | ++---------------------------------------------------------+ +> select substr_index('www.apache.org', '.', -1); ++----------------------------------------------------------+ +| substr_index(Utf8("www.apache.org"),Utf8("."),Int64(-1)) | ++----------------------------------------------------------+ +| org | ++----------------------------------------------------------+ +``` + +#### Aliases + +- substring_index + +### `substring` + +_Alias of [substr](#substr)._ + +### `substring_index` + +_Alias of [substr_index](#substr_index)._ + +### `to_hex` + +Converts an integer to a hexadecimal string. + +``` +to_hex(int) +``` + +#### Arguments + +- **int**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select to_hex(12345689); ++-------------------------+ +| to_hex(Int64(12345689)) | ++-------------------------+ +| bc6159 | ++-------------------------+ +``` + +### `translate` + +Translates characters in a string to specified translation characters. + +``` +translate(str, chars, translation) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **chars**: Characters to translate. +- **translation**: Translation characters. Translation characters replace only characters at the same position in the **chars** string. + +#### Example + +```sql +> select translate('twice', 'wic', 'her'); ++--------------------------------------------------+ +| translate(Utf8("twice"),Utf8("wic"),Utf8("her")) | ++--------------------------------------------------+ +| there | ++--------------------------------------------------+ +``` + +### `trim` + +_Alias of [btrim](#btrim)._ + +### `upper` + +Converts a string to upper-case. + +``` +upper(str) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select upper('dataFusion'); ++---------------------------+ +| upper(Utf8("dataFusion")) | ++---------------------------+ +| DATAFUSION | ++---------------------------+ +``` + +**Related functions**: + +- [initcap](#initcap) +- [lower](#lower) + +### `uuid` + +Returns [`UUID v4`]() string value which is unique per row. + +``` +uuid() +``` + +#### Example + +```sql +> select uuid(); ++--------------------------------------+ +| uuid() | ++--------------------------------------+ +| 6ec17ef8-1934-41cc-8d59-d0c8f9eea1f0 | ++--------------------------------------+ +``` + +## Binary String Functions + +- [decode](#decode) +- [encode](#encode) + +### `decode` + +Decode binary data from textual representation in string. + +``` +decode(expression, format) +``` + +#### Arguments + +- **expression**: Expression containing encoded string data +- **format**: Same arguments as [encode](#encode) + +**Related functions**: + +- [encode](#encode) + +### `encode` + +Encode binary data into a textual representation. + +``` +encode(expression, format) +``` + +#### Arguments + +- **expression**: Expression containing string or binary data +- **format**: Supported formats are: `base64`, `hex` + +**Related functions**: + +- [decode](#decode) + +## Regular Expression Functions + +Apache DataFusion uses a [PCRE-like](https://en.wikibooks.org/wiki/Regular_Expressions/Perl-Compatible_Regular_Expressions) +regular expression [syntax](https://docs.rs/regex/latest/regex/#syntax) +(minus support for several features including look-around and backreferences). +The following regular expression functions are supported: + +- [regexp_count](#regexp_count) +- [regexp_like](#regexp_like) +- [regexp_match](#regexp_match) +- [regexp_replace](#regexp_replace) + +### `regexp_count` + +Returns the number of matches that a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has in a string. + +``` +regexp_count(str, regexp[, start, flags]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **regexp**: Regular expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **start**: - **start**: Optional start position (the first position is 1) to search for the regular expression. Can be a constant, column, or function. +- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: + - **i**: case-insensitive: letters match both upper and lower case + - **m**: multi-line mode: ^ and $ match begin/end of line + - **s**: allow . to match \n + - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used + - **U**: swap the meaning of x* and x*? + +#### Example + +```sql +> select regexp_count('abcAbAbc', 'abc', 2, 'i'); ++---------------------------------------------------------------+ +| regexp_count(Utf8("abcAbAbc"),Utf8("abc"),Int64(2),Utf8("i")) | ++---------------------------------------------------------------+ +| 1 | ++---------------------------------------------------------------+ +``` + +### `regexp_like` + +Returns true if a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has at least one match in a string, false otherwise. + +``` +regexp_like(str, regexp[, flags]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **regexp**: Regular expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: + - **i**: case-insensitive: letters match both upper and lower case + - **m**: multi-line mode: ^ and $ match begin/end of line + - **s**: allow . to match \n + - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used + - **U**: swap the meaning of x* and x*? + +#### Example + +```sql +select regexp_like('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); ++--------------------------------------------------------+ +| regexp_like(Utf8("Köln"),Utf8("[a-zA-Z]ö[a-zA-Z]{2}")) | ++--------------------------------------------------------+ +| true | ++--------------------------------------------------------+ +SELECT regexp_like('aBc', '(b|d)', 'i'); ++--------------------------------------------------+ +| regexp_like(Utf8("aBc"),Utf8("(b|d)"),Utf8("i")) | ++--------------------------------------------------+ +| true | ++--------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs) + +### `regexp_match` + +Returns the first [regular expression](https://docs.rs/regex/latest/regex/#syntax) matches in a string. + +``` +regexp_match(str, regexp[, flags]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **regexp**: Regular expression to match against. + Can be a constant, column, or function. +- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: + - **i**: case-insensitive: letters match both upper and lower case + - **m**: multi-line mode: ^ and $ match begin/end of line + - **s**: allow . to match \n + - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used + - **U**: swap the meaning of x* and x*? + +#### Example + +```sql + > select regexp_match('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); + +---------------------------------------------------------+ + | regexp_match(Utf8("Köln"),Utf8("[a-zA-Z]ö[a-zA-Z]{2}")) | + +---------------------------------------------------------+ + | [Köln] | + +---------------------------------------------------------+ + SELECT regexp_match('aBc', '(b|d)', 'i'); + +---------------------------------------------------+ + | regexp_match(Utf8("aBc"),Utf8("(b|d)"),Utf8("i")) | + +---------------------------------------------------+ + | [B] | + +---------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs) + +### `regexp_replace` + +Replaces substrings in a string that match a [regular expression](https://docs.rs/regex/latest/regex/#syntax). + +``` +regexp_replace(str, regexp, replacement[, flags]) +``` + +#### Arguments + +- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **regexp**: Regular expression to match against. + Can be a constant, column, or function. +- **replacement**: Replacement string expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: +- **g**: (global) Search globally and don't return after the first match +- **i**: case-insensitive: letters match both upper and lower case +- **m**: multi-line mode: ^ and $ match begin/end of line +- **s**: allow . to match \n +- **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used +- **U**: swap the meaning of x* and x*? + +#### Example + +```sql +> select regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g'); ++------------------------------------------------------------------------+ +| regexp_replace(Utf8("foobarbaz"),Utf8("b(..)"),Utf8("X\1Y"),Utf8("g")) | ++------------------------------------------------------------------------+ +| fooXarYXazY | ++------------------------------------------------------------------------+ +SELECT regexp_replace('aBc', '(b|d)', 'Ab\\1a', 'i'); ++-------------------------------------------------------------------+ +| regexp_replace(Utf8("aBc"),Utf8("(b|d)"),Utf8("Ab\1a"),Utf8("i")) | ++-------------------------------------------------------------------+ +| aAbBac | ++-------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs) + +## Time and Date Functions + +- [current_date](#current_date) +- [current_time](#current_time) +- [current_timestamp](#current_timestamp) +- [date_bin](#date_bin) +- [date_format](#date_format) +- [date_part](#date_part) +- [date_trunc](#date_trunc) +- [datepart](#datepart) +- [datetrunc](#datetrunc) +- [from_unixtime](#from_unixtime) +- [make_date](#make_date) +- [now](#now) +- [to_char](#to_char) +- [to_date](#to_date) +- [to_local_time](#to_local_time) +- [to_timestamp](#to_timestamp) +- [to_timestamp_micros](#to_timestamp_micros) +- [to_timestamp_millis](#to_timestamp_millis) +- [to_timestamp_nanos](#to_timestamp_nanos) +- [to_timestamp_seconds](#to_timestamp_seconds) +- [to_unixtime](#to_unixtime) +- [today](#today) + +### `current_date` + +Returns the current UTC date. + +The `current_date()` return value is determined at query time and will return the same date, no matter when in the query plan the function executes. + +``` +current_date() +``` + +#### Aliases + +- today + +### `current_time` + +Returns the current UTC time. + +The `current_time()` return value is determined at query time and will return the same time, no matter when in the query plan the function executes. + +``` +current_time() +``` + +### `current_timestamp` + +_Alias of [now](#now)._ + +### `date_bin` + +Calculates time intervals and returns the start of the interval nearest to the specified timestamp. Use `date_bin` to downsample time series data by grouping rows into time-based "bins" or "windows" and applying an aggregate or selector function to each window. + +For example, if you "bin" or "window" data into 15 minute intervals, an input timestamp of `2023-01-01T18:18:18Z` will be updated to the start time of the 15 minute bin it is in: `2023-01-01T18:15:00Z`. + +``` +date_bin(interval, expression, origin-timestamp) +``` + +#### Arguments + +- **interval**: Bin interval. +- **expression**: Time expression to operate on. Can be a constant, column, or function. +- **origin-timestamp**: Optional. Starting point used to determine bin boundaries. If not specified defaults 1970-01-01T00:00:00Z (the UNIX epoch in UTC). + +The following intervals are supported: + +- nanoseconds +- microseconds +- milliseconds +- seconds +- minutes +- hours +- days +- weeks +- months +- years +- century + +### `date_format` + +_Alias of [to_char](#to_char)._ + +### `date_part` + +Returns the specified part of the date as an integer. + +``` +date_part(part, expression) +``` + +#### Arguments + +- **part**: Part of the date to return. The following date parts are supported: + + - year + - quarter (emits value in inclusive range [1, 4] based on which quartile of the year the date is in) + - month + - week (week of the year) + - day (day of the month) + - hour + - minute + - second + - millisecond + - microsecond + - nanosecond + - dow (day of the week) + - doy (day of the year) + - epoch (seconds since Unix epoch) + +- **expression**: Time expression to operate on. Can be a constant, column, or function. + +#### Alternative Syntax + +```sql +extract(field FROM source) +``` + +#### Aliases + +- datepart + +### `date_trunc` + +Truncates a timestamp value to a specified precision. + +``` +date_trunc(precision, expression) +``` + +#### Arguments + +- **precision**: Time precision to truncate to. The following precisions are supported: + + - year / YEAR + - quarter / QUARTER + - month / MONTH + - week / WEEK + - day / DAY + - hour / HOUR + - minute / MINUTE + - second / SECOND + +- **expression**: Time expression to operate on. Can be a constant, column, or function. + +#### Aliases + +- datetrunc + +### `datepart` + +_Alias of [date_part](#date_part)._ + +### `datetrunc` + +_Alias of [date_trunc](#date_trunc)._ + +### `from_unixtime` + +Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp. + +``` +from_unixtime(expression) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. + +### `make_date` + +Make a date from year/month/day component parts. + +``` +make_date(year, month, day) +``` + +#### Arguments + +- **year**: Year to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. +- **month**: Month to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. +- **day**: Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. + +#### Example + +```sql +> select make_date(2023, 1, 31); ++-------------------------------------------+ +| make_date(Int64(2023),Int64(1),Int64(31)) | ++-------------------------------------------+ +| 2023-01-31 | ++-------------------------------------------+ +> select make_date('2023', '01', '31'); ++-----------------------------------------------+ +| make_date(Utf8("2023"),Utf8("01"),Utf8("31")) | ++-----------------------------------------------+ +| 2023-01-31 | ++-----------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/make_date.rs) + +### `now` + +Returns the current UTC timestamp. + +The `now()` return value is determined at query time and will return the same timestamp, no matter when in the query plan the function executes. + +``` +now() +``` + +#### Aliases + +- current_timestamp + +### `to_char` + +Returns a string representation of a date, time, timestamp or duration based on a [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html). Unlike the PostgreSQL equivalent of this function numerical formatting is not supported. + +``` +to_char(expression, format) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function that results in a date, time, timestamp or duration. +- **format**: A [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) string to use to convert the expression. +- **day**: Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. + +#### Example + +```sql +> select to_char('2023-03-01'::date, '%d-%m-%Y'); ++----------------------------------------------+ +| to_char(Utf8("2023-03-01"),Utf8("%d-%m-%Y")) | ++----------------------------------------------+ +| 01-03-2023 | ++----------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_char.rs) + +#### Aliases + +- date_format + +### `to_date` + +Converts a value to a date (`YYYY-MM-DD`). +Supports strings, integer and double types as input. +Strings are parsed as YYYY-MM-DD (e.g. '2023-07-20') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. +Integers and doubles are interpreted as days since the unix epoch (`1970-01-01T00:00:00Z`). +Returns the corresponding date. + +Note: `to_date` returns Date32, which represents its values as the number of days since unix epoch(`1970-01-01`) stored as signed 32 bit value. The largest supported date value is `9999-12-31`. + +``` +to_date('2017-05-31', '%Y-%m-%d') +``` + +#### Arguments + +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order + they appear with the first successful one being returned. If none of the formats successfully parse the expression + an error will be returned. + +#### Example + +```sql +> select to_date('2023-01-31'); ++-----------------------------+ +| to_date(Utf8("2023-01-31")) | ++-----------------------------+ +| 2023-01-31 | ++-----------------------------+ +> select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d'); ++---------------------------------------------------------------+ +| to_date(Utf8("2023/01/31"),Utf8("%Y-%m-%d"),Utf8("%Y/%m/%d")) | ++---------------------------------------------------------------+ +| 2023-01-31 | ++---------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs) + +### `to_local_time` + +Converts a timestamp with a timezone to a timestamp without a timezone (with no offset or timezone information). This function handles daylight saving time changes. + +``` +to_local_time(expression) +``` + +#### Arguments + +- **expression**: Time expression to operate on. Can be a constant, column, or function. + +#### Example + +```sql +> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp); ++---------------------------------------------+ +| to_local_time(Utf8("2024-04-01T00:00:20Z")) | ++---------------------------------------------+ +| 2024-04-01T00:00:20 | ++---------------------------------------------+ + +> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels'); ++---------------------------------------------+ +| to_local_time(Utf8("2024-04-01T00:00:20Z")) | ++---------------------------------------------+ +| 2024-04-01T00:00:20 | ++---------------------------------------------+ + +> SELECT + time, + arrow_typeof(time) as type, + to_local_time(time) as to_local_time, + arrow_typeof(to_local_time(time)) as to_local_time_type +FROM ( + SELECT '2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels' AS time +); ++---------------------------+------------------------------------------------+---------------------+-----------------------------+ +| time | type | to_local_time | to_local_time_type | ++---------------------------+------------------------------------------------+---------------------+-----------------------------+ +| 2024-04-01T00:00:20+02:00 | Timestamp(Nanosecond, Some("Europe/Brussels")) | 2024-04-01T00:00:20 | Timestamp(Nanosecond, None) | ++---------------------------+------------------------------------------------+---------------------+-----------------------------+ + +# combine `to_local_time()` with `date_bin()` to bin on boundaries in the timezone rather +# than UTC boundaries + +> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AS date_bin; ++---------------------+ +| date_bin | ++---------------------+ +| 2024-04-01T00:00:00 | ++---------------------+ + +> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AT TIME ZONE 'Europe/Brussels' AS date_bin_with_timezone; ++---------------------------+ +| date_bin_with_timezone | ++---------------------------+ +| 2024-04-01T00:00:00+02:00 | ++---------------------------+ +``` + +### `to_timestamp` + +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). Supports strings, integer, unsigned integer, and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats] are provided. Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. + +Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for integer input is between `-9223372037` and `9223372036`. Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. Please use `to_timestamp_seconds` for the input outside of supported bounds. + +``` +to_timestamp(expression[, ..., format_n]) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. + +#### Example + +```sql +> select to_timestamp('2023-01-31T09:26:56.123456789-05:00'); ++-----------------------------------------------------------+ +| to_timestamp(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++-----------------------------------------------------------+ +| 2023-01-31T14:26:56.123456789 | ++-----------------------------------------------------------+ +> select to_timestamp('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++--------------------------------------------------------------------------------------------------------+ +| to_timestamp(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++--------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123456789 | ++--------------------------------------------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) + +### `to_timestamp_micros` + +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) Returns the corresponding timestamp. + +``` +to_timestamp_micros(expression[, ..., format_n]) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. + +#### Example + +```sql +> select to_timestamp_micros('2023-01-31T09:26:56.123456789-05:00'); ++------------------------------------------------------------------+ +| to_timestamp_micros(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++------------------------------------------------------------------+ +| 2023-01-31T14:26:56.123456 | ++------------------------------------------------------------------+ +> select to_timestamp_micros('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++---------------------------------------------------------------------------------------------------------------+ +| to_timestamp_micros(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++---------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123456 | ++---------------------------------------------------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) + +### `to_timestamp_millis` + +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. + +``` +to_timestamp_millis(expression[, ..., format_n]) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. + +#### Example + +```sql +> select to_timestamp_millis('2023-01-31T09:26:56.123456789-05:00'); ++------------------------------------------------------------------+ +| to_timestamp_millis(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++------------------------------------------------------------------+ +| 2023-01-31T14:26:56.123 | ++------------------------------------------------------------------+ +> select to_timestamp_millis('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++---------------------------------------------------------------------------------------------------------------+ +| to_timestamp_millis(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++---------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123 | ++---------------------------------------------------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) + +### `to_timestamp_nanos` + +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. + +``` +to_timestamp_nanos(expression[, ..., format_n]) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. + +#### Example + +```sql +> select to_timestamp_nanos('2023-01-31T09:26:56.123456789-05:00'); ++-----------------------------------------------------------------+ +| to_timestamp_nanos(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++-----------------------------------------------------------------+ +| 2023-01-31T14:26:56.123456789 | ++-----------------------------------------------------------------+ +> select to_timestamp_nanos('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++--------------------------------------------------------------------------------------------------------------+ +| to_timestamp_nanos(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++--------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123456789 | ++---------------------------------------------------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) + +### `to_timestamp_seconds` + +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. + +``` +to_timestamp_seconds(expression[, ..., format_n]) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. + +#### Example + +```sql +> select to_timestamp_seconds('2023-01-31T09:26:56.123456789-05:00'); ++-------------------------------------------------------------------+ +| to_timestamp_seconds(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++-------------------------------------------------------------------+ +| 2023-01-31T14:26:56 | ++-------------------------------------------------------------------+ +> select to_timestamp_seconds('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++----------------------------------------------------------------------------------------------------------------+ +| to_timestamp_seconds(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++----------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00 | ++----------------------------------------------------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) + +### `to_unixtime` + +Converts a value to seconds since the unix epoch (`1970-01-01T00:00:00Z`). Supports strings, dates, timestamps and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. + +``` +to_unixtime(expression[, ..., format_n]) +``` + +#### Arguments + +- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. + +#### Example + +```sql +> select to_unixtime('2020-09-08T12:00:00+00:00'); ++------------------------------------------------+ +| to_unixtime(Utf8("2020-09-08T12:00:00+00:00")) | ++------------------------------------------------+ +| 1599566400 | ++------------------------------------------------+ +> select to_unixtime('01-14-2023 01:01:30+05:30', '%q', '%d-%m-%Y %H/%M/%S', '%+', '%m-%d-%Y %H:%M:%S%#z'); ++-----------------------------------------------------------------------------------------------------------------------------+ +| to_unixtime(Utf8("01-14-2023 01:01:30+05:30"),Utf8("%q"),Utf8("%d-%m-%Y %H/%M/%S"),Utf8("%+"),Utf8("%m-%d-%Y %H:%M:%S%#z")) | ++-----------------------------------------------------------------------------------------------------------------------------+ +| 1673638290 | ++-----------------------------------------------------------------------------------------------------------------------------+ +``` + +### `today` + +_Alias of [current_date](#current_date)._ + +## Array Functions + +- [array_any_value](#array_any_value) +- [array_append](#array_append) +- [array_cat](#array_cat) +- [array_concat](#array_concat) +- [array_contains](#array_contains) +- [array_dims](#array_dims) +- [array_distance](#array_distance) +- [array_distinct](#array_distinct) +- [array_element](#array_element) +- [array_empty](#array_empty) +- [array_except](#array_except) +- [array_extract](#array_extract) +- [array_has](#array_has) +- [array_has_all](#array_has_all) +- [array_has_any](#array_has_any) +- [array_indexof](#array_indexof) +- [array_intersect](#array_intersect) +- [array_join](#array_join) +- [array_length](#array_length) +- [array_ndims](#array_ndims) +- [array_pop_back](#array_pop_back) +- [array_pop_front](#array_pop_front) +- [array_position](#array_position) +- [array_positions](#array_positions) +- [array_prepend](#array_prepend) +- [array_push_back](#array_push_back) +- [array_push_front](#array_push_front) +- [array_remove](#array_remove) +- [array_remove_all](#array_remove_all) +- [array_remove_n](#array_remove_n) +- [array_repeat](#array_repeat) +- [array_replace](#array_replace) +- [array_replace_all](#array_replace_all) +- [array_replace_n](#array_replace_n) +- [array_resize](#array_resize) +- [array_reverse](#array_reverse) +- [array_slice](#array_slice) +- [array_sort](#array_sort) +- [array_to_string](#array_to_string) +- [array_union](#array_union) +- [cardinality](#cardinality) +- [empty](#empty) +- [flatten](#flatten) +- [generate_series](#generate_series) +- [list_any_value](#list_any_value) +- [list_append](#list_append) +- [list_cat](#list_cat) +- [list_concat](#list_concat) +- [list_contains](#list_contains) +- [list_dims](#list_dims) +- [list_distance](#list_distance) +- [list_distinct](#list_distinct) +- [list_element](#list_element) +- [list_empty](#list_empty) +- [list_except](#list_except) +- [list_extract](#list_extract) +- [list_has](#list_has) +- [list_has_all](#list_has_all) +- [list_has_any](#list_has_any) +- [list_indexof](#list_indexof) +- [list_intersect](#list_intersect) +- [list_join](#list_join) +- [list_length](#list_length) +- [list_ndims](#list_ndims) +- [list_pop_back](#list_pop_back) +- [list_pop_front](#list_pop_front) +- [list_position](#list_position) +- [list_positions](#list_positions) +- [list_prepend](#list_prepend) +- [list_push_back](#list_push_back) +- [list_push_front](#list_push_front) +- [list_remove](#list_remove) +- [list_remove_all](#list_remove_all) +- [list_remove_n](#list_remove_n) +- [list_repeat](#list_repeat) +- [list_replace](#list_replace) +- [list_replace_all](#list_replace_all) +- [list_replace_n](#list_replace_n) +- [list_resize](#list_resize) +- [list_reverse](#list_reverse) +- [list_slice](#list_slice) +- [list_sort](#list_sort) +- [list_to_string](#list_to_string) +- [list_union](#list_union) +- [make_array](#make_array) +- [make_list](#make_list) +- [range](#range) +- [string_to_array](#string_to_array) +- [string_to_list](#string_to_list) + +### `array_any_value` + +Extracts the element with the index n from the array. + +``` +array_element(array, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **index**: Index to extract the element from the array. + +#### Example + +```sql +> select array_element([1, 2, 3, 4], 3); ++-----------------------------------------+ +| array_element(List([1,2,3,4]),Int64(3)) | ++-----------------------------------------+ +| 3 | ++-----------------------------------------+ +``` + +#### Aliases + +- list_any_value + +### `array_append` + +Appends an element to the end of an array. + +``` +array_append(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to append to the array. + +#### Example + +```sql +> select array_append([1, 2, 3], 4); ++--------------------------------------+ +| array_append(List([1,2,3]),Int64(4)) | ++--------------------------------------+ +| [1, 2, 3, 4] | ++--------------------------------------+ +``` + +#### Aliases + +- list_append +- array_push_back +- list_push_back + +### `array_cat` + +_Alias of [array_concat](#array_concat)._ + +### `array_concat` + +Appends an element to the end of an array. + +``` +array_append(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to append to the array. + +#### Example + +```sql +> select array_append([1, 2, 3], 4); ++--------------------------------------+ +| array_append(List([1,2,3]),Int64(4)) | ++--------------------------------------+ +| [1, 2, 3, 4] | ++--------------------------------------+ +``` + +#### Aliases + +- array_cat +- list_concat +- list_cat + +### `array_contains` + +_Alias of [array_has](#array_has)._ + +### `array_dims` + +Returns an array of the array's dimensions. + +``` +array_dims(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_dims([[1, 2, 3], [4, 5, 6]]); ++---------------------------------+ +| array_dims(List([1,2,3,4,5,6])) | ++---------------------------------+ +| [2, 3] | ++---------------------------------+ +``` + +#### Aliases + +- list_dims + +### `array_distance` + +Returns the Euclidean distance between two input arrays of equal length. + +``` +array_distance(array1, array2) +``` + +#### Arguments + +- **array1**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **array2**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_distance([1, 2], [1, 4]); ++------------------------------------+ +| array_distance(List([1,2], [1,4])) | ++------------------------------------+ +| 2.0 | ++------------------------------------+ +``` + +#### Aliases + +- list_distance + +### `array_distinct` + +Returns distinct values from the array after removing duplicates. + +``` +array_distinct(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_distinct([1, 3, 2, 3, 1, 2, 4]); ++---------------------------------+ +| array_distinct(List([1,2,3,4])) | ++---------------------------------+ +| [1, 2, 3, 4] | ++---------------------------------+ +``` + +#### Aliases + +- list_distinct + +### `array_element` + +Extracts the element with the index n from the array. + +``` +array_element(array, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **index**: Index to extract the element from the array. + +#### Example + +```sql +> select array_element([1, 2, 3, 4], 3); ++-----------------------------------------+ +| array_element(List([1,2,3,4]),Int64(3)) | ++-----------------------------------------+ +| 3 | ++-----------------------------------------+ +``` + +#### Aliases + +- array_extract +- list_element +- list_extract + +### `array_empty` + +_Alias of [empty](#empty)._ + +### `array_except` + +Returns an array of the elements that appear in the first array but not in the second. + +``` +array_except(array1, array2) +``` + +#### Arguments + +- **array1**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **array2**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_except([1, 2, 3, 4], [5, 6, 3, 4]); ++----------------------------------------------------+ +| array_except([1, 2, 3, 4], [5, 6, 3, 4]); | ++----------------------------------------------------+ +| [1, 2] | ++----------------------------------------------------+ +> select array_except([1, 2, 3, 4], [3, 4, 5, 6]); ++----------------------------------------------------+ +| array_except([1, 2, 3, 4], [3, 4, 5, 6]); | ++----------------------------------------------------+ +| [1, 2] | ++----------------------------------------------------+ +``` + +#### Aliases + +- list_except + +### `array_extract` + +_Alias of [array_element](#array_element)._ + +### `array_has` + +Returns true if the array contains the element. + +``` +array_has(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_has([1, 2, 3], 2); ++-----------------------------+ +| array_has(List([1,2,3]), 2) | ++-----------------------------+ +| true | ++-----------------------------+ +``` + +#### Aliases + +- list_has +- array_contains +- list_contains + +### `array_has_all` + +Returns true if the array contains the element. + +``` +array_has(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_has([1, 2, 3], 2); ++-----------------------------+ +| array_has(List([1,2,3]), 2) | ++-----------------------------+ +| true | ++-----------------------------+ +``` + +#### Aliases + +- list_has_all + +### `array_has_any` + +Returns true if the array contains the element. + +``` +array_has(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_has([1, 2, 3], 2); ++-----------------------------+ +| array_has(List([1,2,3]), 2) | ++-----------------------------+ +| true | ++-----------------------------+ +``` + +#### Aliases + +- list_has_any + +### `array_indexof` + +_Alias of [array_position](#array_position)._ + +### `array_intersect` + +Returns distinct values from the array after removing duplicates. + +``` +array_distinct(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_distinct([1, 3, 2, 3, 1, 2, 4]); ++---------------------------------+ +| array_distinct(List([1,2,3,4])) | ++---------------------------------+ +| [1, 2, 3, 4] | ++---------------------------------+ +``` + +#### Aliases + +- list_intersect + +### `array_join` + +_Alias of [array_to_string](#array_to_string)._ + +### `array_length` + +Returns the length of the array dimension. + +``` +array_length(array, dimension) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **dimension**: Array dimension. + +#### Example + +```sql +> select array_length([1, 2, 3, 4, 5], 1); ++-------------------------------------------+ +| array_length(List([1,2,3,4,5]), 1) | ++-------------------------------------------+ +| 5 | ++-------------------------------------------+ +``` + +#### Aliases + +- list_length + +### `array_ndims` + +Returns an array of the array's dimensions. + +``` +array_dims(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_dims([[1, 2, 3], [4, 5, 6]]); ++---------------------------------+ +| array_dims(List([1,2,3,4,5,6])) | ++---------------------------------+ +| [2, 3] | ++---------------------------------+ +``` + +#### Aliases + +- list_ndims + +### `array_pop_back` + +Extracts the element with the index n from the array. + +``` +array_element(array, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **index**: Index to extract the element from the array. + +#### Example + +```sql +> select array_element([1, 2, 3, 4], 3); ++-----------------------------------------+ +| array_element(List([1,2,3,4]),Int64(3)) | ++-----------------------------------------+ +| 3 | ++-----------------------------------------+ +``` + +#### Aliases + +- list_pop_back + +### `array_pop_front` + +Extracts the element with the index n from the array. + +``` +array_element(array, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **index**: Index to extract the element from the array. + +#### Example + +```sql +> select array_element([1, 2, 3, 4], 3); ++-----------------------------------------+ +| array_element(List([1,2,3,4]),Int64(3)) | ++-----------------------------------------+ +| 3 | ++-----------------------------------------+ +``` + +#### Aliases + +- list_pop_front + +### `array_position` + +Returns the position of the first occurrence of the specified element in the array. + +``` +array_position(array, element) +array_position(array, element, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to search for position in the array. +- **index**: Index at which to start searching. + +#### Example + +```sql +> select array_position([1, 2, 2, 3, 1, 4], 2); ++----------------------------------------------+ +| array_position(List([1,2,2,3,1,4]),Int64(2)) | ++----------------------------------------------+ +| 2 | ++----------------------------------------------+ +> select array_position([1, 2, 2, 3, 1, 4], 2, 3); ++----------------------------------------------------+ +| array_position(List([1,2,2,3,1,4]),Int64(2), Int64(3)) | ++----------------------------------------------------+ +| 3 | ++----------------------------------------------------+ +``` + +#### Aliases + +- list_position +- array_indexof +- list_indexof + +### `array_positions` + +Returns the position of the first occurrence of the specified element in the array. + +``` +array_position(array, element) +array_position(array, element, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to search for position in the array. +- **index**: Index at which to start searching. + +#### Example + +```sql +> select array_position([1, 2, 2, 3, 1, 4], 2); ++----------------------------------------------+ +| array_position(List([1,2,2,3,1,4]),Int64(2)) | ++----------------------------------------------+ +| 2 | ++----------------------------------------------+ +> select array_position([1, 2, 2, 3, 1, 4], 2, 3); ++----------------------------------------------------+ +| array_position(List([1,2,2,3,1,4]),Int64(2), Int64(3)) | ++----------------------------------------------------+ +| 3 | ++----------------------------------------------------+ +``` + +#### Aliases + +- list_positions + +### `array_prepend` + +Appends an element to the end of an array. + +``` +array_append(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to append to the array. + +#### Example + +```sql +> select array_append([1, 2, 3], 4); ++--------------------------------------+ +| array_append(List([1,2,3]),Int64(4)) | ++--------------------------------------+ +| [1, 2, 3, 4] | ++--------------------------------------+ +``` + +#### Aliases + +- list_prepend +- array_push_front +- list_push_front + +### `array_push_back` + +_Alias of [array_append](#array_append)._ + +### `array_push_front` + +_Alias of [array_prepend](#array_prepend)._ + +### `array_remove` + +Removes the first element from the array equal to the given value. + +``` +array_remove(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to be removed from the array. + +#### Example + +```sql +> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); ++----------------------------------------------+ +| array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | ++----------------------------------------------+ +| [1, 2, 3, 2, 1, 4] | ++----------------------------------------------+ +``` + +#### Aliases + +- list_remove + +### `array_remove_all` + +Removes the first element from the array equal to the given value. + +``` +array_remove(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to be removed from the array. + +#### Example + +```sql +> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); ++----------------------------------------------+ +| array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | ++----------------------------------------------+ +| [1, 2, 3, 2, 1, 4] | ++----------------------------------------------+ +``` + +#### Aliases + +- list_remove_all + +### `array_remove_n` + +Removes the first element from the array equal to the given value. + +``` +array_remove(array, element) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **element**: Element to be removed from the array. + +#### Example + +```sql +> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); ++----------------------------------------------+ +| array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | ++----------------------------------------------+ +| [1, 2, 3, 2, 1, 4] | ++----------------------------------------------+ +``` + +#### Aliases + +- list_remove_n + +### `array_repeat` + +Returns an array containing element `count` times. + +``` +array_repeat(element, count) +``` + +#### Arguments + +- **element**: Element expression. Can be a constant, column, or function, and any combination of array operators. +- **count**: Value of how many times to repeat the element. + +#### Example + +```sql +> select array_repeat(1, 3); ++---------------------------------+ +| array_repeat(Int64(1),Int64(3)) | ++---------------------------------+ +| [1, 1, 1] | ++---------------------------------+ +> select array_repeat([1, 2], 2); ++------------------------------------+ +| array_repeat(List([1,2]),Int64(2)) | ++------------------------------------+ +| [[1, 2], [1, 2]] | ++------------------------------------+ +``` + +#### Aliases + +- list_repeat + +### `array_replace` + +Replaces the first `max` occurrences of the specified element with another specified element. + +``` +array_replace_n(array, from, to, max) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **from**: Initial element. +- **to**: Final element. +- **max**: Number of first occurrences to replace. + +#### Example + +```sql +> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); ++-------------------------------------------------------------------+ +| array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | ++-------------------------------------------------------------------+ +| [1, 5, 5, 3, 2, 1, 4] | ++-------------------------------------------------------------------+ +``` + +#### Aliases + +- list_replace + +### `array_replace_all` + +Replaces the first `max` occurrences of the specified element with another specified element. + +``` +array_replace_n(array, from, to, max) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **from**: Initial element. +- **to**: Final element. +- **max**: Number of first occurrences to replace. + +#### Example + +```sql +> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); ++-------------------------------------------------------------------+ +| array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | ++-------------------------------------------------------------------+ +| [1, 5, 5, 3, 2, 1, 4] | ++-------------------------------------------------------------------+ +``` + +#### Aliases + +- list_replace_all + +### `array_replace_n` + +Replaces the first `max` occurrences of the specified element with another specified element. + +``` +array_replace_n(array, from, to, max) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **from**: Initial element. +- **to**: Final element. +- **max**: Number of first occurrences to replace. + +#### Example + +```sql +> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); ++-------------------------------------------------------------------+ +| array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | ++-------------------------------------------------------------------+ +| [1, 5, 5, 3, 2, 1, 4] | ++-------------------------------------------------------------------+ +``` + +#### Aliases + +- list_replace_n + +### `array_resize` + +Resizes the list to contain size elements. Initializes new elements with value or empty if value is not set. + +``` +array_resize(array, size, value) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **size**: New size of given array. +- **value**: Defines new elements' value or empty if value is not set. + +#### Example + +```sql +> select array_resize([1, 2, 3], 5, 0); ++-------------------------------------+ +| array_resize(List([1,2,3],5,0)) | ++-------------------------------------+ +| [1, 2, 3, 0, 0] | ++-------------------------------------+ +``` + +#### Aliases + +- list_resize + +### `array_reverse` + +Returns the array with the order of the elements reversed. + +``` +array_reverse(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_reverse([1, 2, 3, 4]); ++------------------------------------------------------------+ +| array_reverse(List([1, 2, 3, 4])) | ++------------------------------------------------------------+ +| [4, 3, 2, 1] | ++------------------------------------------------------------+ +``` + +#### Aliases + +- list_reverse + +### `array_slice` + +Extracts the element with the index n from the array. + +``` +array_element(array, index) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **index**: Index to extract the element from the array. + +#### Example + +```sql +> select array_element([1, 2, 3, 4], 3); ++-----------------------------------------+ +| array_element(List([1,2,3,4]),Int64(3)) | ++-----------------------------------------+ +| 3 | ++-----------------------------------------+ +``` + +#### Aliases + +- list_slice + +### `array_sort` + +Sort array. + +``` +array_sort(array, desc, nulls_first) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **desc**: Whether to sort in descending order(`ASC` or `DESC`). +- **nulls_first**: Whether to sort nulls first(`NULLS FIRST` or `NULLS LAST`). + +#### Example + +```sql +> select array_sort([3, 1, 2]); ++-----------------------------+ +| array_sort(List([3,1,2])) | ++-----------------------------+ +| [1, 2, 3] | ++-----------------------------+ +``` + +#### Aliases + +- list_sort + +### `array_to_string` + +Converts each element to its text representation. + +``` +array_to_string(array, delimiter) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **delimiter**: Array element separator. + +#### Example + +```sql +> select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); ++----------------------------------------------------+ +| array_to_string(List([1,2,3,4,5,6,7,8]),Utf8(",")) | ++----------------------------------------------------+ +| 1,2,3,4,5,6,7,8 | ++----------------------------------------------------+ +``` + +#### Aliases + +- list_to_string +- array_join +- list_join + +### `array_union` + +Returns distinct values from the array after removing duplicates. + +``` +array_distinct(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select array_distinct([1, 3, 2, 3, 1, 2, 4]); ++---------------------------------+ +| array_distinct(List([1,2,3,4])) | ++---------------------------------+ +| [1, 2, 3, 4] | ++---------------------------------+ +``` + +#### Aliases + +- list_union + +### `cardinality` + +Returns the total number of elements in the array. + +``` +cardinality(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]); ++--------------------------------------+ +| cardinality(List([1,2,3,4,5,6,7,8])) | ++--------------------------------------+ +| 8 | ++--------------------------------------+ +``` + +### `empty` + +Returns 1 for an empty array or 0 for a non-empty array. + +``` +empty(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select empty([1]); ++------------------+ +| empty(List([1])) | ++------------------+ +| 0 | ++------------------+ +``` + +#### Aliases + +- array_empty +- list_empty + +### `flatten` + +Converts an array of arrays to a flat array. + +- Applies to any depth of nested arrays +- Does not change arrays that are already flat + +The flattened array contains all the elements from all source arrays. + +``` +flatten(array) +``` + +#### Arguments + +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. + +#### Example + +```sql +> select flatten([[1, 2], [3, 4]]); ++------------------------------+ +| flatten(List([1,2], [3,4])) | ++------------------------------+ +| [1, 2, 3, 4] | ++------------------------------+ +``` + +### `generate_series` + +Similar to the range function, but it includes the upper bound. + +``` +generate_series(start, stop, step) +``` + +#### Arguments + +- **start**: start of the series. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. +- **end**: end of the series (included). Type must be the same as start. +- **step**: increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges. + +#### Example + +```sql +> select generate_series(1,3); ++------------------------------------+ +| generate_series(Int64(1),Int64(3)) | ++------------------------------------+ +| [1, 2, 3] | ++------------------------------------+ +``` + +### `list_any_value` + +_Alias of [array_any_value](#array_any_value)._ + +### `list_append` + +_Alias of [array_append](#array_append)._ + +### `list_cat` + +_Alias of [array_concat](#array_concat)._ + +### `list_concat` + +_Alias of [array_concat](#array_concat)._ + +### `list_contains` + +_Alias of [array_has](#array_has)._ + +### `list_dims` + +_Alias of [array_dims](#array_dims)._ + +### `list_distance` + +_Alias of [array_distance](#array_distance)._ + +### `list_distinct` + +_Alias of [array_distinct](#array_distinct)._ + +### `list_element` + +_Alias of [array_element](#array_element)._ + +### `list_empty` + +_Alias of [empty](#empty)._ + +### `list_except` + +_Alias of [array_except](#array_except)._ + +### `list_extract` + +_Alias of [array_element](#array_element)._ + +### `list_has` + +_Alias of [array_has](#array_has)._ + +### `list_has_all` + +_Alias of [array_has_all](#array_has_all)._ + +### `list_has_any` + +_Alias of [array_has_any](#array_has_any)._ + +### `list_indexof` + +_Alias of [array_position](#array_position)._ + +### `list_intersect` + +_Alias of [array_intersect](#array_intersect)._ + +### `list_join` + +_Alias of [array_to_string](#array_to_string)._ + +### `list_length` -Note: this documentation is in the process of being migrated to be [automatically created from the codebase]. -Please see the [Scalar Functions (new)](scalar_functions_new.md) page for -the rest of the documentation. +_Alias of [array_length](#array_length)._ -[automatically created from the codebase]: https://github.com/apache/datafusion/issues/12740 +### `list_ndims` -## Conditional Functions +_Alias of [array_ndims](#array_ndims)._ -See the new documentation [`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) +### `list_pop_back` -## String Functions +_Alias of [array_pop_back](#array_pop_back)._ -See the new documentation [`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) +### `list_pop_front` -### `position` +_Alias of [array_pop_front](#array_pop_front)._ + +### `list_position` + +_Alias of [array_position](#array_position)._ + +### `list_positions` + +_Alias of [array_positions](#array_positions)._ + +### `list_prepend` + +_Alias of [array_prepend](#array_prepend)._ + +### `list_push_back` + +_Alias of [array_append](#array_append)._ + +### `list_push_front` + +_Alias of [array_prepend](#array_prepend)._ + +### `list_remove` + +_Alias of [array_remove](#array_remove)._ + +### `list_remove_all` + +_Alias of [array_remove_all](#array_remove_all)._ + +### `list_remove_n` + +_Alias of [array_remove_n](#array_remove_n)._ + +### `list_repeat` + +_Alias of [array_repeat](#array_repeat)._ + +### `list_replace` + +_Alias of [array_replace](#array_replace)._ + +### `list_replace_all` + +_Alias of [array_replace_all](#array_replace_all)._ + +### `list_replace_n` + +_Alias of [array_replace_n](#array_replace_n)._ + +### `list_resize` + +_Alias of [array_resize](#array_resize)._ + +### `list_reverse` + +_Alias of [array_reverse](#array_reverse)._ + +### `list_slice` -Returns the position of `substr` in `origstr` (counting from 1). If `substr` does -not appear in `origstr`, return 0. +_Alias of [array_slice](#array_slice)._ + +### `list_sort` + +_Alias of [array_sort](#array_sort)._ + +### `list_to_string` + +_Alias of [array_to_string](#array_to_string)._ + +### `list_union` + +_Alias of [array_union](#array_union)._ + +### `make_array` + +Returns an array using the specified input expressions. ``` -position(substr in origstr) +make_array(expression1[, ..., expression_n]) ``` #### Arguments -- **substr**: The pattern string. -- **origstr**: The model string. +- **expression_n**: Expression to include in the output array. Can be a constant, column, or function, and any combination of arithmetic or string operators. -## Time and Date Functions +#### Example + +```sql +> select make_array(1, 2, 3, 4, 5); ++----------------------------------------------------------+ +| make_array(Int64(1),Int64(2),Int64(3),Int64(4),Int64(5)) | ++----------------------------------------------------------+ +| [1, 2, 3, 4, 5] | ++----------------------------------------------------------+ +``` + +#### Aliases + +- make_list -- [extract](#extract) +### `make_list` -### `extract` +_Alias of [make_array](#make_array)._ -Returns a sub-field from a time value as an integer. +### `range` + +Returns an Arrow array between start and stop with step. The range start..end contains all values with start <= x < end. It is empty if start >= end. Step cannot be 0. ``` -extract(field FROM source) +range(start, stop, step) ``` -Equivalent to calling `date_part('field', source)`. For example, these are equivalent: +#### Arguments + +- **start**: Start of the range. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. +- **end**: End of the range (not included). Type must be the same as start. +- **step**: Increase by step (cannot be 0). Steps less than a day are supported only for timestamp ranges. + +#### Example ```sql -extract(day FROM '2024-04-13'::date) -date_part('day', '2024-04-13'::date) +> select range(2, 10, 3); ++-----------------------------------+ +| range(Int64(2),Int64(10),Int64(3))| ++-----------------------------------+ +| [2, 5, 8] | ++-----------------------------------+ + +> select range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH); ++--------------------------------------------------------------+ +| range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH) | ++--------------------------------------------------------------+ +| [1992-09-01, 1992-10-01, 1992-11-01, 1992-12-01, 1993-01-01, 1993-02-01] | ++--------------------------------------------------------------+ ``` -See [date_part](#date_part). +### `string_to_array` -## Array Functions +Converts each element to its text representation. -- [unnest](#unnest) -- [range](#range) +``` +array_to_string(array, delimiter) +``` + +#### Arguments -### `unnest` +- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. +- **delimiter**: Array element separator. -Transforms an array into rows. +#### Example + +```sql +> select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); ++----------------------------------------------------+ +| array_to_string(List([1,2,3,4,5,6,7,8]),Utf8(",")) | ++----------------------------------------------------+ +| 1,2,3,4,5,6,7,8 | ++----------------------------------------------------+ +``` + +#### Aliases + +- string_to_list + +### `string_to_list` + +_Alias of [string_to_array](#string_to_array)._ + +## Struct Functions + +- [named_struct](#named_struct) +- [row](#row) +- [struct](#struct) + +### `named_struct` + +Returns an Arrow struct using the specified name and input expressions pairs. + +``` +named_struct(expression1_name, expression1_input[, ..., expression_n_name, expression_n_input]) +``` #### Arguments -- **array**: Array expression to unnest. - Can be a constant, column, or function, and any combination of array operators. +- **expression_n_name**: Name of the column field. Must be a constant string. +- **expression_n_input**: Expression to include in the output struct. Can be a constant, column, or function, and any combination of arithmetic or string operators. -#### Examples +#### Example +For example, this query converts two columns `a` and `b` to a single column with +a struct type of fields `field_a` and `field_b`: + +```sql +> select * from t; ++---+---+ +| a | b | ++---+---+ +| 1 | 2 | +| 3 | 4 | ++---+---+ +> select named_struct('field_a', a, 'field_b', b) from t; ++-------------------------------------------------------+ +| named_struct(Utf8("field_a"),t.a,Utf8("field_b"),t.b) | ++-------------------------------------------------------+ +| {field_a: 1, field_b: 2} | +| {field_a: 3, field_b: 4} | ++-------------------------------------------------------+ ``` -> select unnest(make_array(1, 2, 3, 4, 5)); -+------------------------------------------------------------------+ -| unnest(make_array(Int64(1),Int64(2),Int64(3),Int64(4),Int64(5))) | -+------------------------------------------------------------------+ -| 1 | -| 2 | -| 3 | -| 4 | -| 5 | -+------------------------------------------------------------------+ + +### `row` + +_Alias of [struct](#struct)._ + +### `struct` + +Returns an Arrow struct using the specified input expressions optionally named. +Fields in the returned struct use the optional name or the `cN` naming convention. +For example: `c0`, `c1`, `c2`, etc. + +``` +struct(expression1[, ..., expression_n]) +``` + +#### Arguments + +- **expression1, expression_n**: Expression to include in the output struct. Can be a constant, column, or function, any combination of arithmetic or string operators. + +#### Example + +For example, this query converts two columns `a` and `b` to a single column with +a struct type of fields `field_a` and `c1`: + +```sql +> select * from t; ++---+---+ +| a | b | ++---+---+ +| 1 | 2 | +| 3 | 4 | ++---+---+ + +-- use default names `c0`, `c1` +> select struct(a, b) from t; ++-----------------+ +| struct(t.a,t.b) | ++-----------------+ +| {c0: 1, c1: 2} | +| {c0: 3, c1: 4} | ++-----------------+ + +-- name the first field `field_a` +select struct(a as field_a, b) from t; ++--------------------------------------------------+ +| named_struct(Utf8("field_a"),t.a,Utf8("c1"),t.b) | ++--------------------------------------------------+ +| {field_a: 1, c1: 2} | +| {field_a: 3, c1: 4} | ++--------------------------------------------------+ ``` +#### Aliases + +- row + +## Map Functions + +- [element_at](#element_at) +- [map](#map) +- [map_extract](#map_extract) +- [map_keys](#map_keys) +- [map_values](#map_values) + +### `element_at` + +_Alias of [map_extract](#map_extract)._ + +### `map` + +Returns an Arrow map with the specified key-value pairs. + +The `make_map` function creates a map from two lists: one for keys and one for values. Each key must be unique and non-null. + ``` -> select unnest(range(0, 10)); -+-----------------------------------+ -| unnest(range(Int64(0),Int64(10))) | -+-----------------------------------+ -| 0 | -| 1 | -| 2 | -| 3 | -| 4 | -| 5 | -| 6 | -| 7 | -| 8 | -| 9 | -+-----------------------------------+ +map(key, value) +map(key: value) +make_map(['key1', 'key2'], ['value1', 'value2']) ``` -### `range` +#### Arguments + +- **key**: For `map`: Expression to be used for key. Can be a constant, column, function, or any combination of arithmetic or string operators. + For `make_map`: The list of keys to be used in the map. Each key must be unique and non-null. +- **value**: For `map`: Expression to be used for value. Can be a constant, column, function, or any combination of arithmetic or string operators. + For `make_map`: The list of values to be mapped to the corresponding keys. + +#### Example + +````sql + -- Using map function + SELECT MAP('type', 'test'); + ---- + {type: test} + + SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33, null]); + ---- + {POST: 41, HEAD: 33, PATCH: } + + SELECT MAP([[1,2], [3,4]], ['a', 'b']); + ---- + {[1, 2]: a, [3, 4]: b} + + SELECT MAP { 'a': 1, 'b': 2 }; + ---- + {a: 1, b: 2} + + -- Using make_map function + SELECT MAKE_MAP(['POST', 'HEAD'], [41, 33]); + ---- + {POST: 41, HEAD: 33} + + SELECT MAKE_MAP(['key1', 'key2'], ['value1', null]); + ---- + {key1: value1, key2: } + ``` + + +### `map_extract` + +Returns a list containing the value for the given key or an empty list if the key is not present in the map. + +```` -Returns an Arrow array between start and stop with step. `SELECT range(2, 10, 3) -> [2, 5, 8]` or -`SELECT range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH);` +map_extract(map, key) -The range start..end contains all values with start <= x < end. It is empty if start >= end. +```` +#### Arguments + +- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. +- **key**: Key to extract from the map. Can be a constant, column, or function, any combination of arithmetic or string operators, or a named expression of the previously listed. + +#### Example + +```sql +SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); +---- +[1] + +SELECT map_extract(MAP {1: 'one', 2: 'two'}, 2); +---- +['two'] + +SELECT map_extract(MAP {'x': 10, 'y': NULL, 'z': 30}, 'y'); +---- +[] +```` + +#### Aliases + +- element_at -Step can not be 0 (then the range will be nonsense.). +### `map_keys` -Note that when the required range is a number, it accepts (stop), (start, stop), and (start, stop, step) as parameters, -but when the required range is a date or timestamp, it must be 3 non-NULL parameters. -For example, +Returns a list of all keys in the map. ``` -SELECT range(3); -SELECT range(1,5); -SELECT range(1,5,1); +map_keys(map) +``` + +#### Arguments + +- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. + +#### Example + +```sql +SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[a, b, c] + +SELECT map_keys(map([100, 5], [42, 43])); +---- +[100, 5] ``` -are allowed in number ranges +### `map_values` -but in date and timestamp ranges, only +Returns a list of all values in the map. ``` -SELECT range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH); -SELECT range(TIMESTAMP '1992-09-01', TIMESTAMP '1993-03-01', INTERVAL '1' MONTH); +map_values(map) +``` + +#### Arguments + +- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. + +#### Example + +```sql +SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[1, , 3] + +SELECT map_values(map([100, 5], [42, 43])); +---- +[42, 43] ``` -is allowed, and +## Hashing Functions + +- [digest](#digest) +- [md5](#md5) +- [sha224](#sha224) +- [sha256](#sha256) +- [sha384](#sha384) +- [sha512](#sha512) + +### `digest` +Computes the binary hash of an expression using the specified algorithm. + +``` +digest(expression, algorithm) ``` -SELECT range(DATE '1992-09-01', DATE '1993-03-01', NULL); -SELECT range(NULL, DATE '1993-03-01', INTERVAL '1' MONTH); -SELECT range(DATE '1992-09-01', NULL, INTERVAL '1' MONTH); + +#### Arguments + +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **algorithm**: String expression specifying algorithm to use. Must be one of: +- md5 +- sha224 +- sha256 +- sha384 +- sha512 +- blake2s +- blake2b +- blake3 + +#### Example + +```sql +> select digest('foo', 'sha256'); ++------------------------------------------+ +| digest(Utf8("foo"), Utf8("sha256")) | ++------------------------------------------+ +| | ++------------------------------------------+ ``` -are not allowed +### `md5` + +Computes an MD5 128-bit checksum for a string expression. + +``` +md5(expression) +``` #### Arguments -- **start**: start of the range. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. -- **end**: end of the range (not included). Type must be the same as start. -- **step**: increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges. +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -#### Aliases +#### Example + +```sql +> select md5('foo'); ++-------------------------------------+ +| md5(Utf8("foo")) | ++-------------------------------------+ +| | ++-------------------------------------+ +``` -- generate_series +### `sha224` -## Struct Functions +Computes the SHA-224 hash of a binary string. + +``` +sha224(expression) +``` + +#### Arguments + +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- [unnest](#unnest-struct) +#### Example -For more struct functions see the new documentation [ -`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) +```sql +> select sha224('foo'); ++------------------------------------------+ +| sha224(Utf8("foo")) | ++------------------------------------------+ +| | ++------------------------------------------+ +``` -### `unnest (struct)` +### `sha256` -Unwraps struct fields into columns. +Computes the SHA-256 hash of a binary string. + +``` +sha256(expression) +``` #### Arguments -- **struct**: Object expression to unnest. - Can be a constant, column, or function, and any combination of object operators. +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -#### Examples +#### Example +```sql +> select sha256('foo'); ++--------------------------------------+ +| sha256(Utf8("foo")) | ++--------------------------------------+ +| | ++--------------------------------------+ ``` -> select * from foo; -+---------------------+ -| column1 | -+---------------------+ -| {a: 5, b: a string} | -+---------------------+ -> select unnest(column1) from foo; -+-----------------------+-----------------------+ -| unnest(foo.column1).a | unnest(foo.column1).b | -+-----------------------+-----------------------+ -| 5 | a string | -+-----------------------+-----------------------+ +### `sha384` + +Computes the SHA-384 hash of a binary string. + +``` +sha384(expression) +``` + +#### Arguments + +- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select sha384('foo'); ++-----------------------------------------+ +| sha384(Utf8("foo")) | ++-----------------------------------------+ +| | ++-----------------------------------------+ +``` + +### `sha512` + +Computes the SHA-512 hash of a binary string. + +``` +sha512(expression) +``` + +#### Arguments + +- **expression**: String + +#### Example + +```sql +> select sha512('foo'); ++-------------------------------------------+ +| sha512(Utf8("foo")) | ++-------------------------------------------+ +| | ++-------------------------------------------+ ``` ## Other Functions -See the new documentation [`here`](https://datafusion.apache.org/user-guide/sql/scalar_functions_new.html) +- [arrow_cast](#arrow_cast) +- [arrow_typeof](#arrow_typeof) +- [get_field](#get_field) +- [version](#version) + +### `arrow_cast` + +Casts a value to a specific Arrow data type. + +``` +arrow_cast(expression, datatype) +``` + +#### Arguments + +- **expression**: Expression to cast. The expression can be a constant, column, or function, and any combination of operators. +- **datatype**: [Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) name to cast to, as a string. The format is the same as that returned by [`arrow_typeof`] + +#### Example + +```sql +> select arrow_cast(-5, 'Int8') as a, + arrow_cast('foo', 'Dictionary(Int32, Utf8)') as b, + arrow_cast('bar', 'LargeUtf8') as c, + arrow_cast('2023-01-02T12:53:02', 'Timestamp(Microsecond, Some("+08:00"))') as d + ; ++----+-----+-----+---------------------------+ +| a | b | c | d | ++----+-----+-----+---------------------------+ +| -5 | foo | bar | 2023-01-02T12:53:02+08:00 | ++----+-----+-----+---------------------------+ +``` + +### `arrow_typeof` + +Returns the name of the underlying [Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) of the expression. + +``` +arrow_typeof(expression) +``` + +#### Arguments + +- **expression**: Expression to evaluate. The expression can be a constant, column, or function, and any combination of operators. + +#### Example + +```sql +> select arrow_typeof('foo'), arrow_typeof(1); ++---------------------------+------------------------+ +| arrow_typeof(Utf8("foo")) | arrow_typeof(Int64(1)) | ++---------------------------+------------------------+ +| Utf8 | Int64 | ++---------------------------+------------------------+ +``` + +### `get_field` + +Returns a field within a map or a struct with the given key. +Note: most users invoke `get_field` indirectly via field access +syntax such as `my_struct_col['field_name']` which results in a call to +`get_field(my_struct_col, 'field_name')`. + +``` +get_field(expression1, expression2) +``` + +#### Arguments + +- **expression1**: The map or struct to retrieve a field for. +- **expression2**: The field name in the map or struct to retrieve data for. Must evaluate to a string. + +#### Example + +```sql +> create table t (idx varchar, v varchar) as values ('data','fusion'), ('apache', 'arrow'); +> select struct(idx, v) from t as c; ++-------------------------+ +| struct(c.idx,c.v) | ++-------------------------+ +| {c0: data, c1: fusion} | +| {c0: apache, c1: arrow} | ++-------------------------+ +> select get_field((select struct(idx, v) from t), 'c0'); ++-----------------------+ +| struct(t.idx,t.v)[c0] | ++-----------------------+ +| data | +| apache | ++-----------------------+ +> select get_field((select struct(idx, v) from t), 'c1'); ++-----------------------+ +| struct(t.idx,t.v)[c1] | ++-----------------------+ +| fusion | +| arrow | ++-----------------------+ +``` + +### `version` + +Returns the version of DataFusion. + +``` +version() +``` + +#### Example + +```sql +> select version(); ++--------------------------------------------+ +| version() | ++--------------------------------------------+ +| Apache DataFusion 42.0.0, aarch64 on macos | ++--------------------------------------------+ +``` diff --git a/docs/source/user-guide/sql/scalar_functions_new.md b/docs/source/user-guide/sql/scalar_functions_new.md deleted file mode 100644 index c15821ac89a3a..0000000000000 --- a/docs/source/user-guide/sql/scalar_functions_new.md +++ /dev/null @@ -1,4325 +0,0 @@ - - - - -# Scalar Functions (NEW) - -Note: this documentation is in the process of being migrated to be [automatically created from the codebase]. -Please see the [Scalar Functions (old)](aggregate_functions.md) page for -the rest of the documentation. - -[automatically created from the codebase]: https://github.com/apache/datafusion/issues/12740 - -## Math Functions - -- [abs](#abs) -- [acos](#acos) -- [acosh](#acosh) -- [asin](#asin) -- [asinh](#asinh) -- [atan](#atan) -- [atan2](#atan2) -- [atanh](#atanh) -- [cbrt](#cbrt) -- [ceil](#ceil) -- [cos](#cos) -- [cosh](#cosh) -- [cot](#cot) -- [degrees](#degrees) -- [exp](#exp) -- [factorial](#factorial) -- [floor](#floor) -- [gcd](#gcd) -- [isnan](#isnan) -- [iszero](#iszero) -- [lcm](#lcm) -- [ln](#ln) -- [log](#log) -- [log10](#log10) -- [log2](#log2) -- [nanvl](#nanvl) -- [pi](#pi) -- [pow](#pow) -- [power](#power) -- [radians](#radians) -- [random](#random) -- [round](#round) -- [signum](#signum) -- [sin](#sin) -- [sinh](#sinh) -- [sqrt](#sqrt) -- [tan](#tan) -- [tanh](#tanh) -- [trunc](#trunc) - -### `abs` - -Returns the absolute value of a number. - -``` -abs(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `acos` - -Returns the arc cosine or inverse cosine of a number. - -``` -acos(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `acosh` - -Returns the area hyperbolic cosine or inverse hyperbolic cosine of a number. - -``` -acosh(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `asin` - -Returns the arc sine or inverse sine of a number. - -``` -asin(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `asinh` - -Returns the area hyperbolic sine or inverse hyperbolic sine of a number. - -``` -asinh(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `atan` - -Returns the arc tangent or inverse tangent of a number. - -``` -atan(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `atan2` - -Returns the arc tangent or inverse tangent of `expression_y / expression_x`. - -``` -atan2(expression_y, expression_x) -``` - -#### Arguments - -- **expression_y**: First numeric expression to operate on. - Can be a constant, column, or function, and any combination of arithmetic operators. -- **expression_x**: Second numeric expression to operate on. - Can be a constant, column, or function, and any combination of arithmetic operators. - -### `atanh` - -Returns the area hyperbolic tangent or inverse hyperbolic tangent of a number. - -``` -atanh(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `cbrt` - -Returns the cube root of a number. - -``` -cbrt(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `ceil` - -Returns the nearest integer greater than or equal to a number. - -``` -ceil(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `cos` - -Returns the cosine of a number. - -``` -cos(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `cosh` - -Returns the hyperbolic cosine of a number. - -``` -cosh(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `cot` - -Returns the cotangent of a number. - -``` -cot(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `degrees` - -Converts radians to degrees. - -``` -degrees(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `exp` - -Returns the base-e exponential of a number. - -``` -exp(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `factorial` - -Factorial. Returns 1 if value is less than 2. - -``` -factorial(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `floor` - -Returns the nearest integer less than or equal to a number. - -``` -floor(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `gcd` - -Returns the greatest common divisor of `expression_x` and `expression_y`. Returns 0 if both inputs are zero. - -``` -gcd(expression_x, expression_y) -``` - -#### Arguments - -- **expression_x**: First numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_y**: Second numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `isnan` - -Returns true if a given number is +NaN or -NaN otherwise returns false. - -``` -isnan(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `iszero` - -Returns true if a given number is +0.0 or -0.0 otherwise returns false. - -``` -iszero(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `lcm` - -Returns the least common multiple of `expression_x` and `expression_y`. Returns 0 if either input is zero. - -``` -lcm(expression_x, expression_y) -``` - -#### Arguments - -- **expression_x**: First numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **expression_y**: Second numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `ln` - -Returns the natural logarithm of a number. - -``` -ln(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `log` - -Returns the base-x logarithm of a number. Can either provide a specified base, or if omitted then takes the base-10 of a number. - -``` -log(base, numeric_expression) -log(numeric_expression) -``` - -#### Arguments - -- **base**: Base numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `log10` - -Returns the base-10 logarithm of a number. - -``` -log10(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `log2` - -Returns the base-2 logarithm of a number. - -``` -log2(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `nanvl` - -Returns the first argument if it's not _NaN_. -Returns the second argument otherwise. - -``` -nanvl(expression_x, expression_y) -``` - -#### Arguments - -- **expression_x**: Numeric expression to return if it's not _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators. -- **expression_y**: Numeric expression to return if the first expression is _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators. - -### `pi` - -Returns an approximate value of π. - -``` -pi() -``` - -### `pow` - -_Alias of [power](#power)._ - -### `power` - -Returns a base expression raised to the power of an exponent. - -``` -power(base, exponent) -``` - -#### Arguments - -- **base**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **exponent**: Exponent numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Aliases - -- pow - -### `radians` - -Converts degrees to radians. - -``` -radians(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `random` - -Returns a random float value in the range [0, 1). -The random seed is unique to each row. - -``` -random() -``` - -### `round` - -Rounds a number to the nearest integer. - -``` -round(numeric_expression[, decimal_places]) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **decimal_places**: Optional. The number of decimal places to round to. Defaults to 0. - -### `signum` - -Returns the sign of a number. -Negative numbers return `-1`. -Zero and positive numbers return `1`. - -``` -signum(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `sin` - -Returns the sine of a number. - -``` -sin(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `sinh` - -Returns the hyperbolic sine of a number. - -``` -sinh(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `sqrt` - -Returns the square root of a number. - -``` -sqrt(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `tan` - -Returns the tangent of a number. - -``` -tan(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `tanh` - -Returns the hyperbolic tangent of a number. - -``` -tanh(numeric_expression) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. - -### `trunc` - -Truncates a number to a whole number or truncated to the specified decimal places. - -``` -trunc(numeric_expression[, decimal_places]) -``` - -#### Arguments - -- **numeric_expression**: Numeric expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **decimal_places**: Optional. The number of decimal places to - truncate to. Defaults to 0 (truncate to a whole number). If - `decimal_places` is a positive integer, truncates digits to the - right of the decimal point. If `decimal_places` is a negative - integer, replaces digits to the left of the decimal point with `0`. - -## Conditional Functions - -- [coalesce](#coalesce) -- [ifnull](#ifnull) -- [nullif](#nullif) -- [nvl](#nvl) -- [nvl2](#nvl2) - -### `coalesce` - -Returns the first of its arguments that is not _null_. Returns _null_ if all arguments are _null_. This function is often used to substitute a default value for _null_ values. - -``` -coalesce(expression1[, ..., expression_n]) -``` - -#### Arguments - -- **expression1, expression_n**: Expression to use if previous expressions are _null_. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary. - -#### Example - -```sql -> select coalesce(null, null, 'datafusion'); -+----------------------------------------+ -| coalesce(NULL,NULL,Utf8("datafusion")) | -+----------------------------------------+ -| datafusion | -+----------------------------------------+ -``` - -### `ifnull` - -_Alias of [nvl](#nvl)._ - -### `nullif` - -Returns _null_ if _expression1_ equals _expression2_; otherwise it returns _expression1_. -This can be used to perform the inverse operation of [`coalesce`](#coalesce). - -``` -nullif(expression1, expression2) -``` - -#### Arguments - -- **expression1**: Expression to compare and return if equal to expression2. Can be a constant, column, or function, and any combination of operators. -- **expression2**: Expression to compare to expression1. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select nullif('datafusion', 'data'); -+-----------------------------------------+ -| nullif(Utf8("datafusion"),Utf8("data")) | -+-----------------------------------------+ -| datafusion | -+-----------------------------------------+ -> select nullif('datafusion', 'datafusion'); -+-----------------------------------------------+ -| nullif(Utf8("datafusion"),Utf8("datafusion")) | -+-----------------------------------------------+ -| | -+-----------------------------------------------+ -``` - -### `nvl` - -Returns _expression2_ if _expression1_ is NULL otherwise it returns _expression1_. - -``` -nvl(expression1, expression2) -``` - -#### Arguments - -- **expression1**: Expression to return if not null. Can be a constant, column, or function, and any combination of operators. -- **expression2**: Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select nvl(null, 'a'); -+---------------------+ -| nvl(NULL,Utf8("a")) | -+---------------------+ -| a | -+---------------------+\ -> select nvl('b', 'a'); -+--------------------------+ -| nvl(Utf8("b"),Utf8("a")) | -+--------------------------+ -| b | -+--------------------------+ -``` - -#### Aliases - -- ifnull - -### `nvl2` - -Returns _expression2_ if _expression1_ is not NULL; otherwise it returns _expression3_. - -``` -nvl2(expression1, expression2, expression3) -``` - -#### Arguments - -- **expression1**: Expression to test for null. Can be a constant, column, or function, and any combination of operators. -- **expression2**: Expression to return if expr1 is not null. Can be a constant, column, or function, and any combination of operators. -- **expression3**: Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select nvl2(null, 'a', 'b'); -+--------------------------------+ -| nvl2(NULL,Utf8("a"),Utf8("b")) | -+--------------------------------+ -| b | -+--------------------------------+ -> select nvl2('data', 'a', 'b'); -+----------------------------------------+ -| nvl2(Utf8("data"),Utf8("a"),Utf8("b")) | -+----------------------------------------+ -| a | -+----------------------------------------+ -``` - -## String Functions - -- [ascii](#ascii) -- [bit_length](#bit_length) -- [btrim](#btrim) -- [char_length](#char_length) -- [character_length](#character_length) -- [chr](#chr) -- [concat](#concat) -- [concat_ws](#concat_ws) -- [contains](#contains) -- [ends_with](#ends_with) -- [find_in_set](#find_in_set) -- [initcap](#initcap) -- [instr](#instr) -- [left](#left) -- [length](#length) -- [levenshtein](#levenshtein) -- [lower](#lower) -- [lpad](#lpad) -- [ltrim](#ltrim) -- [octet_length](#octet_length) -- [position](#position) -- [repeat](#repeat) -- [replace](#replace) -- [reverse](#reverse) -- [right](#right) -- [rpad](#rpad) -- [rtrim](#rtrim) -- [split_part](#split_part) -- [starts_with](#starts_with) -- [strpos](#strpos) -- [substr](#substr) -- [substr_index](#substr_index) -- [substring](#substring) -- [substring_index](#substring_index) -- [to_hex](#to_hex) -- [translate](#translate) -- [trim](#trim) -- [upper](#upper) -- [uuid](#uuid) - -### `ascii` - -Returns the Unicode character code of the first character in a string. - -``` -ascii(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select ascii('abc'); -+--------------------+ -| ascii(Utf8("abc")) | -+--------------------+ -| 97 | -+--------------------+ -> select ascii('🚀'); -+-------------------+ -| ascii(Utf8("🚀")) | -+-------------------+ -| 128640 | -+-------------------+ -``` - -**Related functions**: - -- [chr](#chr) - -### `bit_length` - -Returns the bit length of a string. - -``` -bit_length(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select bit_length('datafusion'); -+--------------------------------+ -| bit_length(Utf8("datafusion")) | -+--------------------------------+ -| 80 | -+--------------------------------+ -``` - -**Related functions**: - -- [length](#length) -- [octet_length](#octet_length) - -### `btrim` - -Trims the specified trim string from the start and end of a string. If no trim string is provided, all whitespace is removed from the start and end of the input string. - -``` -btrim(str[, trim_str]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **trim_str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. _Default is whitespace characters._ - -#### Example - -```sql -> select btrim('__datafusion____', '_'); -+-------------------------------------------+ -| btrim(Utf8("__datafusion____"),Utf8("_")) | -+-------------------------------------------+ -| datafusion | -+-------------------------------------------+ -``` - -#### Aliases - -- trim - -**Related functions**: - -- [ltrim](#ltrim) -- [rtrim](#rtrim) - -### `char_length` - -_Alias of [character_length](#character_length)._ - -### `character_length` - -Returns the number of characters in a string. - -``` -character_length(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select character_length('Ångström'); -+------------------------------------+ -| character_length(Utf8("Ångström")) | -+------------------------------------+ -| 8 | -+------------------------------------+ -``` - -#### Aliases - -- length -- char_length - -**Related functions**: - -- [bit_length](#bit_length) -- [octet_length](#octet_length) - -### `chr` - -Returns the character with the specified ASCII or Unicode code value. - -``` -chr(expression) -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select chr(128640); -+--------------------+ -| chr(Int64(128640)) | -+--------------------+ -| 🚀 | -+--------------------+ -``` - -**Related functions**: - -- [ascii](#ascii) - -### `concat` - -Concatenates multiple strings together. - -``` -concat(str[, ..., str_n]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **str_n**: Subsequent string expressions to concatenate. - -#### Example - -```sql -> select concat('data', 'f', 'us', 'ion'); -+-------------------------------------------------------+ -| concat(Utf8("data"),Utf8("f"),Utf8("us"),Utf8("ion")) | -+-------------------------------------------------------+ -| datafusion | -+-------------------------------------------------------+ -``` - -**Related functions**: - -- [concat_ws](#concat_ws) - -### `concat_ws` - -Concatenates multiple strings together with a specified separator. - -``` -concat_ws(separator, str[, ..., str_n]) -``` - -#### Arguments - -- **separator**: Separator to insert between concatenated strings. -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **str_n**: Subsequent string expressions to concatenate. - -#### Example - -```sql -> select concat_ws('_', 'data', 'fusion'); -+--------------------------------------------------+ -| concat_ws(Utf8("_"),Utf8("data"),Utf8("fusion")) | -+--------------------------------------------------+ -| data_fusion | -+--------------------------------------------------+ -``` - -**Related functions**: - -- [concat](#concat) - -### `contains` - -Return true if search_str is found within string (case-sensitive). - -``` -contains(str, search_str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **search_str**: The string to search for in str. - -#### Example - -```sql -> select contains('the quick brown fox', 'row'); -+---------------------------------------------------+ -| contains(Utf8("the quick brown fox"),Utf8("row")) | -+---------------------------------------------------+ -| true | -+---------------------------------------------------+ -``` - -### `ends_with` - -Tests if a string ends with a substring. - -``` -ends_with(str, substr) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **substr**: Substring to test for. - -#### Example - -```sql -> select ends_with('datafusion', 'soin'); -+--------------------------------------------+ -| ends_with(Utf8("datafusion"),Utf8("soin")) | -+--------------------------------------------+ -| false | -+--------------------------------------------+ -> select ends_with('datafusion', 'sion'); -+--------------------------------------------+ -| ends_with(Utf8("datafusion"),Utf8("sion")) | -+--------------------------------------------+ -| true | -+--------------------------------------------+ -``` - -### `find_in_set` - -Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings. - -``` -find_in_set(str, strlist) -``` - -#### Arguments - -- **str**: String expression to find in strlist. -- **strlist**: A string list is a string composed of substrings separated by , characters. - -#### Example - -```sql -> select find_in_set('b', 'a,b,c,d'); -+----------------------------------------+ -| find_in_set(Utf8("b"),Utf8("a,b,c,d")) | -+----------------------------------------+ -| 2 | -+----------------------------------------+ -``` - -### `initcap` - -Capitalizes the first character in each word in the input string. Words are delimited by non-alphanumeric characters. - -``` -initcap(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select initcap('apache datafusion'); -+------------------------------------+ -| initcap(Utf8("apache datafusion")) | -+------------------------------------+ -| Apache Datafusion | -+------------------------------------+ -``` - -**Related functions**: - -- [lower](#lower) -- [upper](#upper) - -### `instr` - -_Alias of [strpos](#strpos)._ - -### `left` - -Returns a specified number of characters from the left side of a string. - -``` -left(str, n) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **n**: Number of characters to return. - -#### Example - -```sql -> select left('datafusion', 4); -+-----------------------------------+ -| left(Utf8("datafusion"),Int64(4)) | -+-----------------------------------+ -| data | -+-----------------------------------+ -``` - -**Related functions**: - -- [right](#right) - -### `length` - -_Alias of [character_length](#character_length)._ - -### `levenshtein` - -Returns the [`Levenshtein distance`](https://en.wikipedia.org/wiki/Levenshtein_distance) between the two given strings. - -``` -levenshtein(str1, str2) -``` - -#### Arguments - -- **str1**: String expression to compute Levenshtein distance with str2. -- **str2**: String expression to compute Levenshtein distance with str1. - -#### Example - -```sql -> select levenshtein('kitten', 'sitting'); -+---------------------------------------------+ -| levenshtein(Utf8("kitten"),Utf8("sitting")) | -+---------------------------------------------+ -| 3 | -+---------------------------------------------+ -``` - -### `lower` - -Converts a string to lower-case. - -``` -lower(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select lower('Ångström'); -+-------------------------+ -| lower(Utf8("Ångström")) | -+-------------------------+ -| ångström | -+-------------------------+ -``` - -**Related functions**: - -- [initcap](#initcap) -- [upper](#upper) - -### `lpad` - -Pads the left side of a string with another string to a specified string length. - -``` -lpad(str, n[, padding_str]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **n**: String length to pad to. -- **padding_str**: Optional string expression to pad with. Can be a constant, column, or function, and any combination of string operators. _Default is a space._ - -#### Example - -```sql -> select lpad('Dolly', 10, 'hello'); -+---------------------------------------------+ -| lpad(Utf8("Dolly"),Int64(10),Utf8("hello")) | -+---------------------------------------------+ -| helloDolly | -+---------------------------------------------+ -``` - -**Related functions**: - -- [rpad](#rpad) - -### `ltrim` - -Trims the specified trim string from the beginning of a string. If no trim string is provided, all whitespace is removed from the start of the input string. - -``` -ltrim(str[, trim_str]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **trim_str**: String expression to trim from the beginning of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._ - -#### Example - -```sql -> select ltrim(' datafusion '); -+-------------------------------+ -| ltrim(Utf8(" datafusion ")) | -+-------------------------------+ -| datafusion | -+-------------------------------+ -> select ltrim('___datafusion___', '_'); -+-------------------------------------------+ -| ltrim(Utf8("___datafusion___"),Utf8("_")) | -+-------------------------------------------+ -| datafusion___ | -+-------------------------------------------+ -``` - -**Related functions**: - -- [btrim](#btrim) -- [rtrim](#rtrim) - -### `octet_length` - -Returns the length of a string in bytes. - -``` -octet_length(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select octet_length('Ångström'); -+--------------------------------+ -| octet_length(Utf8("Ångström")) | -+--------------------------------+ -| 10 | -+--------------------------------+ -``` - -**Related functions**: - -- [bit_length](#bit_length) -- [length](#length) - -### `position` - -_Alias of [strpos](#strpos)._ - -### `repeat` - -Returns a string with an input string repeated a specified number. - -``` -repeat(str, n) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **n**: Number of times to repeat the input string. - -#### Example - -```sql -> select repeat('data', 3); -+-------------------------------+ -| repeat(Utf8("data"),Int64(3)) | -+-------------------------------+ -| datadatadata | -+-------------------------------+ -``` - -### `replace` - -Replaces all occurrences of a specified substring in a string with a new substring. - -``` -replace(str, substr, replacement) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **substr**: Substring expression to replace in the input string. Substring expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **replacement**: Replacement substring expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select replace('ABabbaBA', 'ab', 'cd'); -+-------------------------------------------------+ -| replace(Utf8("ABabbaBA"),Utf8("ab"),Utf8("cd")) | -+-------------------------------------------------+ -| ABcdbaBA | -+-------------------------------------------------+ -``` - -### `reverse` - -Reverses the character order of a string. - -``` -reverse(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select reverse('datafusion'); -+-----------------------------+ -| reverse(Utf8("datafusion")) | -+-----------------------------+ -| noisufatad | -+-----------------------------+ -``` - -### `right` - -Returns a specified number of characters from the right side of a string. - -``` -right(str, n) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **n**: Number of characters to return - -#### Example - -```sql -> select right('datafusion', 6); -+------------------------------------+ -| right(Utf8("datafusion"),Int64(6)) | -+------------------------------------+ -| fusion | -+------------------------------------+ -``` - -**Related functions**: - -- [left](#left) - -### `rpad` - -Pads the right side of a string with another string to a specified string length. - -``` -rpad(str, n[, padding_str]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **n**: String length to pad to. -- **padding_str**: String expression to pad with. Can be a constant, column, or function, and any combination of string operators. _Default is a space._ - -#### Example - -```sql -> select rpad('datafusion', 20, '_-'); -+-----------------------------------------------+ -| rpad(Utf8("datafusion"),Int64(20),Utf8("_-")) | -+-----------------------------------------------+ -| datafusion_-_-_-_-_- | -+-----------------------------------------------+ -``` - -**Related functions**: - -- [lpad](#lpad) - -### `rtrim` - -Trims the specified trim string from the end of a string. If no trim string is provided, all whitespace is removed from the end of the input string. - -``` -rtrim(str[, trim_str]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **trim_str**: String expression to trim from the end of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._ - -#### Example - -```sql -> select rtrim(' datafusion '); -+-------------------------------+ -| rtrim(Utf8(" datafusion ")) | -+-------------------------------+ -| datafusion | -+-------------------------------+ -> select rtrim('___datafusion___', '_'); -+-------------------------------------------+ -| rtrim(Utf8("___datafusion___"),Utf8("_")) | -+-------------------------------------------+ -| ___datafusion | -+-------------------------------------------+ -``` - -**Related functions**: - -- [btrim](#btrim) -- [ltrim](#ltrim) - -### `split_part` - -Splits a string based on a specified delimiter and returns the substring in the specified position. - -``` -split_part(str, delimiter, pos) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **delimiter**: String or character to split on. -- **pos**: Position of the part to return. - -#### Example - -```sql -> select split_part('1.2.3.4.5', '.', 3); -+--------------------------------------------------+ -| split_part(Utf8("1.2.3.4.5"),Utf8("."),Int64(3)) | -+--------------------------------------------------+ -| 3 | -+--------------------------------------------------+ -``` - -### `starts_with` - -Tests if a string starts with a substring. - -``` -starts_with(str, substr) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **substr**: Substring to test for. - -#### Example - -```sql -> select starts_with('datafusion','data'); -+----------------------------------------------+ -| starts_with(Utf8("datafusion"),Utf8("data")) | -+----------------------------------------------+ -| true | -+----------------------------------------------+ -``` - -### `strpos` - -Returns the starting position of a specified substring in a string. Positions begin at 1. If the substring does not exist in the string, the function returns 0. - -``` -strpos(str, substr) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **substr**: Substring expression to search for. - -#### Example - -```sql -> select strpos('datafusion', 'fus'); -+----------------------------------------+ -| strpos(Utf8("datafusion"),Utf8("fus")) | -+----------------------------------------+ -| 5 | -+----------------------------------------+ -``` - -#### Aliases - -- instr -- position - -### `substr` - -Extracts a substring of a specified number of characters from a specific starting position in a string. - -``` -substr(str, start_pos[, length]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **start_pos**: Character position to start the substring at. The first character in the string has a position of 1. -- **length**: Number of characters to extract. If not specified, returns the rest of the string after the start position. - -#### Example - -```sql -> select substr('datafusion', 5, 3); -+----------------------------------------------+ -| substr(Utf8("datafusion"),Int64(5),Int64(3)) | -+----------------------------------------------+ -| fus | -+----------------------------------------------+ -``` - -#### Aliases - -- substring - -### `substr_index` - -Returns the substring from str before count occurrences of the delimiter delim. -If count is positive, everything to the left of the final delimiter (counting from the left) is returned. -If count is negative, everything to the right of the final delimiter (counting from the right) is returned. - -``` -substr_index(str, delim, count) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **delim**: The string to find in str to split str. -- **count**: The number of times to search for the delimiter. Can be either a positive or negative number. - -#### Example - -```sql -> select substr_index('www.apache.org', '.', 1); -+---------------------------------------------------------+ -| substr_index(Utf8("www.apache.org"),Utf8("."),Int64(1)) | -+---------------------------------------------------------+ -| www | -+---------------------------------------------------------+ -> select substr_index('www.apache.org', '.', -1); -+----------------------------------------------------------+ -| substr_index(Utf8("www.apache.org"),Utf8("."),Int64(-1)) | -+----------------------------------------------------------+ -| org | -+----------------------------------------------------------+ -``` - -#### Aliases - -- substring_index - -### `substring` - -_Alias of [substr](#substr)._ - -### `substring_index` - -_Alias of [substr_index](#substr_index)._ - -### `to_hex` - -Converts an integer to a hexadecimal string. - -``` -to_hex(int) -``` - -#### Arguments - -- **int**: Integer expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select to_hex(12345689); -+-------------------------+ -| to_hex(Int64(12345689)) | -+-------------------------+ -| bc6159 | -+-------------------------+ -``` - -### `translate` - -Translates characters in a string to specified translation characters. - -``` -translate(str, chars, translation) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **chars**: Characters to translate. -- **translation**: Translation characters. Translation characters replace only characters at the same position in the **chars** string. - -#### Example - -```sql -> select translate('twice', 'wic', 'her'); -+--------------------------------------------------+ -| translate(Utf8("twice"),Utf8("wic"),Utf8("her")) | -+--------------------------------------------------+ -| there | -+--------------------------------------------------+ -``` - -### `trim` - -_Alias of [btrim](#btrim)._ - -### `upper` - -Converts a string to upper-case. - -``` -upper(str) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select upper('dataFusion'); -+---------------------------+ -| upper(Utf8("dataFusion")) | -+---------------------------+ -| DATAFUSION | -+---------------------------+ -``` - -**Related functions**: - -- [initcap](#initcap) -- [lower](#lower) - -### `uuid` - -Returns [`UUID v4`]() string value which is unique per row. - -``` -uuid() -``` - -#### Example - -```sql -> select uuid(); -+--------------------------------------+ -| uuid() | -+--------------------------------------+ -| 6ec17ef8-1934-41cc-8d59-d0c8f9eea1f0 | -+--------------------------------------+ -``` - -## Binary String Functions - -- [decode](#decode) -- [encode](#encode) - -### `decode` - -Decode binary data from textual representation in string. - -``` -decode(expression, format) -``` - -#### Arguments - -- **expression**: Expression containing encoded string data -- **format**: Same arguments as [encode](#encode) - -**Related functions**: - -- [encode](#encode) - -### `encode` - -Encode binary data into a textual representation. - -``` -encode(expression, format) -``` - -#### Arguments - -- **expression**: Expression containing string or binary data -- **format**: Supported formats are: `base64`, `hex` - -**Related functions**: - -- [decode](#decode) - -## Regular Expression Functions - -Apache DataFusion uses a [PCRE-like](https://en.wikibooks.org/wiki/Regular_Expressions/Perl-Compatible_Regular_Expressions) -regular expression [syntax](https://docs.rs/regex/latest/regex/#syntax) -(minus support for several features including look-around and backreferences). -The following regular expression functions are supported: - -- [regexp_count](#regexp_count) -- [regexp_like](#regexp_like) -- [regexp_match](#regexp_match) -- [regexp_replace](#regexp_replace) - -### `regexp_count` - -Returns the number of matches that a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has in a string. - -``` -regexp_count(str, regexp[, start, flags]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **regexp**: Regular expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **start**: - **start**: Optional start position (the first position is 1) to search for the regular expression. Can be a constant, column, or function. -- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: - - **i**: case-insensitive: letters match both upper and lower case - - **m**: multi-line mode: ^ and $ match begin/end of line - - **s**: allow . to match \n - - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - - **U**: swap the meaning of x* and x*? - -#### Example - -```sql -> select regexp_count('abcAbAbc', 'abc', 2, 'i'); -+---------------------------------------------------------------+ -| regexp_count(Utf8("abcAbAbc"),Utf8("abc"),Int64(2),Utf8("i")) | -+---------------------------------------------------------------+ -| 1 | -+---------------------------------------------------------------+ -``` - -### `regexp_like` - -Returns true if a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has at least one match in a string, false otherwise. - -``` -regexp_like(str, regexp[, flags]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **regexp**: Regular expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: - - **i**: case-insensitive: letters match both upper and lower case - - **m**: multi-line mode: ^ and $ match begin/end of line - - **s**: allow . to match \n - - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - - **U**: swap the meaning of x* and x*? - -#### Example - -```sql -select regexp_like('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); -+--------------------------------------------------------+ -| regexp_like(Utf8("Köln"),Utf8("[a-zA-Z]ö[a-zA-Z]{2}")) | -+--------------------------------------------------------+ -| true | -+--------------------------------------------------------+ -SELECT regexp_like('aBc', '(b|d)', 'i'); -+--------------------------------------------------+ -| regexp_like(Utf8("aBc"),Utf8("(b|d)"),Utf8("i")) | -+--------------------------------------------------+ -| true | -+--------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs) - -### `regexp_match` - -Returns the first [regular expression](https://docs.rs/regex/latest/regex/#syntax) matches in a string. - -``` -regexp_match(str, regexp[, flags]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **regexp**: Regular expression to match against. - Can be a constant, column, or function. -- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: - - **i**: case-insensitive: letters match both upper and lower case - - **m**: multi-line mode: ^ and $ match begin/end of line - - **s**: allow . to match \n - - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - - **U**: swap the meaning of x* and x*? - -#### Example - -```sql - > select regexp_match('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); - +---------------------------------------------------------+ - | regexp_match(Utf8("Köln"),Utf8("[a-zA-Z]ö[a-zA-Z]{2}")) | - +---------------------------------------------------------+ - | [Köln] | - +---------------------------------------------------------+ - SELECT regexp_match('aBc', '(b|d)', 'i'); - +---------------------------------------------------+ - | regexp_match(Utf8("aBc"),Utf8("(b|d)"),Utf8("i")) | - +---------------------------------------------------+ - | [B] | - +---------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs) - -### `regexp_replace` - -Replaces substrings in a string that match a [regular expression](https://docs.rs/regex/latest/regex/#syntax). - -``` -regexp_replace(str, regexp, replacement[, flags]) -``` - -#### Arguments - -- **str**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **regexp**: Regular expression to match against. - Can be a constant, column, or function. -- **replacement**: Replacement string expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **flags**: Optional regular expression flags that control the behavior of the regular expression. The following flags are supported: -- **g**: (global) Search globally and don't return after the first match -- **i**: case-insensitive: letters match both upper and lower case -- **m**: multi-line mode: ^ and $ match begin/end of line -- **s**: allow . to match \n -- **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used -- **U**: swap the meaning of x* and x*? - -#### Example - -```sql -> select regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g'); -+------------------------------------------------------------------------+ -| regexp_replace(Utf8("foobarbaz"),Utf8("b(..)"),Utf8("X\1Y"),Utf8("g")) | -+------------------------------------------------------------------------+ -| fooXarYXazY | -+------------------------------------------------------------------------+ -SELECT regexp_replace('aBc', '(b|d)', 'Ab\\1a', 'i'); -+-------------------------------------------------------------------+ -| regexp_replace(Utf8("aBc"),Utf8("(b|d)"),Utf8("Ab\1a"),Utf8("i")) | -+-------------------------------------------------------------------+ -| aAbBac | -+-------------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/regexp.rs) - -## Time and Date Functions - -- [current_date](#current_date) -- [current_time](#current_time) -- [current_timestamp](#current_timestamp) -- [date_bin](#date_bin) -- [date_format](#date_format) -- [date_part](#date_part) -- [date_trunc](#date_trunc) -- [datepart](#datepart) -- [datetrunc](#datetrunc) -- [from_unixtime](#from_unixtime) -- [make_date](#make_date) -- [now](#now) -- [to_char](#to_char) -- [to_date](#to_date) -- [to_local_time](#to_local_time) -- [to_timestamp](#to_timestamp) -- [to_timestamp_micros](#to_timestamp_micros) -- [to_timestamp_millis](#to_timestamp_millis) -- [to_timestamp_nanos](#to_timestamp_nanos) -- [to_timestamp_seconds](#to_timestamp_seconds) -- [to_unixtime](#to_unixtime) -- [today](#today) - -### `current_date` - -Returns the current UTC date. - -The `current_date()` return value is determined at query time and will return the same date, no matter when in the query plan the function executes. - -``` -current_date() -``` - -#### Aliases - -- today - -### `current_time` - -Returns the current UTC time. - -The `current_time()` return value is determined at query time and will return the same time, no matter when in the query plan the function executes. - -``` -current_time() -``` - -### `current_timestamp` - -_Alias of [now](#now)._ - -### `date_bin` - -Calculates time intervals and returns the start of the interval nearest to the specified timestamp. Use `date_bin` to downsample time series data by grouping rows into time-based "bins" or "windows" and applying an aggregate or selector function to each window. - -For example, if you "bin" or "window" data into 15 minute intervals, an input timestamp of `2023-01-01T18:18:18Z` will be updated to the start time of the 15 minute bin it is in: `2023-01-01T18:15:00Z`. - -``` -date_bin(interval, expression, origin-timestamp) -``` - -#### Arguments - -- **interval**: Bin interval. -- **expression**: Time expression to operate on. Can be a constant, column, or function. -- **origin-timestamp**: Optional. Starting point used to determine bin boundaries. If not specified defaults 1970-01-01T00:00:00Z (the UNIX epoch in UTC). - -The following intervals are supported: - -- nanoseconds -- microseconds -- milliseconds -- seconds -- minutes -- hours -- days -- weeks -- months -- years -- century - -### `date_format` - -_Alias of [to_char](#to_char)._ - -### `date_part` - -Returns the specified part of the date as an integer. - -``` -date_part(part, expression) -``` - -#### Arguments - -- **part**: Part of the date to return. The following date parts are supported: - - - year - - quarter (emits value in inclusive range [1, 4] based on which quartile of the year the date is in) - - month - - week (week of the year) - - day (day of the month) - - hour - - minute - - second - - millisecond - - microsecond - - nanosecond - - dow (day of the week) - - doy (day of the year) - - epoch (seconds since Unix epoch) - -- **expression**: Time expression to operate on. Can be a constant, column, or function. - -#### Aliases - -- datepart - -### `date_trunc` - -Truncates a timestamp value to a specified precision. - -``` -date_trunc(precision, expression) -``` - -#### Arguments - -- **precision**: Time precision to truncate to. The following precisions are supported: - - - year / YEAR - - quarter / QUARTER - - month / MONTH - - week / WEEK - - day / DAY - - hour / HOUR - - minute / MINUTE - - second / SECOND - -- **expression**: Time expression to operate on. Can be a constant, column, or function. - -#### Aliases - -- datetrunc - -### `datepart` - -_Alias of [date_part](#date_part)._ - -### `datetrunc` - -_Alias of [date_trunc](#date_trunc)._ - -### `from_unixtime` - -Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp. - -``` -from_unixtime(expression) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. - -### `make_date` - -Make a date from year/month/day component parts. - -``` -make_date(year, month, day) -``` - -#### Arguments - -- **year**: Year to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. -- **month**: Month to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. -- **day**: Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. - -#### Example - -```sql -> select make_date(2023, 1, 31); -+-------------------------------------------+ -| make_date(Int64(2023),Int64(1),Int64(31)) | -+-------------------------------------------+ -| 2023-01-31 | -+-------------------------------------------+ -> select make_date('2023', '01', '31'); -+-----------------------------------------------+ -| make_date(Utf8("2023"),Utf8("01"),Utf8("31")) | -+-----------------------------------------------+ -| 2023-01-31 | -+-----------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/make_date.rs) - -### `now` - -Returns the current UTC timestamp. - -The `now()` return value is determined at query time and will return the same timestamp, no matter when in the query plan the function executes. - -``` -now() -``` - -#### Aliases - -- current_timestamp - -### `to_char` - -Returns a string representation of a date, time, timestamp or duration based on a [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html). Unlike the PostgreSQL equivalent of this function numerical formatting is not supported. - -``` -to_char(expression, format) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function that results in a date, time, timestamp or duration. -- **format**: A [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) string to use to convert the expression. -- **day**: Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators. - -#### Example - -```sql -> select to_char('2023-03-01'::date, '%d-%m-%Y'); -+----------------------------------------------+ -| to_char(Utf8("2023-03-01"),Utf8("%d-%m-%Y")) | -+----------------------------------------------+ -| 01-03-2023 | -+----------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_char.rs) - -#### Aliases - -- date_format - -### `to_date` - -Converts a value to a date (`YYYY-MM-DD`). -Supports strings, integer and double types as input. -Strings are parsed as YYYY-MM-DD (e.g. '2023-07-20') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. -Integers and doubles are interpreted as days since the unix epoch (`1970-01-01T00:00:00Z`). -Returns the corresponding date. - -Note: `to_date` returns Date32, which represents its values as the number of days since unix epoch(`1970-01-01`) stored as signed 32 bit value. The largest supported date value is `9999-12-31`. - -``` -to_date('2017-05-31', '%Y-%m-%d') -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order - they appear with the first successful one being returned. If none of the formats successfully parse the expression - an error will be returned. - -#### Example - -```sql -> select to_date('2023-01-31'); -+-----------------------------+ -| to_date(Utf8("2023-01-31")) | -+-----------------------------+ -| 2023-01-31 | -+-----------------------------+ -> select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d'); -+---------------------------------------------------------------+ -| to_date(Utf8("2023/01/31"),Utf8("%Y-%m-%d"),Utf8("%Y/%m/%d")) | -+---------------------------------------------------------------+ -| 2023-01-31 | -+---------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs) - -### `to_local_time` - -Converts a timestamp with a timezone to a timestamp without a timezone (with no offset or timezone information). This function handles daylight saving time changes. - -``` -to_local_time(expression) -``` - -#### Arguments - -- **expression**: Time expression to operate on. Can be a constant, column, or function. - -#### Example - -```sql -> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp); -+---------------------------------------------+ -| to_local_time(Utf8("2024-04-01T00:00:20Z")) | -+---------------------------------------------+ -| 2024-04-01T00:00:20 | -+---------------------------------------------+ - -> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels'); -+---------------------------------------------+ -| to_local_time(Utf8("2024-04-01T00:00:20Z")) | -+---------------------------------------------+ -| 2024-04-01T00:00:20 | -+---------------------------------------------+ - -> SELECT - time, - arrow_typeof(time) as type, - to_local_time(time) as to_local_time, - arrow_typeof(to_local_time(time)) as to_local_time_type -FROM ( - SELECT '2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels' AS time -); -+---------------------------+------------------------------------------------+---------------------+-----------------------------+ -| time | type | to_local_time | to_local_time_type | -+---------------------------+------------------------------------------------+---------------------+-----------------------------+ -| 2024-04-01T00:00:20+02:00 | Timestamp(Nanosecond, Some("Europe/Brussels")) | 2024-04-01T00:00:20 | Timestamp(Nanosecond, None) | -+---------------------------+------------------------------------------------+---------------------+-----------------------------+ - -# combine `to_local_time()` with `date_bin()` to bin on boundaries in the timezone rather -# than UTC boundaries - -> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AS date_bin; -+---------------------+ -| date_bin | -+---------------------+ -| 2024-04-01T00:00:00 | -+---------------------+ - -> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AT TIME ZONE 'Europe/Brussels' AS date_bin_with_timezone; -+---------------------------+ -| date_bin_with_timezone | -+---------------------------+ -| 2024-04-01T00:00:00+02:00 | -+---------------------------+ -``` - -### `to_timestamp` - -Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). Supports strings, integer, unsigned integer, and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats] are provided. Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. - -Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for integer input is between `-9223372037` and `9223372036`. Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. Please use `to_timestamp_seconds` for the input outside of supported bounds. - -``` -to_timestamp(expression[, ..., format_n]) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. - -#### Example - -```sql -> select to_timestamp('2023-01-31T09:26:56.123456789-05:00'); -+-----------------------------------------------------------+ -| to_timestamp(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+-----------------------------------------------------------+ -| 2023-01-31T14:26:56.123456789 | -+-----------------------------------------------------------+ -> select to_timestamp('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+--------------------------------------------------------------------------------------------------------+ -| to_timestamp(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+--------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123456789 | -+--------------------------------------------------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) - -### `to_timestamp_micros` - -Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) Returns the corresponding timestamp. - -``` -to_timestamp_micros(expression[, ..., format_n]) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. - -#### Example - -```sql -> select to_timestamp_micros('2023-01-31T09:26:56.123456789-05:00'); -+------------------------------------------------------------------+ -| to_timestamp_micros(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+------------------------------------------------------------------+ -| 2023-01-31T14:26:56.123456 | -+------------------------------------------------------------------+ -> select to_timestamp_micros('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+---------------------------------------------------------------------------------------------------------------+ -| to_timestamp_micros(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+---------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123456 | -+---------------------------------------------------------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) - -### `to_timestamp_millis` - -Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. - -``` -to_timestamp_millis(expression[, ..., format_n]) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. - -#### Example - -```sql -> select to_timestamp_millis('2023-01-31T09:26:56.123456789-05:00'); -+------------------------------------------------------------------+ -| to_timestamp_millis(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+------------------------------------------------------------------+ -| 2023-01-31T14:26:56.123 | -+------------------------------------------------------------------+ -> select to_timestamp_millis('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+---------------------------------------------------------------------------------------------------------------+ -| to_timestamp_millis(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+---------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123 | -+---------------------------------------------------------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) - -### `to_timestamp_nanos` - -Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. - -``` -to_timestamp_nanos(expression[, ..., format_n]) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. - -#### Example - -```sql -> select to_timestamp_nanos('2023-01-31T09:26:56.123456789-05:00'); -+-----------------------------------------------------------------+ -| to_timestamp_nanos(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+-----------------------------------------------------------------+ -| 2023-01-31T14:26:56.123456789 | -+-----------------------------------------------------------------+ -> select to_timestamp_nanos('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+--------------------------------------------------------------------------------------------------------------+ -| to_timestamp_nanos(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+--------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123456789 | -+---------------------------------------------------------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) - -### `to_timestamp_seconds` - -Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. - -``` -to_timestamp_seconds(expression[, ..., format_n]) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. - -#### Example - -```sql -> select to_timestamp_seconds('2023-01-31T09:26:56.123456789-05:00'); -+-------------------------------------------------------------------+ -| to_timestamp_seconds(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+-------------------------------------------------------------------+ -| 2023-01-31T14:26:56 | -+-------------------------------------------------------------------+ -> select to_timestamp_seconds('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+----------------------------------------------------------------------------------------------------------------+ -| to_timestamp_seconds(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+----------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00 | -+----------------------------------------------------------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) - -### `to_unixtime` - -Converts a value to seconds since the unix epoch (`1970-01-01T00:00:00Z`). Supports strings, dates, timestamps and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. - -``` -to_unixtime(expression[, ..., format_n]) -``` - -#### Arguments - -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -- **format_n**: Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned. - -#### Example - -```sql -> select to_unixtime('2020-09-08T12:00:00+00:00'); -+------------------------------------------------+ -| to_unixtime(Utf8("2020-09-08T12:00:00+00:00")) | -+------------------------------------------------+ -| 1599566400 | -+------------------------------------------------+ -> select to_unixtime('01-14-2023 01:01:30+05:30', '%q', '%d-%m-%Y %H/%M/%S', '%+', '%m-%d-%Y %H:%M:%S%#z'); -+-----------------------------------------------------------------------------------------------------------------------------+ -| to_unixtime(Utf8("01-14-2023 01:01:30+05:30"),Utf8("%q"),Utf8("%d-%m-%Y %H/%M/%S"),Utf8("%+"),Utf8("%m-%d-%Y %H:%M:%S%#z")) | -+-----------------------------------------------------------------------------------------------------------------------------+ -| 1673638290 | -+-----------------------------------------------------------------------------------------------------------------------------+ -``` - -### `today` - -_Alias of [current_date](#current_date)._ - -## Array Functions - -- [array_any_value](#array_any_value) -- [array_append](#array_append) -- [array_cat](#array_cat) -- [array_concat](#array_concat) -- [array_contains](#array_contains) -- [array_dims](#array_dims) -- [array_distance](#array_distance) -- [array_distinct](#array_distinct) -- [array_element](#array_element) -- [array_empty](#array_empty) -- [array_except](#array_except) -- [array_extract](#array_extract) -- [array_has](#array_has) -- [array_has_all](#array_has_all) -- [array_has_any](#array_has_any) -- [array_indexof](#array_indexof) -- [array_intersect](#array_intersect) -- [array_join](#array_join) -- [array_length](#array_length) -- [array_ndims](#array_ndims) -- [array_pop_back](#array_pop_back) -- [array_pop_front](#array_pop_front) -- [array_position](#array_position) -- [array_positions](#array_positions) -- [array_prepend](#array_prepend) -- [array_push_back](#array_push_back) -- [array_push_front](#array_push_front) -- [array_remove](#array_remove) -- [array_remove_all](#array_remove_all) -- [array_remove_n](#array_remove_n) -- [array_repeat](#array_repeat) -- [array_replace](#array_replace) -- [array_replace_all](#array_replace_all) -- [array_replace_n](#array_replace_n) -- [array_resize](#array_resize) -- [array_reverse](#array_reverse) -- [array_slice](#array_slice) -- [array_sort](#array_sort) -- [array_to_string](#array_to_string) -- [array_union](#array_union) -- [cardinality](#cardinality) -- [empty](#empty) -- [flatten](#flatten) -- [generate_series](#generate_series) -- [list_any_value](#list_any_value) -- [list_append](#list_append) -- [list_cat](#list_cat) -- [list_concat](#list_concat) -- [list_contains](#list_contains) -- [list_dims](#list_dims) -- [list_distance](#list_distance) -- [list_distinct](#list_distinct) -- [list_element](#list_element) -- [list_empty](#list_empty) -- [list_except](#list_except) -- [list_extract](#list_extract) -- [list_has](#list_has) -- [list_has_all](#list_has_all) -- [list_has_any](#list_has_any) -- [list_indexof](#list_indexof) -- [list_intersect](#list_intersect) -- [list_join](#list_join) -- [list_length](#list_length) -- [list_ndims](#list_ndims) -- [list_pop_back](#list_pop_back) -- [list_pop_front](#list_pop_front) -- [list_position](#list_position) -- [list_positions](#list_positions) -- [list_prepend](#list_prepend) -- [list_push_back](#list_push_back) -- [list_push_front](#list_push_front) -- [list_remove](#list_remove) -- [list_remove_all](#list_remove_all) -- [list_remove_n](#list_remove_n) -- [list_repeat](#list_repeat) -- [list_replace](#list_replace) -- [list_replace_all](#list_replace_all) -- [list_replace_n](#list_replace_n) -- [list_resize](#list_resize) -- [list_reverse](#list_reverse) -- [list_slice](#list_slice) -- [list_sort](#list_sort) -- [list_to_string](#list_to_string) -- [list_union](#list_union) -- [make_array](#make_array) -- [make_list](#make_list) -- [range](#range) -- [string_to_array](#string_to_array) -- [string_to_list](#string_to_list) - -### `array_any_value` - -Extracts the element with the index n from the array. - -``` -array_element(array, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **index**: Index to extract the element from the array. - -#### Example - -```sql -> select array_element([1, 2, 3, 4], 3); -+-----------------------------------------+ -| array_element(List([1,2,3,4]),Int64(3)) | -+-----------------------------------------+ -| 3 | -+-----------------------------------------+ -``` - -#### Aliases - -- list_any_value - -### `array_append` - -Appends an element to the end of an array. - -``` -array_append(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to append to the array. - -#### Example - -```sql -> select array_append([1, 2, 3], 4); -+--------------------------------------+ -| array_append(List([1,2,3]),Int64(4)) | -+--------------------------------------+ -| [1, 2, 3, 4] | -+--------------------------------------+ -``` - -#### Aliases - -- list_append -- array_push_back -- list_push_back - -### `array_cat` - -_Alias of [array_concat](#array_concat)._ - -### `array_concat` - -Appends an element to the end of an array. - -``` -array_append(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to append to the array. - -#### Example - -```sql -> select array_append([1, 2, 3], 4); -+--------------------------------------+ -| array_append(List([1,2,3]),Int64(4)) | -+--------------------------------------+ -| [1, 2, 3, 4] | -+--------------------------------------+ -``` - -#### Aliases - -- array_cat -- list_concat -- list_cat - -### `array_contains` - -_Alias of [array_has](#array_has)._ - -### `array_dims` - -Returns an array of the array's dimensions. - -``` -array_dims(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_dims([[1, 2, 3], [4, 5, 6]]); -+---------------------------------+ -| array_dims(List([1,2,3,4,5,6])) | -+---------------------------------+ -| [2, 3] | -+---------------------------------+ -``` - -#### Aliases - -- list_dims - -### `array_distance` - -Returns the Euclidean distance between two input arrays of equal length. - -``` -array_distance(array1, array2) -``` - -#### Arguments - -- **array1**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **array2**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_distance([1, 2], [1, 4]); -+------------------------------------+ -| array_distance(List([1,2], [1,4])) | -+------------------------------------+ -| 2.0 | -+------------------------------------+ -``` - -#### Aliases - -- list_distance - -### `array_distinct` - -Returns distinct values from the array after removing duplicates. - -``` -array_distinct(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_distinct([1, 3, 2, 3, 1, 2, 4]); -+---------------------------------+ -| array_distinct(List([1,2,3,4])) | -+---------------------------------+ -| [1, 2, 3, 4] | -+---------------------------------+ -``` - -#### Aliases - -- list_distinct - -### `array_element` - -Extracts the element with the index n from the array. - -``` -array_element(array, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **index**: Index to extract the element from the array. - -#### Example - -```sql -> select array_element([1, 2, 3, 4], 3); -+-----------------------------------------+ -| array_element(List([1,2,3,4]),Int64(3)) | -+-----------------------------------------+ -| 3 | -+-----------------------------------------+ -``` - -#### Aliases - -- array_extract -- list_element -- list_extract - -### `array_empty` - -_Alias of [empty](#empty)._ - -### `array_except` - -Returns an array of the elements that appear in the first array but not in the second. - -``` -array_except(array1, array2) -``` - -#### Arguments - -- **array1**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **array2**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_except([1, 2, 3, 4], [5, 6, 3, 4]); -+----------------------------------------------------+ -| array_except([1, 2, 3, 4], [5, 6, 3, 4]); | -+----------------------------------------------------+ -| [1, 2] | -+----------------------------------------------------+ -> select array_except([1, 2, 3, 4], [3, 4, 5, 6]); -+----------------------------------------------------+ -| array_except([1, 2, 3, 4], [3, 4, 5, 6]); | -+----------------------------------------------------+ -| [1, 2] | -+----------------------------------------------------+ -``` - -#### Aliases - -- list_except - -### `array_extract` - -_Alias of [array_element](#array_element)._ - -### `array_has` - -Returns true if the array contains the element. - -``` -array_has(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_has([1, 2, 3], 2); -+-----------------------------+ -| array_has(List([1,2,3]), 2) | -+-----------------------------+ -| true | -+-----------------------------+ -``` - -#### Aliases - -- list_has -- array_contains -- list_contains - -### `array_has_all` - -Returns true if the array contains the element. - -``` -array_has(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_has([1, 2, 3], 2); -+-----------------------------+ -| array_has(List([1,2,3]), 2) | -+-----------------------------+ -| true | -+-----------------------------+ -``` - -#### Aliases - -- list_has_all - -### `array_has_any` - -Returns true if the array contains the element. - -``` -array_has(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_has([1, 2, 3], 2); -+-----------------------------+ -| array_has(List([1,2,3]), 2) | -+-----------------------------+ -| true | -+-----------------------------+ -``` - -#### Aliases - -- list_has_any - -### `array_indexof` - -_Alias of [array_position](#array_position)._ - -### `array_intersect` - -Returns distinct values from the array after removing duplicates. - -``` -array_distinct(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_distinct([1, 3, 2, 3, 1, 2, 4]); -+---------------------------------+ -| array_distinct(List([1,2,3,4])) | -+---------------------------------+ -| [1, 2, 3, 4] | -+---------------------------------+ -``` - -#### Aliases - -- list_intersect - -### `array_join` - -_Alias of [array_to_string](#array_to_string)._ - -### `array_length` - -Returns the length of the array dimension. - -``` -array_length(array, dimension) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **dimension**: Array dimension. - -#### Example - -```sql -> select array_length([1, 2, 3, 4, 5], 1); -+-------------------------------------------+ -| array_length(List([1,2,3,4,5]), 1) | -+-------------------------------------------+ -| 5 | -+-------------------------------------------+ -``` - -#### Aliases - -- list_length - -### `array_ndims` - -Returns an array of the array's dimensions. - -``` -array_dims(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_dims([[1, 2, 3], [4, 5, 6]]); -+---------------------------------+ -| array_dims(List([1,2,3,4,5,6])) | -+---------------------------------+ -| [2, 3] | -+---------------------------------+ -``` - -#### Aliases - -- list_ndims - -### `array_pop_back` - -Extracts the element with the index n from the array. - -``` -array_element(array, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **index**: Index to extract the element from the array. - -#### Example - -```sql -> select array_element([1, 2, 3, 4], 3); -+-----------------------------------------+ -| array_element(List([1,2,3,4]),Int64(3)) | -+-----------------------------------------+ -| 3 | -+-----------------------------------------+ -``` - -#### Aliases - -- list_pop_back - -### `array_pop_front` - -Extracts the element with the index n from the array. - -``` -array_element(array, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **index**: Index to extract the element from the array. - -#### Example - -```sql -> select array_element([1, 2, 3, 4], 3); -+-----------------------------------------+ -| array_element(List([1,2,3,4]),Int64(3)) | -+-----------------------------------------+ -| 3 | -+-----------------------------------------+ -``` - -#### Aliases - -- list_pop_front - -### `array_position` - -Returns the position of the first occurrence of the specified element in the array. - -``` -array_position(array, element) -array_position(array, element, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to search for position in the array. -- **index**: Index at which to start searching. - -#### Example - -```sql -> select array_position([1, 2, 2, 3, 1, 4], 2); -+----------------------------------------------+ -| array_position(List([1,2,2,3,1,4]),Int64(2)) | -+----------------------------------------------+ -| 2 | -+----------------------------------------------+ -> select array_position([1, 2, 2, 3, 1, 4], 2, 3); -+----------------------------------------------------+ -| array_position(List([1,2,2,3,1,4]),Int64(2), Int64(3)) | -+----------------------------------------------------+ -| 3 | -+----------------------------------------------------+ -``` - -#### Aliases - -- list_position -- array_indexof -- list_indexof - -### `array_positions` - -Returns the position of the first occurrence of the specified element in the array. - -``` -array_position(array, element) -array_position(array, element, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to search for position in the array. -- **index**: Index at which to start searching. - -#### Example - -```sql -> select array_position([1, 2, 2, 3, 1, 4], 2); -+----------------------------------------------+ -| array_position(List([1,2,2,3,1,4]),Int64(2)) | -+----------------------------------------------+ -| 2 | -+----------------------------------------------+ -> select array_position([1, 2, 2, 3, 1, 4], 2, 3); -+----------------------------------------------------+ -| array_position(List([1,2,2,3,1,4]),Int64(2), Int64(3)) | -+----------------------------------------------------+ -| 3 | -+----------------------------------------------------+ -``` - -#### Aliases - -- list_positions - -### `array_prepend` - -Appends an element to the end of an array. - -``` -array_append(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to append to the array. - -#### Example - -```sql -> select array_append([1, 2, 3], 4); -+--------------------------------------+ -| array_append(List([1,2,3]),Int64(4)) | -+--------------------------------------+ -| [1, 2, 3, 4] | -+--------------------------------------+ -``` - -#### Aliases - -- list_prepend -- array_push_front -- list_push_front - -### `array_push_back` - -_Alias of [array_append](#array_append)._ - -### `array_push_front` - -_Alias of [array_prepend](#array_prepend)._ - -### `array_remove` - -Removes the first element from the array equal to the given value. - -``` -array_remove(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to be removed from the array. - -#### Example - -```sql -> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); -+----------------------------------------------+ -| array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | -+----------------------------------------------+ -| [1, 2, 3, 2, 1, 4] | -+----------------------------------------------+ -``` - -#### Aliases - -- list_remove - -### `array_remove_all` - -Removes the first element from the array equal to the given value. - -``` -array_remove(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to be removed from the array. - -#### Example - -```sql -> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); -+----------------------------------------------+ -| array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | -+----------------------------------------------+ -| [1, 2, 3, 2, 1, 4] | -+----------------------------------------------+ -``` - -#### Aliases - -- list_remove_all - -### `array_remove_n` - -Removes the first element from the array equal to the given value. - -``` -array_remove(array, element) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **element**: Element to be removed from the array. - -#### Example - -```sql -> select array_remove([1, 2, 2, 3, 2, 1, 4], 2); -+----------------------------------------------+ -| array_remove(List([1,2,2,3,2,1,4]),Int64(2)) | -+----------------------------------------------+ -| [1, 2, 3, 2, 1, 4] | -+----------------------------------------------+ -``` - -#### Aliases - -- list_remove_n - -### `array_repeat` - -Returns an array containing element `count` times. - -``` -array_repeat(element, count) -``` - -#### Arguments - -- **element**: Element expression. Can be a constant, column, or function, and any combination of array operators. -- **count**: Value of how many times to repeat the element. - -#### Example - -```sql -> select array_repeat(1, 3); -+---------------------------------+ -| array_repeat(Int64(1),Int64(3)) | -+---------------------------------+ -| [1, 1, 1] | -+---------------------------------+ -> select array_repeat([1, 2], 2); -+------------------------------------+ -| array_repeat(List([1,2]),Int64(2)) | -+------------------------------------+ -| [[1, 2], [1, 2]] | -+------------------------------------+ -``` - -#### Aliases - -- list_repeat - -### `array_replace` - -Replaces the first `max` occurrences of the specified element with another specified element. - -``` -array_replace_n(array, from, to, max) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **from**: Initial element. -- **to**: Final element. -- **max**: Number of first occurrences to replace. - -#### Example - -```sql -> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); -+-------------------------------------------------------------------+ -| array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | -+-------------------------------------------------------------------+ -| [1, 5, 5, 3, 2, 1, 4] | -+-------------------------------------------------------------------+ -``` - -#### Aliases - -- list_replace - -### `array_replace_all` - -Replaces the first `max` occurrences of the specified element with another specified element. - -``` -array_replace_n(array, from, to, max) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **from**: Initial element. -- **to**: Final element. -- **max**: Number of first occurrences to replace. - -#### Example - -```sql -> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); -+-------------------------------------------------------------------+ -| array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | -+-------------------------------------------------------------------+ -| [1, 5, 5, 3, 2, 1, 4] | -+-------------------------------------------------------------------+ -``` - -#### Aliases - -- list_replace_all - -### `array_replace_n` - -Replaces the first `max` occurrences of the specified element with another specified element. - -``` -array_replace_n(array, from, to, max) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **from**: Initial element. -- **to**: Final element. -- **max**: Number of first occurrences to replace. - -#### Example - -```sql -> select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); -+-------------------------------------------------------------------+ -| array_replace_n(List([1,2,2,3,2,1,4]),Int64(2),Int64(5),Int64(2)) | -+-------------------------------------------------------------------+ -| [1, 5, 5, 3, 2, 1, 4] | -+-------------------------------------------------------------------+ -``` - -#### Aliases - -- list_replace_n - -### `array_resize` - -Resizes the list to contain size elements. Initializes new elements with value or empty if value is not set. - -``` -array_resize(array, size, value) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **size**: New size of given array. -- **value**: Defines new elements' value or empty if value is not set. - -#### Example - -```sql -> select array_resize([1, 2, 3], 5, 0); -+-------------------------------------+ -| array_resize(List([1,2,3],5,0)) | -+-------------------------------------+ -| [1, 2, 3, 0, 0] | -+-------------------------------------+ -``` - -#### Aliases - -- list_resize - -### `array_reverse` - -Returns the array with the order of the elements reversed. - -``` -array_reverse(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_reverse([1, 2, 3, 4]); -+------------------------------------------------------------+ -| array_reverse(List([1, 2, 3, 4])) | -+------------------------------------------------------------+ -| [4, 3, 2, 1] | -+------------------------------------------------------------+ -``` - -#### Aliases - -- list_reverse - -### `array_slice` - -Extracts the element with the index n from the array. - -``` -array_element(array, index) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **index**: Index to extract the element from the array. - -#### Example - -```sql -> select array_element([1, 2, 3, 4], 3); -+-----------------------------------------+ -| array_element(List([1,2,3,4]),Int64(3)) | -+-----------------------------------------+ -| 3 | -+-----------------------------------------+ -``` - -#### Aliases - -- list_slice - -### `array_sort` - -Sort array. - -``` -array_sort(array, desc, nulls_first) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **desc**: Whether to sort in descending order(`ASC` or `DESC`). -- **nulls_first**: Whether to sort nulls first(`NULLS FIRST` or `NULLS LAST`). - -#### Example - -```sql -> select array_sort([3, 1, 2]); -+-----------------------------+ -| array_sort(List([3,1,2])) | -+-----------------------------+ -| [1, 2, 3] | -+-----------------------------+ -``` - -#### Aliases - -- list_sort - -### `array_to_string` - -Converts each element to its text representation. - -``` -array_to_string(array, delimiter) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **delimiter**: Array element separator. - -#### Example - -```sql -> select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); -+----------------------------------------------------+ -| array_to_string(List([1,2,3,4,5,6,7,8]),Utf8(",")) | -+----------------------------------------------------+ -| 1,2,3,4,5,6,7,8 | -+----------------------------------------------------+ -``` - -#### Aliases - -- list_to_string -- array_join -- list_join - -### `array_union` - -Returns distinct values from the array after removing duplicates. - -``` -array_distinct(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select array_distinct([1, 3, 2, 3, 1, 2, 4]); -+---------------------------------+ -| array_distinct(List([1,2,3,4])) | -+---------------------------------+ -| [1, 2, 3, 4] | -+---------------------------------+ -``` - -#### Aliases - -- list_union - -### `cardinality` - -Returns the total number of elements in the array. - -``` -cardinality(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]); -+--------------------------------------+ -| cardinality(List([1,2,3,4,5,6,7,8])) | -+--------------------------------------+ -| 8 | -+--------------------------------------+ -``` - -### `empty` - -Returns 1 for an empty array or 0 for a non-empty array. - -``` -empty(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select empty([1]); -+------------------+ -| empty(List([1])) | -+------------------+ -| 0 | -+------------------+ -``` - -#### Aliases - -- array_empty -- list_empty - -### `flatten` - -Converts an array of arrays to a flat array. - -- Applies to any depth of nested arrays -- Does not change arrays that are already flat - -The flattened array contains all the elements from all source arrays. - -``` -flatten(array) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. - -#### Example - -```sql -> select flatten([[1, 2], [3, 4]]); -+------------------------------+ -| flatten(List([1,2], [3,4])) | -+------------------------------+ -| [1, 2, 3, 4] | -+------------------------------+ -``` - -### `generate_series` - -Similar to the range function, but it includes the upper bound. - -``` -generate_series(start, stop, step) -``` - -#### Arguments - -- **start**: start of the series. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. -- **end**: end of the series (included). Type must be the same as start. -- **step**: increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges. - -#### Example - -```sql -> select generate_series(1,3); -+------------------------------------+ -| generate_series(Int64(1),Int64(3)) | -+------------------------------------+ -| [1, 2, 3] | -+------------------------------------+ -``` - -### `list_any_value` - -_Alias of [array_any_value](#array_any_value)._ - -### `list_append` - -_Alias of [array_append](#array_append)._ - -### `list_cat` - -_Alias of [array_concat](#array_concat)._ - -### `list_concat` - -_Alias of [array_concat](#array_concat)._ - -### `list_contains` - -_Alias of [array_has](#array_has)._ - -### `list_dims` - -_Alias of [array_dims](#array_dims)._ - -### `list_distance` - -_Alias of [array_distance](#array_distance)._ - -### `list_distinct` - -_Alias of [array_distinct](#array_distinct)._ - -### `list_element` - -_Alias of [array_element](#array_element)._ - -### `list_empty` - -_Alias of [empty](#empty)._ - -### `list_except` - -_Alias of [array_except](#array_except)._ - -### `list_extract` - -_Alias of [array_element](#array_element)._ - -### `list_has` - -_Alias of [array_has](#array_has)._ - -### `list_has_all` - -_Alias of [array_has_all](#array_has_all)._ - -### `list_has_any` - -_Alias of [array_has_any](#array_has_any)._ - -### `list_indexof` - -_Alias of [array_position](#array_position)._ - -### `list_intersect` - -_Alias of [array_intersect](#array_intersect)._ - -### `list_join` - -_Alias of [array_to_string](#array_to_string)._ - -### `list_length` - -_Alias of [array_length](#array_length)._ - -### `list_ndims` - -_Alias of [array_ndims](#array_ndims)._ - -### `list_pop_back` - -_Alias of [array_pop_back](#array_pop_back)._ - -### `list_pop_front` - -_Alias of [array_pop_front](#array_pop_front)._ - -### `list_position` - -_Alias of [array_position](#array_position)._ - -### `list_positions` - -_Alias of [array_positions](#array_positions)._ - -### `list_prepend` - -_Alias of [array_prepend](#array_prepend)._ - -### `list_push_back` - -_Alias of [array_append](#array_append)._ - -### `list_push_front` - -_Alias of [array_prepend](#array_prepend)._ - -### `list_remove` - -_Alias of [array_remove](#array_remove)._ - -### `list_remove_all` - -_Alias of [array_remove_all](#array_remove_all)._ - -### `list_remove_n` - -_Alias of [array_remove_n](#array_remove_n)._ - -### `list_repeat` - -_Alias of [array_repeat](#array_repeat)._ - -### `list_replace` - -_Alias of [array_replace](#array_replace)._ - -### `list_replace_all` - -_Alias of [array_replace_all](#array_replace_all)._ - -### `list_replace_n` - -_Alias of [array_replace_n](#array_replace_n)._ - -### `list_resize` - -_Alias of [array_resize](#array_resize)._ - -### `list_reverse` - -_Alias of [array_reverse](#array_reverse)._ - -### `list_slice` - -_Alias of [array_slice](#array_slice)._ - -### `list_sort` - -_Alias of [array_sort](#array_sort)._ - -### `list_to_string` - -_Alias of [array_to_string](#array_to_string)._ - -### `list_union` - -_Alias of [array_union](#array_union)._ - -### `make_array` - -Returns an array using the specified input expressions. - -``` -make_array(expression1[, ..., expression_n]) -``` - -#### Arguments - -- **expression_n**: Expression to include in the output array. Can be a constant, column, or function, and any combination of arithmetic or string operators. - -#### Example - -```sql -> select make_array(1, 2, 3, 4, 5); -+----------------------------------------------------------+ -| make_array(Int64(1),Int64(2),Int64(3),Int64(4),Int64(5)) | -+----------------------------------------------------------+ -| [1, 2, 3, 4, 5] | -+----------------------------------------------------------+ -``` - -#### Aliases - -- make_list - -### `make_list` - -_Alias of [make_array](#make_array)._ - -### `range` - -Returns an Arrow array between start and stop with step. The range start..end contains all values with start <= x < end. It is empty if start >= end. Step cannot be 0. - -``` -range(start, stop, step) -``` - -#### Arguments - -- **start**: Start of the range. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. -- **end**: End of the range (not included). Type must be the same as start. -- **step**: Increase by step (cannot be 0). Steps less than a day are supported only for timestamp ranges. - -#### Example - -```sql -> select range(2, 10, 3); -+-----------------------------------+ -| range(Int64(2),Int64(10),Int64(3))| -+-----------------------------------+ -| [2, 5, 8] | -+-----------------------------------+ - -> select range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH); -+--------------------------------------------------------------+ -| range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH) | -+--------------------------------------------------------------+ -| [1992-09-01, 1992-10-01, 1992-11-01, 1992-12-01, 1993-01-01, 1993-02-01] | -+--------------------------------------------------------------+ -``` - -### `string_to_array` - -Converts each element to its text representation. - -``` -array_to_string(array, delimiter) -``` - -#### Arguments - -- **array**: Array expression. Can be a constant, column, or function, and any combination of array operators. -- **delimiter**: Array element separator. - -#### Example - -```sql -> select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); -+----------------------------------------------------+ -| array_to_string(List([1,2,3,4,5,6,7,8]),Utf8(",")) | -+----------------------------------------------------+ -| 1,2,3,4,5,6,7,8 | -+----------------------------------------------------+ -``` - -#### Aliases - -- string_to_list - -### `string_to_list` - -_Alias of [string_to_array](#string_to_array)._ - -## Struct Functions - -- [named_struct](#named_struct) -- [row](#row) -- [struct](#struct) - -### `named_struct` - -Returns an Arrow struct using the specified name and input expressions pairs. - -``` -named_struct(expression1_name, expression1_input[, ..., expression_n_name, expression_n_input]) -``` - -#### Arguments - -- **expression_n_name**: Name of the column field. Must be a constant string. -- **expression_n_input**: Expression to include in the output struct. Can be a constant, column, or function, and any combination of arithmetic or string operators. - -#### Example - -For example, this query converts two columns `a` and `b` to a single column with -a struct type of fields `field_a` and `field_b`: - -```sql -> select * from t; -+---+---+ -| a | b | -+---+---+ -| 1 | 2 | -| 3 | 4 | -+---+---+ -> select named_struct('field_a', a, 'field_b', b) from t; -+-------------------------------------------------------+ -| named_struct(Utf8("field_a"),t.a,Utf8("field_b"),t.b) | -+-------------------------------------------------------+ -| {field_a: 1, field_b: 2} | -| {field_a: 3, field_b: 4} | -+-------------------------------------------------------+ -``` - -### `row` - -_Alias of [struct](#struct)._ - -### `struct` - -Returns an Arrow struct using the specified input expressions optionally named. -Fields in the returned struct use the optional name or the `cN` naming convention. -For example: `c0`, `c1`, `c2`, etc. - -``` -struct(expression1[, ..., expression_n]) -``` - -#### Arguments - -- **expression1, expression_n**: Expression to include in the output struct. Can be a constant, column, or function, any combination of arithmetic or string operators. - -#### Example - -For example, this query converts two columns `a` and `b` to a single column with -a struct type of fields `field_a` and `c1`: - -```sql -> select * from t; -+---+---+ -| a | b | -+---+---+ -| 1 | 2 | -| 3 | 4 | -+---+---+ - --- use default names `c0`, `c1` -> select struct(a, b) from t; -+-----------------+ -| struct(t.a,t.b) | -+-----------------+ -| {c0: 1, c1: 2} | -| {c0: 3, c1: 4} | -+-----------------+ - --- name the first field `field_a` -select struct(a as field_a, b) from t; -+--------------------------------------------------+ -| named_struct(Utf8("field_a"),t.a,Utf8("c1"),t.b) | -+--------------------------------------------------+ -| {field_a: 1, c1: 2} | -| {field_a: 3, c1: 4} | -+--------------------------------------------------+ -``` - -#### Aliases - -- row - -## Map Functions - -- [element_at](#element_at) -- [map](#map) -- [map_extract](#map_extract) -- [map_keys](#map_keys) -- [map_values](#map_values) - -### `element_at` - -_Alias of [map_extract](#map_extract)._ - -### `map` - -Returns an Arrow map with the specified key-value pairs. - -The `make_map` function creates a map from two lists: one for keys and one for values. Each key must be unique and non-null. - -``` -map(key, value) -map(key: value) -make_map(['key1', 'key2'], ['value1', 'value2']) -``` - -#### Arguments - -- **key**: For `map`: Expression to be used for key. Can be a constant, column, function, or any combination of arithmetic or string operators. - For `make_map`: The list of keys to be used in the map. Each key must be unique and non-null. -- **value**: For `map`: Expression to be used for value. Can be a constant, column, function, or any combination of arithmetic or string operators. - For `make_map`: The list of values to be mapped to the corresponding keys. - -#### Example - -````sql - -- Using map function - SELECT MAP('type', 'test'); - ---- - {type: test} - - SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33, null]); - ---- - {POST: 41, HEAD: 33, PATCH: } - - SELECT MAP([[1,2], [3,4]], ['a', 'b']); - ---- - {[1, 2]: a, [3, 4]: b} - - SELECT MAP { 'a': 1, 'b': 2 }; - ---- - {a: 1, b: 2} - - -- Using make_map function - SELECT MAKE_MAP(['POST', 'HEAD'], [41, 33]); - ---- - {POST: 41, HEAD: 33} - - SELECT MAKE_MAP(['key1', 'key2'], ['value1', null]); - ---- - {key1: value1, key2: } - ``` - - -### `map_extract` - -Returns a list containing the value for the given key or an empty list if the key is not present in the map. - -```` - -map_extract(map, key) - -```` -#### Arguments - -- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. -- **key**: Key to extract from the map. Can be a constant, column, or function, any combination of arithmetic or string operators, or a named expression of the previously listed. - -#### Example - -```sql -SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); ----- -[1] - -SELECT map_extract(MAP {1: 'one', 2: 'two'}, 2); ----- -['two'] - -SELECT map_extract(MAP {'x': 10, 'y': NULL, 'z': 30}, 'y'); ----- -[] -```` - -#### Aliases - -- element_at - -### `map_keys` - -Returns a list of all keys in the map. - -``` -map_keys(map) -``` - -#### Arguments - -- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. - -#### Example - -```sql -SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); ----- -[a, b, c] - -SELECT map_keys(map([100, 5], [42, 43])); ----- -[100, 5] -``` - -### `map_values` - -Returns a list of all values in the map. - -``` -map_values(map) -``` - -#### Arguments - -- **map**: Map expression. Can be a constant, column, or function, and any combination of map operators. - -#### Example - -```sql -SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); ----- -[1, , 3] - -SELECT map_values(map([100, 5], [42, 43])); ----- -[42, 43] -``` - -## Hashing Functions - -- [digest](#digest) -- [md5](#md5) -- [sha224](#sha224) -- [sha256](#sha256) -- [sha384](#sha384) -- [sha512](#sha512) - -### `digest` - -Computes the binary hash of an expression using the specified algorithm. - -``` -digest(expression, algorithm) -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. -- **algorithm**: String expression specifying algorithm to use. Must be one of: -- md5 -- sha224 -- sha256 -- sha384 -- sha512 -- blake2s -- blake2b -- blake3 - -#### Example - -```sql -> select digest('foo', 'sha256'); -+------------------------------------------+ -| digest(Utf8("foo"), Utf8("sha256")) | -+------------------------------------------+ -| | -+------------------------------------------+ -``` - -### `md5` - -Computes an MD5 128-bit checksum for a string expression. - -``` -md5(expression) -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select md5('foo'); -+-------------------------------------+ -| md5(Utf8("foo")) | -+-------------------------------------+ -| | -+-------------------------------------+ -``` - -### `sha224` - -Computes the SHA-224 hash of a binary string. - -``` -sha224(expression) -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select sha224('foo'); -+------------------------------------------+ -| sha224(Utf8("foo")) | -+------------------------------------------+ -| | -+------------------------------------------+ -``` - -### `sha256` - -Computes the SHA-256 hash of a binary string. - -``` -sha256(expression) -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select sha256('foo'); -+--------------------------------------+ -| sha256(Utf8("foo")) | -+--------------------------------------+ -| | -+--------------------------------------+ -``` - -### `sha384` - -Computes the SHA-384 hash of a binary string. - -``` -sha384(expression) -``` - -#### Arguments - -- **expression**: String expression to operate on. Can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select sha384('foo'); -+-----------------------------------------+ -| sha384(Utf8("foo")) | -+-----------------------------------------+ -| | -+-----------------------------------------+ -``` - -### `sha512` - -Computes the SHA-512 hash of a binary string. - -``` -sha512(expression) -``` - -#### Arguments - -- **expression**: String - -#### Example - -```sql -> select sha512('foo'); -+-------------------------------------------+ -| sha512(Utf8("foo")) | -+-------------------------------------------+ -| | -+-------------------------------------------+ -``` - -## Other Functions - -- [arrow_cast](#arrow_cast) -- [arrow_typeof](#arrow_typeof) -- [get_field](#get_field) -- [version](#version) - -### `arrow_cast` - -Casts a value to a specific Arrow data type. - -``` -arrow_cast(expression, datatype) -``` - -#### Arguments - -- **expression**: Expression to cast. The expression can be a constant, column, or function, and any combination of operators. -- **datatype**: [Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) name to cast to, as a string. The format is the same as that returned by [`arrow_typeof`] - -#### Example - -```sql -> select arrow_cast(-5, 'Int8') as a, - arrow_cast('foo', 'Dictionary(Int32, Utf8)') as b, - arrow_cast('bar', 'LargeUtf8') as c, - arrow_cast('2023-01-02T12:53:02', 'Timestamp(Microsecond, Some("+08:00"))') as d - ; -+----+-----+-----+---------------------------+ -| a | b | c | d | -+----+-----+-----+---------------------------+ -| -5 | foo | bar | 2023-01-02T12:53:02+08:00 | -+----+-----+-----+---------------------------+ -``` - -### `arrow_typeof` - -Returns the name of the underlying [Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) of the expression. - -``` -arrow_typeof(expression) -``` - -#### Arguments - -- **expression**: Expression to evaluate. The expression can be a constant, column, or function, and any combination of operators. - -#### Example - -```sql -> select arrow_typeof('foo'), arrow_typeof(1); -+---------------------------+------------------------+ -| arrow_typeof(Utf8("foo")) | arrow_typeof(Int64(1)) | -+---------------------------+------------------------+ -| Utf8 | Int64 | -+---------------------------+------------------------+ -``` - -### `get_field` - -Returns a field within a map or a struct with the given key. -Note: most users invoke `get_field` indirectly via field access -syntax such as `my_struct_col['field_name']` which results in a call to -`get_field(my_struct_col, 'field_name')`. - -``` -get_field(expression1, expression2) -``` - -#### Arguments - -- **expression1**: The map or struct to retrieve a field for. -- **expression2**: The field name in the map or struct to retrieve data for. Must evaluate to a string. - -#### Example - -```sql -> create table t (idx varchar, v varchar) as values ('data','fusion'), ('apache', 'arrow'); -> select struct(idx, v) from t as c; -+-------------------------+ -| struct(c.idx,c.v) | -+-------------------------+ -| {c0: data, c1: fusion} | -| {c0: apache, c1: arrow} | -+-------------------------+ -> select get_field((select struct(idx, v) from t), 'c0'); -+-----------------------+ -| struct(t.idx,t.v)[c0] | -+-----------------------+ -| data | -| apache | -+-----------------------+ -> select get_field((select struct(idx, v) from t), 'c1'); -+-----------------------+ -| struct(t.idx,t.v)[c1] | -+-----------------------+ -| fusion | -| arrow | -+-----------------------+ -``` - -### `version` - -Returns the version of DataFusion. - -``` -version() -``` - -#### Example - -```sql -> select version(); -+--------------------------------------------+ -| version() | -+--------------------------------------------+ -| Apache DataFusion 42.0.0, aarch64 on macos | -+--------------------------------------------+ -``` diff --git a/docs/source/user-guide/sql/special_functions.md b/docs/source/user-guide/sql/special_functions.md new file mode 100644 index 0000000000000..7c9efbb66218f --- /dev/null +++ b/docs/source/user-guide/sql/special_functions.md @@ -0,0 +1,100 @@ + + +# Special Functions + +## Expansion Functions + +- [unnest](#unnest) +- [unnest(struct)](#unnest-struct) + +### `unnest` + +Expands an array or map into rows. + +#### Arguments + +- **array**: Array expression to unnest. + Can be a constant, column, or function, and any combination of array operators. + +#### Examples + +```sql +> select unnest(make_array(1, 2, 3, 4, 5)) as unnested; ++----------+ +| unnested | ++----------+ +| 1 | +| 2 | +| 3 | +| 4 | +| 5 | ++----------+ +``` + +```sql +> select unnest(range(0, 10)) as unnested_range; ++----------------+ +| unnested_range | ++----------------+ +| 0 | +| 1 | +| 2 | +| 3 | +| 4 | +| 5 | +| 6 | +| 7 | +| 8 | +| 9 | ++----------------+ +``` + +### `unnest (struct)` + +Expand a struct fields into individual columns. + +#### Arguments + +- **struct**: Object expression to unnest. + Can be a constant, column, or function, and any combination of object operators. + +#### Examples + +```sql +> create table foo as values ({a: 5, b: 'a string'}), ({a:6, b: 'another string'}); + +> create view foov as select column1 as struct_column from foo; + +> select * from foov; ++---------------------------+ +| struct_column | ++---------------------------+ +| {a: 5, b: a string} | +| {a: 6, b: another string} | ++---------------------------+ + +> select unnest(struct_column) from foov; ++------------------------------------------+------------------------------------------+ +| unnest_placeholder(foov.struct_column).a | unnest_placeholder(foov.struct_column).b | ++------------------------------------------+------------------------------------------+ +| 5 | a string | +| 6 | another string | ++------------------------------------------+------------------------------------------+ +``` diff --git a/docs/source/user-guide/sql/window_functions.md b/docs/source/user-guide/sql/window_functions.md index 6bf2005dabf9b..8216a3b258b8c 100644 --- a/docs/source/user-guide/sql/window_functions.md +++ b/docs/source/user-guide/sql/window_functions.md @@ -148,28 +148,10 @@ All [aggregate functions](aggregate_functions.md) can be used as window function ## Analytical functions -- [cume_dist](#cume_dist) -- [percent_rank](#percent_rank) - [first_value](#first_value) - [last_value](#last_value) - [nth_value](#nth_value) -### `cume_dist` - -Relative rank of the current row: (number of rows preceding or peer with current row) / (total rows). - -```sql -cume_dist() -``` - -### `percent_rank` - -Relative rank of the current row: (rank - 1) / (total rows - 1). - -```sql -percent_rank() -``` - ### `first_value` Returns value evaluated at the row that is the first row of the window frame. diff --git a/test-utils/src/array_gen/primitive.rs b/test-utils/src/array_gen/primitive.rs index f70ebf6686d00..0581862d63bd6 100644 --- a/test-utils/src/array_gen/primitive.rs +++ b/test-utils/src/array_gen/primitive.rs @@ -15,14 +15,45 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, PrimitiveArray, UInt32Array}; -use arrow::datatypes::{ - Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, UInt16Type, - UInt32Type, UInt64Type, UInt8Type, -}; +use arrow::array::{ArrayRef, ArrowPrimitiveType, PrimitiveArray, UInt32Array}; +use arrow::datatypes::DataType; +use rand::distributions::Standard; +use rand::prelude::Distribution; use rand::rngs::StdRng; use rand::Rng; +/// Trait for converting type safely from a native type T impl this trait. +pub trait FromNative: std::fmt::Debug + Send + Sync + Copy + Default { + /// Convert native type from i64. + fn from_i64(_: i64) -> Option { + None + } +} + +macro_rules! native_type { + ($t: ty $(, $from:ident)*) => { + impl FromNative for $t { + $( + #[inline] + fn $from(v: $t) -> Option { + Some(v) + } + )* + } + }; +} + +native_type!(i8); +native_type!(i16); +native_type!(i32); +native_type!(i64, from_i64); +native_type!(u8); +native_type!(u16); +native_type!(u32); +native_type!(u64); +native_type!(f32); +native_type!(f64); + /// Randomly generate primitive array pub struct PrimitiveArrayGenerator { /// the total number of strings in the output @@ -35,46 +66,61 @@ pub struct PrimitiveArrayGenerator { pub rng: StdRng, } -macro_rules! impl_gen_data { - ($NATIVE_TYPE:ty, $ARROW_TYPE:ident) => { - paste::paste! { - pub fn [< gen_data_ $NATIVE_TYPE >](&mut self) -> ArrayRef { - // table of strings from which to draw - let distinct_primitives: PrimitiveArray<$ARROW_TYPE> = (0..self.num_distinct_primitives) - .map(|_| Some(self.rng.gen::<$NATIVE_TYPE>())) - .collect(); +// TODO: support generating more primitive arrays +impl PrimitiveArrayGenerator { + pub fn gen_data(&mut self) -> ArrayRef + where + A: ArrowPrimitiveType, + A::Native: FromNative, + Standard: Distribution<::Native>, + { + // table of primitives from which to draw + let distinct_primitives: PrimitiveArray = (0..self.num_distinct_primitives) + .map(|_| { + Some(match A::DATA_TYPE { + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 + | DataType::Float32 + | DataType::Float64 + | DataType::Date32 => self.rng.gen::(), - // pick num_strings randomly from the distinct string table - let indicies: UInt32Array = (0..self.num_primitives) - .map(|_| { - if self.rng.gen::() < self.null_pct { - None - } else if self.num_distinct_primitives > 1 { - let range = 1..(self.num_distinct_primitives as u32); - Some(self.rng.gen_range(range)) - } else { - Some(0) - } - }) - .collect(); + DataType::Date64 => { + // TODO: constrain this range to valid dates if necessary + let date_value = self.rng.gen_range(i64::MIN..=i64::MAX); + let millis_per_day = 86_400_000; + let adjusted_value = date_value - (date_value % millis_per_day); + A::Native::from_i64(adjusted_value).unwrap() + } - let options = None; - arrow::compute::take(&distinct_primitives, &indicies, options).unwrap() - } - } - }; -} + _ => { + let arrow_type = A::DATA_TYPE; + panic!("Unsupported arrow data type: {arrow_type}") + } + }) + }) + .collect(); -// TODO: support generating more primitive arrays -impl PrimitiveArrayGenerator { - impl_gen_data!(i8, Int8Type); - impl_gen_data!(i16, Int16Type); - impl_gen_data!(i32, Int32Type); - impl_gen_data!(i64, Int64Type); - impl_gen_data!(u8, UInt8Type); - impl_gen_data!(u16, UInt16Type); - impl_gen_data!(u32, UInt32Type); - impl_gen_data!(u64, UInt64Type); - impl_gen_data!(f32, Float32Type); - impl_gen_data!(f64, Float64Type); + // pick num_primitves randomly from the distinct string table + let indicies: UInt32Array = (0..self.num_primitives) + .map(|_| { + if self.rng.gen::() < self.null_pct { + None + } else if self.num_distinct_primitives > 1 { + let range = 1..(self.num_distinct_primitives as u32); + Some(self.rng.gen_range(range)) + } else { + Some(0) + } + }) + .collect(); + + let options = None; + arrow::compute::take(&distinct_primitives, &indicies, options).unwrap() + } }