diff --git a/.github/workflows/audit.yml b/.github/workflows/audit.yml index a6eec722ed3e1..bca3390370175 100644 --- a/.github/workflows/audit.yml +++ b/.github/workflows/audit.yml @@ -45,7 +45,7 @@ jobs: steps: - uses: actions/checkout@df4cb1c069e1874edd31b4311f1884172cec0e10 # v6.0.3 - name: Install cargo-audit - uses: taiki-e/install-action@0631aa6515c7d545823c67cfae7ef4fc7f490154 # v2.81.8 + uses: taiki-e/install-action@15449e3094499af05d8d964a1c884208e4b8b595 # v2.81.11 with: tool: cargo-audit - name: Run audit check diff --git a/.github/workflows/breaking_changes_detector.yml b/.github/workflows/breaking_changes_detector.yml index 264ba16e3c8d0..a0d7bf6520ee5 100644 --- a/.github/workflows/breaking_changes_detector.yml +++ b/.github/workflows/breaking_changes_detector.yml @@ -89,7 +89,7 @@ jobs: - name: Install cargo-semver-checks if: steps.changed_crates.outputs.packages != '' - uses: taiki-e/install-action@0631aa6515c7d545823c67cfae7ef4fc7f490154 # v2.81.8 + uses: taiki-e/install-action@15449e3094499af05d8d964a1c884208e4b8b595 # v2.81.11 with: tool: cargo-semver-checks diff --git a/.github/workflows/dependencies.yml b/.github/workflows/dependencies.yml index 47948ac5c8b9d..d43ca1d0a9a55 100644 --- a/.github/workflows/dependencies.yml +++ b/.github/workflows/dependencies.yml @@ -50,9 +50,9 @@ jobs: with: rust-version: stable - name: Check dependencies + working-directory: dev/depcheck run: | - cd dev/depcheck - cargo run + cargo run --locked detect-unused-dependencies: name: Detect Unused Dependencies diff --git a/.github/workflows/dev.yml b/.github/workflows/dev.yml index 48f5ea8939c8c..43da64c569162 100644 --- a/.github/workflows/dev.yml +++ b/.github/workflows/dev.yml @@ -64,7 +64,7 @@ jobs: source ci/scripts/utils/tool_versions.sh echo "LYCHEE_VERSION=${LYCHEE_VERSION}" >> "$GITHUB_ENV" - name: Install lychee - uses: taiki-e/install-action@0631aa6515c7d545823c67cfae7ef4fc7f490154 # v2.81.8 + uses: taiki-e/install-action@15449e3094499af05d8d964a1c884208e4b8b595 # v2.81.11 with: tool: lychee@${{ env.LYCHEE_VERSION }} - name: Run markdown link check diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 770e705dddc90..cc48a54f27ec4 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -429,7 +429,7 @@ jobs: sudo apt-get update -qq sudo apt-get install -y -qq clang - name: Setup wasm-pack - uses: taiki-e/install-action@0631aa6515c7d545823c67cfae7ef4fc7f490154 # v2.81.8 + uses: taiki-e/install-action@15449e3094499af05d8d964a1c884208e4b8b595 # v2.81.11 with: tool: wasm-pack - name: Run tests with headless mode @@ -773,7 +773,7 @@ jobs: - name: Setup Rust toolchain uses: ./.github/actions/setup-builder - name: Install cargo-msrv - uses: taiki-e/install-action@0631aa6515c7d545823c67cfae7ef4fc7f490154 # v2.81.8 + uses: taiki-e/install-action@15449e3094499af05d8d964a1c884208e4b8b595 # v2.81.11 with: tool: cargo-msrv diff --git a/Cargo.lock b/Cargo.lock index ed90dd25bda7b..9e70bf18ae5b7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3721,9 +3721,9 @@ dependencies = [ [[package]] name = "insta-cmd" -version = "0.6.0" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffeeefa927925cced49ccb01bf3e57c9d4cd132df21e576eb9415baeab2d3de6" +checksum = "bffdf4af1db390cf0401535d7c1303cd079a074d28d8473b026fdb6559c41403" dependencies = [ "insta", "serde", @@ -4698,9 +4698,9 @@ dependencies = [ [[package]] name = "postgres-derive" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca1dad89d9ffdbf78502fde418eeede499b87772d88be780478f7f76dc8d471f" +checksum = "4d9d9089bb0ce62f4b5d52a0be0f4acfb35738b979380670d3dea85fe38d6ddd" dependencies = [ "heck", "proc-macro2", @@ -4710,9 +4710,9 @@ dependencies = [ [[package]] name = "postgres-protocol" -version = "0.6.11" +version = "0.6.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56201207dac53e2f38e848e31b4b91616a6bb6e0c7205b77718994a7f49e70fc" +checksum = "08808e3c483c46e999108051c78334f473d5adb59d78bb80a1268c7e6aa6c514" dependencies = [ "base64 0.22.1", "byteorder", @@ -4728,9 +4728,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.13" +version = "0.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8dc729a129e682e8d24170cd30ae1aa01b336b096cbb56df6d534ffec133d186" +checksum = "851ca9db4932932d69f3ea811b1abe63087a0f740a47692619dd40d4899b68be" dependencies = [ "bytes", "chrono", @@ -4803,9 +4803,9 @@ dependencies = [ [[package]] name = "prost" -version = "0.14.3" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2ea70524a2f82d518bce41317d0fae74151505651af45faf1ffbd6fd33f0568" +checksum = "528ac67416ff8646872a3c02cad9cc4ee5dc9f9540c9b10771855c95cb2e5ae1" dependencies = [ "bytes", "prost-derive", @@ -4813,12 +4813,12 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.14.3" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "343d3bd7056eda839b03204e68deff7d1b13aba7af2b2fd16890697274262ee7" +checksum = "03da047801ff44bb6a4d407d4860c05fd70bb81714e6b2f3812603d5b145b042" dependencies = [ "heck", - "itertools 0.13.0", + "itertools 0.14.0", "log", "multimap", "petgraph", @@ -4832,12 +4832,12 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.14.3" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27c6023962132f4b30eb4c172c91ce92d933da334c59c23cddee82358ddafb0b" +checksum = "b570b25f7617e43d59005d0990ccb79e950a423952cea19671b7a876da390adf" dependencies = [ "anyhow", - "itertools 0.13.0", + "itertools 0.14.0", "proc-macro2", "quote", "syn 2.0.117", @@ -4845,9 +4845,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.14.3" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8991c4cbdb8bc5b11f0b074ffe286c30e523de90fee5ba8132f1399f23cb3dd7" +checksum = "f94967dc7688f3054c7fac87473ffae4cc4c3904800e2d9f5b857246d8963b0a" dependencies = [ "prost", ] @@ -6236,9 +6236,9 @@ dependencies = [ [[package]] name = "tokio-postgres" -version = "0.7.17" +version = "0.7.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dd8df5ef180f6364759a6f00f7aadda4fbbac86cdee37480826a6ff9f3574ce" +checksum = "a528f7d280f6d5b9cd149635c8705b0dd049754bc67d81d31fa25169a93809d3" dependencies = [ "async-trait", "byteorder", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 441ae00c11db0..62eedafe798d4 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -75,7 +75,7 @@ workspace = true [dev-dependencies] ctor = { workspace = true } insta = { workspace = true } -insta-cmd = "0.6.0" +insta-cmd = "0.7.0" rstest = { workspace = true } testcontainers-modules = { workspace = true, features = ["minio"] } # Makes sure `test_display_pg_json` behaves in a consistent way regardless of diff --git a/datafusion-cli/src/exec.rs b/datafusion-cli/src/exec.rs index f7d1541b93bff..f43854821b2d5 100644 --- a/datafusion-cli/src/exec.rs +++ b/datafusion-cli/src/exec.rs @@ -23,7 +23,7 @@ use crate::print_format::PrintFormat; use crate::{ command::{Command, OutputFormat}, helper::CliHelper, - object_storage::get_object_store, + object_storage::{get_object_store, stdin::StdinUtils}, print_options::{MaxRows, PrintOptions}, }; use datafusion::common::instant::Instant; @@ -417,9 +417,14 @@ async fn create_plan( // Note that cmd is a mutable reference so that create_external_table function can remove all // datafusion-cli specific options before passing through to datafusion. Otherwise, datafusion // will raise Configuration errors. - if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &plan { + if let LogicalPlan::Ddl(DdlStatement::CreateExternalTable(cmd)) = &mut plan { // To support custom formats, treat error as None let format = config_file_type_from_str(&cmd.file_type); + + // Expose stdin (e.g. `cat data.csv | datafusion-cli`) as a `stdin://` + // object store, registered like any other scheme in `get_object_store`. + cmd.location = StdinUtils::rewrite_location(&cmd.location, format.as_ref()); + register_object_store_and_config_extensions( ctx, &cmd.location, diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 0d8ada1367826..4646c5cce9380 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -37,6 +37,7 @@ use datafusion_cli::functions::{ use datafusion_cli::object_storage::instrumented::{ InstrumentedObjectStoreMode, InstrumentedObjectStoreRegistry, }; +use datafusion_cli::object_storage::{StdinCarriesCommands, is_stdin_location}; use datafusion_cli::{ DATAFUSION_CLI_VERSION, exec, pool_type::PoolType, @@ -158,6 +159,23 @@ struct Args { object_store_profiling: InstrumentedObjectStoreMode, } +impl Args { + /// Without -c/-f the CLI enters the REPL, which reads its SQL from + /// stdin — interactively or piped. + fn repl_mode(&self) -> bool { + self.command.is_empty() && self.file.is_empty() + } + + /// Whether the CLI consumes stdin for its own SQL input. This covers the + /// REPL (no -c/-f, reading SQL interactively or piped) as well as an + /// explicit `-f /dev/stdin` (or the other stdin pseudo-paths), where the + /// SQL file *is* stdin. In either case stdin is already spoken for and + /// cannot also back a `LOCATION '/dev/stdin'` table. + fn reads_sql_from_stdin(&self) -> bool { + self.repl_mode() || self.file.iter().any(|f| is_stdin_location(f)) + } +} + #[tokio::main] /// Calls [`main_inner`], then handles printing errors and returning the correct exit code pub async fn main() -> ExitCode { @@ -268,6 +286,7 @@ async fn main_inner() -> Result<()> { instrumented_registry: Arc::clone(&instrumented_registry), }; + let repl_mode = args.repl_mode(); let commands = args.command; let files = args.file; let rc = match args.rc { @@ -285,7 +304,7 @@ async fn main_inner() -> Result<()> { } }; - if commands.is_empty() && files.is_empty() { + if repl_mode { if !rc.is_empty() { exec::exec_from_files(&ctx, rc, &print_options).await?; } @@ -330,8 +349,16 @@ fn get_session_config(args: &Args) -> Result { config_options.format.null = String::from("NULL"); } - let session_config = + let mut session_config = SessionConfig::from(config_options).with_information_schema(true); + + if args.reads_sql_from_stdin() { + // When stdin carries the session's SQL — the REPL (including any rc + // file run before it) or an explicit `-f /dev/stdin` — it cannot also + // serve as a data source for `LOCATION '/dev/stdin'`. + session_config = session_config.with_extension(Arc::new(StdinCarriesCommands)); + } + Ok(session_config) } diff --git a/datafusion-cli/src/object_storage.rs b/datafusion-cli/src/object_storage.rs index 34787838929f1..4293788e0c03a 100644 --- a/datafusion-cli/src/object_storage.rs +++ b/datafusion-cli/src/object_storage.rs @@ -16,6 +16,9 @@ // under the License. pub mod instrumented; +pub(crate) mod stdin; + +pub use stdin::{StdinCarriesCommands, is_stdin_location}; use async_trait::async_trait; use aws_config::BehaviorVersion; @@ -564,6 +567,9 @@ pub(crate) async fn get_object_store( .with_url(url.origin().ascii_serialization()) .build()?, ), + _ if scheme == stdin::StdinUtils::SCHEME => { + stdin::StdinUtils::get_or_create(state, url).await? + } _ => { // For other types, try to get from `object_store_registry`: state diff --git a/datafusion-cli/src/object_storage/stdin.rs b/datafusion-cli/src/object_storage/stdin.rs new file mode 100644 index 0000000000000..602b00a9b90f6 --- /dev/null +++ b/datafusion-cli/src/object_storage/stdin.rs @@ -0,0 +1,377 @@ +// 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. + +//! Exposes the process's standard input as a `stdin://` object store so that +//! piped data (e.g. `cat data.csv | datafusion-cli`) can be queried via +//! `CREATE EXTERNAL TABLE ... LOCATION '/dev/stdin'`. + +use std::io::{IsTerminal, Read}; +use std::sync::Arc; + +use datafusion::common::exec_datafusion_err; +use datafusion::config::ConfigFileType; +use datafusion::error::Result; +use datafusion::execution::context::SessionState; +use futures::TryStreamExt; + +use object_store::memory::InMemory; +use object_store::path::Path as ObjectStorePath; +use object_store::{ObjectStore, ObjectStoreExt}; +use url::Url; + +/// Marker [`SessionConfig`] extension recording that the session reads its SQL +/// commands from stdin (the interactive or piped REPL). stdin cannot then also +/// serve as a data source: reading it for table data would silently consume +/// the remaining SQL statements. +/// +/// [`SessionConfig`]: datafusion::execution::context::SessionConfig +#[derive(Debug)] +pub struct StdinCarriesCommands; + +/// Filesystem paths that refer to the process's standard input. +/// +/// These are intentionally limited to the well known pseudo-files exposed by +/// the operating system so that ordinary files are never accidentally treated +/// as stdin. +const STDIN_LOCATIONS: [&str; 3] = ["/dev/stdin", "/dev/fd/0", "/proc/self/fd/0"]; + +/// Returns `true` if `path` refers to the process's standard input. +/// +/// Re-exported as [`crate::object_storage::is_stdin_location`] so the CLI entry +/// point can detect when it reads its SQL from stdin via `-f /dev/stdin` and +/// avoid also offering stdin as a `LOCATION '/dev/stdin'` data source. +pub fn is_stdin_location(path: &str) -> bool { + STDIN_LOCATIONS.contains(&path) +} + +/// Utilities for exposing the process's standard input as an object store. +/// +/// stdin is surfaced as a `stdin://` object store and dispatched alongside the +/// other schemes (`s3`, `gs`, `http`, ...) so that reading piped data flows +/// through the normal object-store/listing code path, conceptually similar to +/// DuckDB's `PipeFileSystem`. +pub(crate) struct StdinUtils; + +impl StdinUtils { + /// The URL scheme used to expose stdin as an object store, mirroring how + /// `s3`, `gs`, `http`, etc. are addressed. + pub(crate) const SCHEME: &'static str = "stdin"; + + /// Rewrites the well known stdin pseudo-paths (e.g. `/dev/stdin`) to a + /// canonical `stdin://` URL so that reading from standard input flows + /// through the same object-store/listing code path as any other scheme. + /// Non-stdin locations are returned unchanged. + /// + /// The listing layer filters candidate files by extension, so the canonical + /// object is named with the extension matching the declared `STORED AS` + /// format. The name thereby also records which format stdin was consumed + /// as: a later stdin-backed table declaring a different format resolves to + /// a path the buffered store does not contain and is rejected by + /// [`Self::get_or_create`]. + pub(crate) fn rewrite_location( + location: &str, + format: Option<&ConfigFileType>, + ) -> String { + if !is_stdin_location(location) { + return location.to_string(); + } + + let object_name = match format { + Some(ConfigFileType::CSV) => "stdin.csv", + Some(ConfigFileType::JSON) => "stdin.json", + Some(ConfigFileType::PARQUET) => "stdin.parquet", + _ => "stdin", + }; + format!("{}:///{object_name}", Self::SCHEME) + } + + /// Returns the object store backing the `stdin://` scheme, reading and + /// buffering standard input on first use and reusing that buffer for any + /// subsequent `stdin://` table created in the same session. + /// + /// stdin is a one-shot stream: it can only be read once. The object store + /// registry keys by scheme/authority, so every `stdin://` URL maps to the + /// same store. Without this guard, a second `CREATE EXTERNAL TABLE ... + /// LOCATION '/dev/stdin'` would re-read (now-EOF) stdin, build an empty + /// store, and overwrite the populated one, silently emptying the earlier + /// table. Reusing the already-registered store avoids that. + /// + /// A later stdin-backed table declaring a different `STORED AS` format + /// resolves to an object the store does not contain (the object name + /// records the format stdin was consumed as) and is rejected with a clear + /// error — both reading the buffer as another format and re-reading stdin + /// would be silently wrong. + pub(crate) async fn get_or_create( + state: &SessionState, + url: &Url, + ) -> Result> { + let Ok(existing) = state.runtime_env().object_store_registry.get_store(url) + else { + return Self::object_store(state, url).await; + }; + + let path = ObjectStorePath::from_url_path(url.path())?; + if existing.head(&path).await.is_err() { + let buffered = existing + .list(None) + .try_next() + .await + .ok() + .flatten() + .map(|meta| format!(" as '{}'", meta.location)) + .unwrap_or_default(); + return Err(exec_datafusion_err!( + "stdin was already read{buffered} by an earlier statement; all \ + tables backed by stdin in a session must declare the same \ + STORED AS format" + )); + } + Ok(existing) + } + + /// Builds the object store backing the `stdin://` scheme by reading all of + /// standard input into memory. + /// + /// A pipe (e.g. `cat data.csv | datafusion-cli`) is not seekable and reports + /// a size of `0`, so it cannot be read directly by the file based formats + /// (CSV requires seeking, Parquet needs the footer at the end of the file). + /// Buffering the whole input up front sidesteps these limitations and lets + /// the data be read like any other object, including being scanned more than + /// once. + async fn object_store( + state: &SessionState, + url: &Url, + ) -> Result> { + if state + .config() + .get_extension::() + .is_some() + { + return Err(exec_datafusion_err!( + "stdin is already being read for SQL commands, so it cannot \ + also supply table data; pass the query with -c/--command or \ + -f/--file so that stdin carries the data, e.g. \ + `cat data.csv | datafusion-cli -f query.sql`" + )); + } + if std::io::stdin().is_terminal() { + return Err(exec_datafusion_err!( + "stdin is connected to a terminal, not piped data; pipe the \ + input in, e.g. `cat data.csv | datafusion-cli -f query.sql`" + )); + } + + let mut buffer = Vec::new(); + std::io::stdin() + .lock() + .read_to_end(&mut buffer) + .map_err(|e| exec_datafusion_err!("Failed to read from stdin: {e}"))?; + Self::in_memory_object_store(url, buffer).await + } + + /// Stores `data` at the path referenced by `url` in a fresh [`InMemory`] + /// store. + async fn in_memory_object_store( + url: &Url, + data: Vec, + ) -> Result> { + let store = InMemory::new(); + store + .put(&ObjectStorePath::from_url_path(url.path())?, data.into()) + .await?; + Ok(Arc::new(store)) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + use datafusion::prelude::{SessionConfig, SessionContext}; + + #[test] + fn rewrites_stdin_locations() { + // stdin pseudo-paths are rewritten to a `stdin://` URL carrying the + // extension that matches the declared format. + assert_eq!( + StdinUtils::rewrite_location("/dev/stdin", Some(&ConfigFileType::CSV)), + "stdin:///stdin.csv" + ); + assert_eq!( + StdinUtils::rewrite_location("/dev/fd/0", Some(&ConfigFileType::JSON)), + "stdin:///stdin.json" + ); + assert_eq!( + StdinUtils::rewrite_location( + "/proc/self/fd/0", + Some(&ConfigFileType::PARQUET) + ), + "stdin:///stdin.parquet" + ); + assert_eq!( + StdinUtils::rewrite_location("/dev/stdin", None), + "stdin:///stdin" + ); + + // Ordinary locations are left untouched. + for location in ["/dev/stdout", "data/stdin.csv", "stdin", "s3://b/f.csv"] { + assert_eq!( + StdinUtils::rewrite_location(location, Some(&ConfigFileType::CSV)), + location + ); + } + } + + /// Buffers `data` into the `stdin://` object store and reads it back through + /// a `CREATE EXTERNAL TABLE`, returning the number of rows in the table. + /// + /// This exercises the full path used for `/dev/stdin` short of the actual + /// stdin read, which cannot be driven from a unit test. + async fn count_stdin_rows( + data: Vec, + stored_as: &str, + format: Option, + options: &str, + ) -> Result { + let location = StdinUtils::rewrite_location("/dev/stdin", format.as_ref()); + let url = Url::parse(&location).unwrap(); + let store = StdinUtils::in_memory_object_store(&url, data).await?; + + let ctx = SessionContext::new(); + ctx.register_object_store(&url, store); + ctx.sql(&format!( + "CREATE EXTERNAL TABLE t STORED AS {stored_as} LOCATION '{location}' {options}" + )) + .await? + .collect() + .await?; + + ctx.sql("SELECT * FROM t").await?.count().await + } + + #[tokio::test] + async fn reuses_buffered_stdin_store() -> Result<()> { + // stdin can only be read once, so a second `stdin://` table must reuse + // the store buffered by the first instead of re-reading (now-empty) + // stdin and overwriting it. + let url = Url::parse("stdin:///stdin.csv").unwrap(); + let store = + StdinUtils::in_memory_object_store(&url, b"a\n1\n2\n".to_vec()).await?; + + let ctx = SessionContext::new(); + ctx.register_object_store(&url, store); + + let reused = StdinUtils::get_or_create(&ctx.state(), &url).await?; + let path = ObjectStorePath::from_url_path(url.path())?; + let bytes = reused.get(&path).await?.bytes().await?; + assert_eq!(bytes.as_ref(), b"a\n1\n2\n"); + Ok(()) + } + + #[tokio::test] + async fn rejects_second_stdin_table_with_different_format() -> Result<()> { + // The buffered object's name records the format stdin was consumed + // as; a later stdin table declaring a different format must fail with + // a clear error rather than a downstream "not found" (or silently + // misreading the bytes as another format). + let csv_url = Url::parse("stdin:///stdin.csv").unwrap(); + let store = + StdinUtils::in_memory_object_store(&csv_url, b"a\n1\n".to_vec()).await?; + + let ctx = SessionContext::new(); + ctx.register_object_store(&csv_url, store); + + let json_url = Url::parse("stdin:///stdin.json").unwrap(); + let err = StdinUtils::get_or_create(&ctx.state(), &json_url) + .await + .unwrap_err() + .to_string(); + assert!( + err.contains("must declare the same STORED AS format") + && err.contains("stdin.csv"), + "unexpected error: {err}" + ); + Ok(()) + } + + #[tokio::test] + async fn errors_when_stdin_carries_commands() { + // Once the REPL owns stdin for SQL commands, building the stdin store + // must fail with a clear error instead of swallowing the remaining + // statements as table data. + let config = SessionConfig::new().with_extension(Arc::new(StdinCarriesCommands)); + let ctx = SessionContext::new_with_config(config); + + let url = Url::parse("stdin:///stdin.csv").unwrap(); + let err = StdinUtils::get_or_create(&ctx.state(), &url) + .await + .unwrap_err(); + assert!( + err.to_string().contains("SQL commands"), + "unexpected error: {err}" + ); + } + + #[tokio::test] + async fn stdin_object_store_reads_csv() -> Result<()> { + let data = b"a,b\n1,foo\n2,bar\n".to_vec(); + let rows = count_stdin_rows( + data, + "CSV", + Some(ConfigFileType::CSV), + "OPTIONS ('format.has_header' 'true')", + ) + .await?; + assert_eq!(rows, 2); + Ok(()) + } + + #[tokio::test] + async fn stdin_object_store_reads_json() -> Result<()> { + let data = b"{\"a\": 1, \"b\": \"foo\"}\n{\"a\": 2, \"b\": \"bar\"}\n".to_vec(); + let rows = count_stdin_rows(data, "JSON", Some(ConfigFileType::JSON), "").await?; + assert_eq!(rows, 2); + Ok(()) + } + + #[tokio::test] + async fn stdin_object_store_reads_parquet() -> Result<()> { + use datafusion::arrow::array::Int32Array; + use datafusion::arrow::datatypes::{DataType, Field, Schema}; + use datafusion::arrow::record_batch::RecordBatch; + use parquet::arrow::ArrowWriter; + + // Parquet requires random access to the footer, which a real pipe cannot + // provide; the in-memory buffer makes this work. + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + ) + .unwrap(); + + let mut data = Vec::new(); + let mut writer = ArrowWriter::try_new(&mut data, schema, None).unwrap(); + writer.write(&batch).unwrap(); + writer.close().unwrap(); + + let rows = + count_stdin_rows(data, "PARQUET", Some(ConfigFileType::PARQUET), "").await?; + assert_eq!(rows, 3); + Ok(()) + } +} diff --git a/datafusion-cli/tests/cli_integration.rs b/datafusion-cli/tests/cli_integration.rs index 4849ac9e9a5e2..4dc244445a2eb 100644 --- a/datafusion-cli/tests/cli_integration.rs +++ b/datafusion-cli/tests/cli_integration.rs @@ -173,6 +173,222 @@ fn cli_quick_test<'a>( assert_cmd_snapshot!(cmd); } +/// Read data piped into the CLI via the `/dev/stdin` pseudo-path. +/// +/// Unix-only: `/dev/stdin` does not exist on Windows. This drives the real +/// binary through an actual pipe, exercising the stdin read that the in-process +/// unit tests cannot. +#[cfg(unix)] +#[test] +fn test_cli_read_from_stdin() { + let stdout = run_cli_with_stdin( + "CREATE EXTERNAL TABLE t STORED AS CSV LOCATION '/dev/stdin' \ + OPTIONS ('format.has_header' 'true'); \ + SELECT b, count(*) AS c FROM t GROUP BY b ORDER BY b;", + b"a,b\n1,foo\n2,bar\n3,foo\n", + ); + + assert!( + stdout.contains("| foo | 2 |") && stdout.contains("| bar | 1 |"), + "unexpected output:\n{stdout}" + ); +} + +/// stdin is a one-shot stream, so a second `/dev/stdin` table in the same +/// session must reuse the buffered input rather than re-reading (now-empty) +/// stdin and silently emptying the first table. +#[cfg(unix)] +#[test] +fn test_cli_read_from_stdin_twice_reuses_buffer() { + let stdout = run_cli_with_stdin( + "CREATE EXTERNAL TABLE t STORED AS CSV LOCATION '/dev/stdin' \ + OPTIONS ('format.has_header' 'true'); \ + CREATE EXTERNAL TABLE t2 STORED AS CSV LOCATION '/dev/stdin' \ + OPTIONS ('format.has_header' 'true'); \ + SELECT count(*) AS t_count FROM t; \ + SELECT count(*) AS t2_count FROM t2;", + b"a,b\n1,foo\n2,bar\n", + ); + + // Both tables must still see the two buffered rows. + let counts: Vec<&str> = stdout + .lines() + .filter(|line| line.trim_start().starts_with("| 2 ")) + .collect(); + assert_eq!( + counts.len(), + 2, + "expected both stdin tables to report 2 rows, got:\n{stdout}" + ); +} + +/// A later `/dev/stdin` table declaring a different `STORED AS` format must be +/// rejected with a clear error: stdin is one-shot, its bytes were already +/// buffered under the first table's format, and silently reading them as +/// another format would be wrong. +#[cfg(unix)] +#[test] +fn test_cli_read_from_stdin_mixed_formats_rejected() { + use std::io::Write; + use std::process::Stdio; + + let mut child = cli() + .args([ + "-q", + "--command", + "CREATE EXTERNAL TABLE t STORED AS CSV LOCATION '/dev/stdin' \ + OPTIONS ('format.has_header' 'true'); \ + CREATE EXTERNAL TABLE t2 STORED AS JSON LOCATION '/dev/stdin';", + ]) + .stdin(Stdio::piped()) + .stdout(Stdio::piped()) + .stderr(Stdio::piped()) + .spawn() + .expect("failed to spawn datafusion-cli"); + + child + .stdin + .take() + .unwrap() + .write_all(b"a,b\n1,foo\n2,bar\n") + .unwrap(); + + let output = child.wait_with_output().unwrap(); + // Fatal errors in `--command` mode are reported on stdout. + let stdout = String::from_utf8_lossy(&output.stdout); + + assert!( + !output.status.success(), + "expected the mismatched format to fail, stdout:\n{stdout}" + ); + assert!( + stdout.contains("must declare the same STORED AS format"), + "expected a clear mismatch error, got:\n{stdout}" + ); +} + +/// When the SQL itself arrives on stdin (the piped REPL, e.g. `cat script.sql +/// | datafusion-cli`), stdin cannot double as a data source: the statement +/// must fail with a clear error instead of silently consuming the rest of the +/// script as table data, and the remaining statements must still run. +#[cfg(unix)] +#[test] +fn test_cli_stdin_location_rejected_when_sql_comes_from_stdin() { + use std::io::Write; + use std::process::Stdio; + + let mut child = cli() + .arg("-q") + .stdin(Stdio::piped()) + .stdout(Stdio::piped()) + .stderr(Stdio::piped()) + .spawn() + .expect("failed to spawn datafusion-cli"); + + child + .stdin + .take() + .unwrap() + .write_all( + b"CREATE EXTERNAL TABLE t STORED AS CSV LOCATION '/dev/stdin';\n\ + SELECT 123 + 456;\n", + ) + .unwrap(); + + let output = child.wait_with_output().unwrap(); + let stdout = String::from_utf8_lossy(&output.stdout); + let stderr = String::from_utf8_lossy(&output.stderr); + + assert!( + stderr.contains("SQL commands"), + "expected a clear error about stdin carrying SQL.\nstdout:\n{stdout}\nstderr:\n{stderr}" + ); + // The statement after the failed CREATE must still execute rather than + // being consumed as table data. + assert!( + stdout.contains("579"), + "expected the following statement to still run.\nstdout:\n{stdout}\nstderr:\n{stderr}" + ); +} + +/// `-f /dev/stdin` reads the SQL script from stdin, exactly like the piped +/// REPL, so stdin still cannot double as a `LOCATION '/dev/stdin'` data source. +/// The offending statement must fail with the same clear error, and later +/// statements in the script must still run. +/// +/// `/dev/stdin` only passes the `-f` file check when stdin is a redirected +/// regular file (a pipe is not `is_file()`), so the binary is driven with a +/// temp script file as its stdin rather than a pipe. +#[cfg(unix)] +#[test] +fn test_cli_dash_f_stdin_location_rejected() { + use std::process::Stdio; + + let script = env::temp_dir().join(format!( + "datafusion_cli_dash_f_stdin_{}.sql", + std::process::id() + )); + fs::write( + &script, + b"CREATE EXTERNAL TABLE t STORED AS CSV LOCATION '/dev/stdin';\n\ + SELECT 123 + 456;\n", + ) + .unwrap(); + let stdin = fs::File::open(&script).unwrap(); + + let output = cli() + .args(["-q", "-f", "/dev/stdin"]) + .stdin(Stdio::from(stdin)) + .stdout(Stdio::piped()) + .stderr(Stdio::piped()) + .output() + .expect("failed to spawn datafusion-cli"); + + let _ = fs::remove_file(&script); + + let stdout = String::from_utf8_lossy(&output.stdout); + let stderr = String::from_utf8_lossy(&output.stderr); + + assert!( + stderr.contains("SQL commands"), + "expected a clear error about stdin carrying SQL.\nstdout:\n{stdout}\nstderr:\n{stderr}" + ); + // The statement after the failed CREATE must still execute rather than + // being consumed as table data. + assert!( + stdout.contains("579"), + "expected the following statement to still run.\nstdout:\n{stdout}\nstderr:\n{stderr}" + ); +} + +/// Spawns the real `datafusion-cli` binary, pipes `stdin` into it, and returns +/// its stdout after asserting a successful exit. +#[cfg(unix)] +fn run_cli_with_stdin(command: &str, stdin: &[u8]) -> String { + use std::io::Write; + use std::process::Stdio; + + let mut child = cli() + .args(["-q", "--command", command]) + .stdin(Stdio::piped()) + .stdout(Stdio::piped()) + .stderr(Stdio::piped()) + .spawn() + .expect("failed to spawn datafusion-cli"); + + child.stdin.take().unwrap().write_all(stdin).unwrap(); + + let output = child.wait_with_output().unwrap(); + let stdout = String::from_utf8_lossy(&output.stdout).into_owned(); + let stderr = String::from_utf8_lossy(&output.stderr); + + assert!( + output.status.success(), + "datafusion-cli failed.\nstdout:\n{stdout}\nstderr:\n{stderr}" + ); + stdout +} + #[test] fn cli_explain_environment_overrides() { let mut settings = make_settings(); diff --git a/datafusion-examples/examples/udf/advanced_udaf.rs b/datafusion-examples/examples/udf/advanced_udaf.rs index f1651dbf28913..b990740159906 100644 --- a/datafusion-examples/examples/udf/advanced_udaf.rs +++ b/datafusion-examples/examples/udf/advanced_udaf.rs @@ -268,7 +268,6 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&arrow::array::BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 2, "two arguments to merge_batch"); @@ -280,7 +279,7 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { self.null_state.accumulate( group_indices, partial_counts, - opt_filter, + None, total_num_groups, |group_index, partial_count| { self.counts[group_index] += partial_count; @@ -292,7 +291,7 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { self.null_state.accumulate( group_indices, partial_prods, - opt_filter, + None, total_num_groups, |group_index, new_value: ::Native| { let prod = &mut self.prods[group_index]; diff --git a/datafusion/catalog/src/table.rs b/datafusion/catalog/src/table.rs index c6468fd5ad131..43eebcb36adca 100644 --- a/datafusion/catalog/src/table.rs +++ b/datafusion/catalog/src/table.rs @@ -28,7 +28,7 @@ use datafusion_common::{Result, internal_err}; use datafusion_expr::Expr; use datafusion_expr::statistics::StatisticsRequest; -use datafusion_expr::dml::InsertOp; +use datafusion_expr::dml::{InsertOp, MergeIntoClause}; use datafusion_expr::{ CreateExternalTable, LogicalPlan, TableProviderFilterPushDown, TableType, }; @@ -379,6 +379,23 @@ pub trait TableProvider: Any + Debug + Sync + Send { async fn truncate(&self, _state: &dyn Session) -> Result> { not_impl_err!("TRUNCATE not supported for {} table", self.table_type()) } + + /// Merge rows from a source into this table. + /// + /// The `source` is an [`ExecutionPlan`] representing the USING clause. + /// The `on` condition is the join predicate from the ON clause. + /// The `clauses` describe the WHEN MATCHED / WHEN NOT MATCHED actions. + /// + /// Returns an [`ExecutionPlan`] producing a single row with `count` (UInt64). + async fn merge_into( + &self, + _state: &dyn Session, + _source: Arc, + _on: Expr, + _clauses: Vec, + ) -> Result> { + not_impl_err!("MERGE INTO not supported for {} table", self.table_type()) + } } impl dyn TableProvider { diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 0c26bd0841883..07196d009c54c 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -722,7 +722,7 @@ config_namespace! { /// will be removed after the migration is finished. /// /// See for details. - pub enable_migration_aggregate: bool, default = false + pub enable_migration_aggregate: bool, default = true /// Sets the compression codec used when spilling data to disk. /// diff --git a/datafusion/common/src/heap_size.rs b/datafusion/common/src/heap_size.rs index 494ad35e1eeb4..869946d82414f 100644 --- a/datafusion/common/src/heap_size.rs +++ b/datafusion/common/src/heap_size.rs @@ -326,47 +326,6 @@ impl DFHeapSize for Fields { } } -impl DFHeapSize for StructArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} - -impl DFHeapSize for LargeListArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} - -impl DFHeapSize for LargeListViewArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} - -impl DFHeapSize for ListArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} - -impl DFHeapSize for ListViewArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} - -impl DFHeapSize for FixedSizeListArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} -impl DFHeapSize for MapArray { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - self.get_array_memory_size() - } -} - impl DFHeapSize for Box { fn heap_size(&self, ctx: &mut DFHeapSizeCtx) -> usize { size_of::() + self.as_ref().heap_size(ctx) @@ -410,24 +369,6 @@ impl DFHeapSize for UnionFields { } } -impl DFHeapSize for UnionMode { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for TimeUnit { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for IntervalUnit { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - impl DFHeapSize for Field { fn heap_size(&self, ctx: &mut DFHeapSizeCtx) -> usize { self.name().heap_size(ctx) @@ -452,98 +393,63 @@ impl DFHeapSize for IntervalDayTime { } } -impl DFHeapSize for DateTime { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for bool { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} -impl DFHeapSize for u8 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for u16 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for u32 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for u64 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for i8 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for i16 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for i32 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} -impl DFHeapSize for i64 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for i128 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for i256 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for f16 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} - -impl DFHeapSize for f32 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} -impl DFHeapSize for f64 { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } +/// Implement [`DFHeapSize`] for types that own no heap allocations. +macro_rules! impl_zero_heap_size { + ($($t:ty),+ $(,)?) => { + $( + impl DFHeapSize for $t { + fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { + 0 // no heap allocations + } + } + )+ + }; +} + +impl_zero_heap_size!( + bool, + u8, + u16, + u32, + u64, + usize, + i8, + i16, + i32, + i64, + i128, + i256, + f16, + f32, + f64, + UnionMode, + TimeUnit, + IntervalUnit, + DateTime, +); + +/// Implement [`DFHeapSize`] for Arrow arrays types. +macro_rules! impl_array_heap_size { + ($($t:ty),+ $(,)?) => { + $( + impl DFHeapSize for $t { + fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { + self.get_array_memory_size() + } + } + )+ + }; } -impl DFHeapSize for usize { - fn heap_size(&self, _: &mut DFHeapSizeCtx) -> usize { - 0 // no heap allocations - } -} +impl_array_heap_size!( + StructArray, + LargeListArray, + LargeListViewArray, + ListArray, + ListViewArray, + FixedSizeListArray, + MapArray, +); #[cfg(test)] mod tests { @@ -621,6 +527,20 @@ mod tests { assert_eq!(size(&f16::from_f32(0.0)), 0); } + #[test] + fn test_heap_size_union_mode() { + assert_eq!(size(&UnionMode::Sparse), 0); + assert_eq!(size(&UnionMode::Dense), 0); + } + + #[test] + fn test_heap_size_time_units() { + assert_eq!(size(&TimeUnit::Second), 0); + assert_eq!(size(&IntervalUnit::YearMonth), 0); + assert_eq!(size(&DateTime::::UNIX_EPOCH), 0); + assert_eq!(size(&Utc::now()), 0); + } + #[test] fn test_string() { let mut s = String::with_capacity(32); @@ -758,4 +678,61 @@ mod tests { let field = Field::new("temperature", DataType::Float64, true); assert!(size(&field) > 0); } + + #[test] + fn test_list_array() { + use arrow::array::types::Int32Type; + + let array = ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + Some(vec![Some(4)]), + ]); + assert_eq!(size(&array), array.get_array_memory_size()); + assert!(size(&array) > 0); + + let large = + LargeListArray::from_iter_primitive::(vec![Some(vec![ + Some(1), + Some(2), + ])]); + assert_eq!(size(&large), large.get_array_memory_size()); + assert!(size(&large) > 0); + } + + #[test] + fn test_struct_array() { + use arrow::array::Int32Array; + + let array = StructArray::from(vec![( + Arc::new(Field::new("a", DataType::Int32, true)), + Arc::new(Int32Array::from(vec![1, 2, 3])) as _, + )]); + assert_eq!(size(&array), array.get_array_memory_size()); + assert!(size(&array) > 0); + } + + #[test] + fn test_fixed_size_list_array() { + use arrow::array::Int32Array; + + let values = Arc::new(Int32Array::from(vec![1, 2, 3, 4])); + let field = Arc::new(Field::new("item", DataType::Int32, true)); + let array = FixedSizeListArray::new(field, 2, values, None); + assert_eq!(size(&array), array.get_array_memory_size()); + assert!(size(&array) > 0); + } + + #[test] + fn test_map_array() { + use arrow::array::{Int32Builder, MapBuilder, StringBuilder}; + + let mut builder = + MapBuilder::new(None, StringBuilder::new(), Int32Builder::new()); + builder.keys().append_value("key"); + builder.values().append_value(1); + builder.append(true).unwrap(); + let array = builder.finish(); + assert_eq!(size(&array), array.get_array_memory_size()); + assert!(size(&array) > 0); + } } diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index c9013af72619c..8a8a47b3bb50b 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -4292,7 +4292,14 @@ impl ScalarValue { .or_else(|| timestamp_to_timestamp_multiplier(&source_type, target_type)) && let Some(value) = self.temporal_scalar_value_as_i64() { - ensure_timestamp_in_bounds(value, multiplier, &source_type, target_type)?; + match ensure_timestamp_in_bounds(value, multiplier, &source_type, target_type) + { + Ok(()) => {} + Err(_) if cast_options.safe => { + return ScalarValue::try_new_null(target_type); + } + Err(e) => return Err(e), + } } let scalar_array = self.to_array()?; @@ -10190,6 +10197,24 @@ mod tests { ); } + #[test] + fn safe_cast_date_to_timestamp_overflow_returns_null() { + let scalar = ScalarValue::Date32(Some(i32::MAX)); + let safe_options = CastOptions { + safe: true, + ..DEFAULT_CAST_OPTIONS + }; + + let casted = scalar + .cast_to_with_options( + &DataType::Timestamp(TimeUnit::Nanosecond, None), + &safe_options, + ) + .expect("expected safe cast to return null"); + + assert_eq!(casted, ScalarValue::TimestampNanosecond(None, None)); + } + #[test] fn cast_timestamp_to_timestamp_overflow_returns_error() { let scalar = ScalarValue::TimestampSecond(Some(i64::MAX), None); @@ -10203,6 +10228,24 @@ mod tests { ); } + #[test] + fn safe_cast_timestamp_to_timestamp_overflow_returns_null() { + let scalar = ScalarValue::TimestampSecond(Some(i64::MAX), None); + let safe_options = CastOptions { + safe: true, + ..DEFAULT_CAST_OPTIONS + }; + + let casted = scalar + .cast_to_with_options( + &DataType::Timestamp(TimeUnit::Nanosecond, None), + &safe_options, + ) + .expect("expected safe cast to return null"); + + assert_eq!(casted, ScalarValue::TimestampNanosecond(None, None)); + } + #[test] fn null_dictionary_scalar_produces_null_dictionary_array() { let dictionary_scalar = ScalarValue::Dictionary( diff --git a/datafusion/common/src/utils/memory.rs b/datafusion/common/src/utils/memory.rs index 78ec434d2b577..21c084119e120 100644 --- a/datafusion/common/src/utils/memory.rs +++ b/datafusion/common/src/utils/memory.rs @@ -21,7 +21,8 @@ use crate::error::_exec_datafusion_err; use crate::{HashSet, Result}; use arrow::array::ArrayData; use arrow::record_batch::RecordBatch; -use std::{mem::size_of, ptr::NonNull}; +use std::mem::size_of; +use std::num::NonZero; /// Estimates the memory size required for a hash table prior to allocation. /// @@ -131,34 +132,74 @@ pub fn estimate_memory_size(num_elements: usize, fixed_size: usize) -> Result /// `Buffer`. This method provides temporary fix until the issue is resolved: /// pub fn get_record_batch_memory_size(batch: &RecordBatch) -> usize { - // Store pointers to `Buffer`'s start memory address (instead of actual - // used data region's pointer represented by current `Array`) - let mut counted_buffers: HashSet> = HashSet::new(); - let mut total_size = 0; - - for array in batch.columns() { - let array_data = array.to_data(); - count_array_data_memory_size(&array_data, &mut counted_buffers, &mut total_size); + RecordBatchMemoryCounter::new().count_batch(batch) +} + +/// Tracks the memory used by a sequence of [`RecordBatch`]es that may share +/// underlying buffers, counting each buffer exactly once. +/// +/// Use this instead of [`get_record_batch_memory_size`] to account for the +/// total memory of a sequence of batches, e.g. when buffering the batches of +/// an input stream. Such batches can share buffers (for example, operators +/// like aggregates emit one large batch as multiple zero-copy slices), and +/// calling [`get_record_batch_memory_size`] per batch counts the shared +/// buffers once per batch, while this counter counts them exactly once. A +/// batch's buffers are kept alive by the batch even when only a sub-range is +/// referenced, so counting unique buffers in full reflects the memory the +/// batches actually retain. +#[derive(Debug, Default)] +pub struct RecordBatchMemoryCounter { + /// Start addresses of `Buffer`s that have already been counted (instead of + /// actual used data region's pointer represented by current `Array`) + counted_buffers: HashSet>, + /// Total memory of all unique buffers counted so far + memory_usage: usize, +} + +impl RecordBatchMemoryCounter { + pub fn new() -> Self { + Self::default() } - total_size + /// Count `batch`, returning the memory used by its buffers that have not + /// been counted before. + pub fn count_batch(&mut self, batch: &RecordBatch) -> usize { + let mut total_size = 0; + + for array in batch.columns() { + let array_data = array.to_data(); + count_array_data_memory_size( + &array_data, + &mut self.counted_buffers, + &mut total_size, + ); + } + + self.memory_usage += total_size; + total_size + } + + /// Total memory of the unique buffers of all batches counted so far. + pub fn memory_usage(&self) -> usize { + self.memory_usage + } } /// Count the memory usage of `array_data` and its children recursively. fn count_array_data_memory_size( array_data: &ArrayData, - counted_buffers: &mut HashSet>, + counted_buffers: &mut HashSet>, total_size: &mut usize, ) { // Count memory usage for `array_data` for buffer in array_data.buffers() { - if counted_buffers.insert(buffer.data_ptr()) { + if counted_buffers.insert(buffer.data_ptr().addr()) { *total_size += buffer.capacity(); } // Otherwise the buffer's memory is already counted } if let Some(null_buffer) = array_data.nulls() - && counted_buffers.insert(null_buffer.inner().inner().data_ptr()) + && counted_buffers.insert(null_buffer.inner().inner().data_ptr().addr()) { *total_size += null_buffer.inner().inner().capacity(); } @@ -295,6 +336,29 @@ mod record_batch_tests { assert_eq!(size_origin, size_sliced); } + #[test] + fn test_record_batch_memory_counter_buffer_shared_across_batches() { + let schema = Arc::new(Schema::new(vec![Field::new( + "ints", + DataType::Int32, + false, + )])); + + let int_array = Int32Array::from(vec![1, 2, 3, 4, 5, 6]); + let batch = RecordBatch::try_new(schema, vec![Arc::new(int_array)]).unwrap(); + let slices = [batch.slice(0, 2), batch.slice(2, 2), batch.slice(4, 2)]; + + // Counting each slice individually counts the shared buffer once per slice + let summed: usize = slices.iter().map(get_record_batch_memory_size).sum(); + assert_eq!(summed, 3 * get_record_batch_memory_size(&batch)); + + // A counter shared across the batches counts it exactly once + let mut counter = RecordBatchMemoryCounter::new(); + let deduped: usize = slices.iter().map(|slice| counter.count_batch(slice)).sum(); + assert_eq!(deduped, get_record_batch_memory_size(&batch)); + assert_eq!(counter.memory_usage(), get_record_batch_memory_size(&batch)); + } + #[test] fn test_get_record_batch_memory_size_nested_array() { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 190a08da12222..36d2a0c2ee168 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -924,6 +924,26 @@ impl DefaultPhysicalPlanner { ); } } + LogicalPlan::Dml(DmlStatement { + table_name, + target, + op: WriteOp::MergeInto(merge_op), + .. + }) => { + let provider = source_as_provider(target)?; + let input_exec = children.one()?; + provider + .merge_into( + session_state, + input_exec, + merge_op.on.clone(), + merge_op.clauses.clone(), + ) + .await + .map_err(|e| { + e.context(format!("MERGE INTO operation on table '{table_name}'")) + })? + } LogicalPlan::Window(Window { window_expr, .. }) => { assert_or_internal_err!( !window_expr.is_empty(), diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 7d22c5df70dfc..b895cb9c7ce2c 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -872,7 +872,6 @@ impl GroupsAccumulator for TestGroupsAccumulator { &mut self, _values: &[ArrayRef], _group_indices: &[usize], - _opt_filter: Option<&arrow::array::BooleanArray>, _total_num_groups: usize, ) -> Result<()> { Ok(()) diff --git a/datafusion/datasource-parquet/src/row_filter.rs b/datafusion/datasource-parquet/src/row_filter.rs index f19dbd6c6fa63..8ce359942cc4f 100644 --- a/datafusion/datasource-parquet/src/row_filter.rs +++ b/datafusion/datasource-parquet/src/row_filter.rs @@ -72,6 +72,7 @@ use arrow::array::BooleanArray; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::error::{ArrowError, Result as ArrowResult}; use arrow::record_batch::RecordBatch; +use datafusion_functions::core::file_row_index::FileRowIndexFunc; use datafusion_functions::core::getfield::GetFieldFunc; use parquet::arrow::ProjectionMask; use parquet::arrow::arrow_reader::{ArrowPredicate, RowFilter}; @@ -260,6 +261,9 @@ struct PushdownChecker<'schema> { non_primitive_columns: bool, /// Does the expression reference any columns not present in the file schema? projected_columns: bool, + /// Does the expression references a ScalarUDF that requires some rewrite + /// and therefore can't be pushed down into the row-filter. + has_unpushable_udfs: bool, /// Indices into the file schema of columns required to evaluate the expression. /// Does not include struct columns accessed via `get_field`. required_columns: Vec, @@ -276,6 +280,7 @@ impl<'schema> PushdownChecker<'schema> { Self { non_primitive_columns: false, projected_columns: false, + has_unpushable_udfs: false, required_columns: Vec::new(), struct_field_accesses: Vec::new(), allow_list_columns, @@ -372,7 +377,7 @@ impl<'schema> PushdownChecker<'schema> { #[inline] fn prevents_pushdown(&self) -> bool { - self.non_primitive_columns || self.projected_columns + self.non_primitive_columns || self.projected_columns || self.has_unpushable_udfs } /// Consumes the checker and returns sorted, deduplicated column indices @@ -484,6 +489,13 @@ impl TreeNodeVisitor<'_> for PushdownChecker<'_> { return Ok(recursion); } + if ScalarFunctionExpr::try_downcast_func::(node.as_ref()) + .is_some() + { + self.has_unpushable_udfs = true; + return Ok(TreeNodeRecursion::Jump); + } + Ok(TreeNodeRecursion::Continue) } } diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 8228cd273eae6..840b86dcb875d 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -26,6 +26,9 @@ use crate::opener::ParquetMorselizer; use crate::opener::build_pruning_predicates; use crate::opener::build_virtual_columns_state; use crate::row_filter::can_expr_be_pushed_down_with_schemas; +use arrow_schema::Fields; +use arrow_schema::extension::ExtensionType; +use arrow_schema::{DataType, Field}; use datafusion_common::config::ConfigOptions; #[cfg(feature = "parquet_encryption")] use datafusion_common::config::EncryptionFactoryOptions; @@ -40,9 +43,14 @@ use datafusion_common::config::TableParquetOptions; use datafusion_datasource::TableSchema; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_functions::core::file_row_index::FileRowIndexFunc; +use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_expr::{EquivalenceProperties, conjunction}; -use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory; +use datafusion_physical_expr_adapter::expr_references_scalar_udf; +use datafusion_physical_expr_adapter::{ + DefaultPhysicalExprAdapterFactory, rewrite_file_row_index_projection, +}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_plan::DisplayFormatType; @@ -60,6 +68,7 @@ use datafusion_execution::parquet_encryption::EncryptionFactory; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use object_store::ObjectStore; +use parquet::arrow::RowNumber; #[cfg(feature = "parquet_encryption")] use parquet::encryption::decrypt::FileDecryptionProperties; @@ -669,7 +678,28 @@ impl FileSource for ParquetSource { projection: &ProjectionExprs, ) -> datafusion_common::Result>> { let mut source = self.clone(); - source.projection = self.projection.try_merge(projection)?; + + // If there's no reference to `FileRowIndexFunc` in the projection, we can just merge + // both projections as-is, there's no need to modify the projection first. + if !projection.iter().any(|projection_expr| { + expr_references_scalar_udf::(&projection_expr.expr) + }) { + source.projection = self.projection.try_merge(projection)?; + return Ok(Some(Arc::new(source))); + } + + // If we can find a reference to `FileRowIndexFunc`, we add it as a virtual column + // or re-use an existing one in the table's schema. + let (table_schema, row_index_col) = + table_schema_with_row_index_col(self.table_schema()); + + source.table_schema = table_schema; + source.projection = rewrite_file_row_index_projection( + &self.projection, + projection, + &row_index_col, + )?; + Ok(Some(Arc::new(source))) } @@ -952,15 +982,12 @@ impl FileSource for ParquetSource { reversed_eq_properties.ordering_satisfy(order.iter().cloned())?; let sort_order = LexOrdering::new(order.iter().cloned()); let column_in_file_schema = sort_order.as_ref().is_some_and(|s| { - s.first() - .expr - .downcast_ref::() - .is_some_and(|col| { - self.table_schema - .file_schema() - .field_with_name(col.name()) - .is_ok() - }) + s.first().expr.downcast_ref::().is_some_and(|col| { + self.table_schema + .file_schema() + .field_with_name(col.name()) + .is_ok() + }) }); if !column_in_file_schema && !reversed_satisfies { @@ -989,6 +1016,69 @@ impl FileSource for ParquetSource { } } +/// Returns the a [`TableSchema`] containing a [`RowNumber`] virtual column and a [`Column`] expression referencing its row index column. +/// The expression is then merged into a projection. +/// +/// - If the schema already has a virtual column with the [`RowNumber`] type, it returns the schema unchanged. +/// - If the schema doesn't have the appropriate virtual column, it returns a modified schema with the virtual column appended to it. +fn table_schema_with_row_index_col(table_schema: &TableSchema) -> (TableSchema, Column) { + // If we can find a virtual column with the `RowNumber` type, we just return the schema + // and create the appropriate `column` we're going to use + if let Some((idx, field)) = + table_schema + .virtual_columns() + .iter() + .enumerate() + .find(|(_, field)| { + field + .extension_type_name() + .is_some_and(|name| name == RowNumber::NAME) + }) + { + let virtual_offset = table_schema.file_schema().fields().len() + + table_schema.table_partition_cols().len(); + + return ( + table_schema.clone(), + Column::new(field.name(), virtual_offset + idx), + ); + } + + // The hidden field is shared across all files in this scan, but it must + // have a unique table-schema name because later rewrites resolve it by + // column name and index. + let base_row_index_name = "__datafusion_file_row_index"; + let mut row_index_name = base_row_index_name.to_string(); + let mut suffix = 0; + while table_schema + .table_schema() + .field_with_name(&row_index_name) + .is_ok() + { + suffix += 1; + row_index_name = format!("{base_row_index_name}_{suffix}"); + } + + let row_index_table_idx = table_schema.table_schema().fields().len(); + let row_index_field = Arc::new( + Field::new(&row_index_name, DataType::Int64, true).with_extension_type(RowNumber), + ); + ( + TableSchema::builder(Arc::clone(table_schema.file_schema())) + .with_table_partition_cols(table_schema.table_partition_cols().clone()) + .with_virtual_columns( + table_schema + .virtual_columns() + .iter() + .cloned() + .chain([row_index_field]) + .collect::(), + ) + .build(), + Column::new(&row_index_name, row_index_table_idx), + ) +} + #[cfg(test)] mod tests { use super::*; @@ -1622,7 +1712,9 @@ mod tests { use datafusion_common::config::ConfigOptions; use datafusion_datasource::TableSchema; use datafusion_expr::{col, lit as logical_lit}; + use datafusion_functions::core::expr_fn::file_row_index; use datafusion_physical_expr::planner::logical2physical; + use datafusion_physical_expr_adapter::rewrite_file_row_index_expr; use datafusion_physical_plan::filter_pushdown::PushedDown; use parquet::arrow::RowNumber; @@ -1652,13 +1744,20 @@ mod tests { .or(col("value").eq(logical_lit(4i64))), full_schema, ); + let (_, row_index_col) = table_schema_with_row_index_col(source.table_schema()); + let row_index = rewrite_file_row_index_expr( + logical2physical(&file_row_index().gt(logical_lit(2i64)), full_schema), + row_index_col.name(), + row_index_col.index(), + ) + .expect("file_row_index should rewrite to the row_number virtual column"); let config = ConfigOptions::default(); let prop = source - .try_pushdown_filters(vec![pushable, virtual_only, mixed], &config) + .try_pushdown_filters(vec![pushable, virtual_only, mixed, row_index], &config) .expect("try_pushdown_filters must not error"); - assert_eq!(prop.filters.len(), 3); + assert_eq!(prop.filters.len(), 4); assert!( matches!(prop.filters[0], PushedDown::Yes), "file-column filter should be pushable" @@ -1672,5 +1771,10 @@ mod tests { "filter mixing a virtual column with a file column must not be \ pushed down (row filter would silently drop it)" ); + assert!( + matches!(prop.filters[3], PushedDown::No), + "file_row_index() rewrites to a virtual column and must not be \ + pushed down" + ); } } diff --git a/datafusion/expr-common/src/columnar_value.rs b/datafusion/expr-common/src/columnar_value.rs index caeb3f10da752..ef9192c3569d9 100644 --- a/datafusion/expr-common/src/columnar_value.rs +++ b/datafusion/expr-common/src/columnar_value.rs @@ -325,7 +325,9 @@ fn cast_array_by_name( ) { datafusion_common::nested_struct::cast_column(array, cast_type, cast_options) } else { - ensure_temporal_array_timestamp_bounds(array, cast_type)?; + if !cast_options.safe { + ensure_temporal_array_timestamp_bounds(array, cast_type)?; + } Ok(kernels::cast::cast_with_options( array, cast_type, @@ -766,4 +768,32 @@ mod tests { "unexpected error: {err}" ); } + + #[test] + fn safe_cast_timestamp_array_to_timestamp_overflow_returns_null() { + let overflow_value = i64::MAX / 1_000_000_000 + 1; + let array: ArrayRef = + Arc::new(TimestampSecondArray::from(vec![Some(overflow_value)])); + let value = ColumnarValue::Array(array); + let safe_options = CastOptions { + safe: true, + ..DEFAULT_CAST_OPTIONS + }; + + let casted = value + .cast_to( + &DataType::Timestamp(TimeUnit::Nanosecond, None), + Some(&safe_options), + ) + .expect("expected safe cast to return null"); + + let ColumnarValue::Array(array) = casted else { + panic!("expected array after cast"); + }; + let array = array + .as_any() + .downcast_ref::() + .expect("expected TimestampNanosecondArray"); + assert!(array.is_null(0)); + } } diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index da5da384c7b4e..b021674cbec2c 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -183,12 +183,14 @@ pub trait GroupsAccumulator: Send + std::any::Any { /// /// * `values`: arrays produced from previously calling `state` on other accumulators. /// - /// Other arguments are the same as for [`Self::update_batch`]. + /// Other arguments are the same as for [`Self::update_batch`], except that + /// there is no `opt_filter` — aggregate filters are applied during the + /// partial (update) phase, so by the time intermediate states are merged + /// no per-row filtering is needed. fn merge_batch( &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()>; diff --git a/datafusion/expr/src/logical_plan/dml.rs b/datafusion/expr/src/logical_plan/dml.rs index b668cbfe2cc35..7717dfaff7a33 100644 --- a/datafusion/expr/src/logical_plan/dml.rs +++ b/datafusion/expr/src/logical_plan/dml.rs @@ -23,9 +23,9 @@ use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::file_options::file_type::FileType; -use datafusion_common::{DFSchemaRef, TableReference}; +use datafusion_common::{DFSchemaRef, Result, TableReference, internal_err}; -use crate::{LogicalPlan, TableSource}; +use crate::{Expr, LogicalPlan, TableSource}; /// Operator that copies the contents of a database to file(s) #[derive(Clone)] @@ -227,7 +227,11 @@ impl PartialOrd for DmlStatement { /// The type of DML operation to perform. /// /// See [`DmlStatement`] for more details. +/// +/// Marked `#[non_exhaustive]` so adding new variants in future releases is +/// not a SemVer break for downstream matchers. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] +#[non_exhaustive] pub enum WriteOp { /// `INSERT INTO` operation Insert(InsertOp), @@ -239,6 +243,8 @@ pub enum WriteOp { Ctas, /// `TRUNCATE` operation Truncate, + /// `MERGE INTO` operation + MergeInto(Box), } impl WriteOp { @@ -250,6 +256,7 @@ impl WriteOp { WriteOp::Update => "Update", WriteOp::Ctas => "Ctas", WriteOp::Truncate => "Truncate", + WriteOp::MergeInto(_) => "MergeInto", } } } @@ -291,6 +298,196 @@ impl Display for InsertOp { } } +/// Describes a MERGE INTO operation's parameters. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] +pub struct MergeIntoOp { + /// The join condition from `ON `. + pub on: Expr, + /// The WHEN clauses, in the order they appeared in the SQL. + pub clauses: Vec, +} + +impl MergeIntoOp { + /// Count of top-level [`Expr`]s owned by this operation (no allocation). + /// + /// Matches the length of [`Self::exprs`] and the `exprs` vec consumed by + /// [`Self::with_new_exprs`]. + fn expr_count(&self) -> usize { + 1 + self + .clauses + .iter() + .map(|c| { + c.predicate.is_some() as usize + + match &c.action { + MergeIntoAction::Update(a) => a.len(), + MergeIntoAction::Insert { values, .. } => values.len(), + MergeIntoAction::Delete => 0, + } + }) + .sum::() + } + + /// Top-level [`Expr`]s in stable order: `on`, then per-clause predicate + /// (if any) and action value expressions. + pub fn exprs(&self) -> Vec<&Expr> { + let mut out = Vec::with_capacity(self.expr_count()); + out.push(&self.on); + for clause in &self.clauses { + if let Some(predicate) = &clause.predicate { + out.push(predicate); + } + match &clause.action { + MergeIntoAction::Update(assignments) => { + out.extend(assignments.iter().map(|(_, value)| value)); + } + MergeIntoAction::Insert { values, .. } => { + out.extend(values.iter()); + } + MergeIntoAction::Delete => {} + } + } + out + } + + /// Rebuild this `MergeIntoOp` from a flat vector of new expressions, in + /// the same order produced by [`Self::exprs`]. The clause kinds, action + /// kinds, column lists, and presence/absence of each predicate are + /// preserved from `self`. + pub fn with_new_exprs(&self, exprs: Vec) -> Result { + let expected = self.expr_count(); + if exprs.len() != expected { + return internal_err!( + "MergeIntoOp::with_new_exprs expected {expected} expressions, got {}", + exprs.len() + ); + } + let mut iter = exprs.into_iter(); + let on = iter.next().expect("non-empty by length check"); + let clauses = self + .clauses + .iter() + .map(|clause| { + let predicate = clause + .predicate + .is_some() + .then(|| iter.next().expect("non-empty by length check")); + let action = match &clause.action { + MergeIntoAction::Update(assignments) => { + let assignments = assignments + .iter() + .map(|(name, _)| { + ( + name.clone(), + iter.next().expect("non-empty by length check"), + ) + }) + .collect(); + MergeIntoAction::Update(assignments) + } + MergeIntoAction::Insert { columns, values } => { + let values = values + .iter() + .map(|_| iter.next().expect("non-empty by length check")) + .collect(); + MergeIntoAction::Insert { + columns: columns.clone(), + values, + } + } + MergeIntoAction::Delete => MergeIntoAction::Delete, + }; + MergeIntoClause { + kind: clause.kind, + predicate, + action, + } + }) + .collect(); + Ok(Self { on, clauses }) + } +} + +/// A single WHEN clause within a MERGE INTO statement. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] +pub struct MergeIntoClause { + /// Whether this fires on matched or unmatched rows. + pub kind: MergeIntoClauseKind, + /// Optional additional predicate (`AND `). + pub predicate: Option, + /// The action to take. + pub action: MergeIntoAction, +} + +/// Which rows a MERGE WHEN clause applies to. +/// +/// Mirrors `sqlparser::ast::MergeClauseKind` so that the SQL spelling is +/// preserved through the logical plan. +/// +/// **Note on `NotMatched` vs `NotMatchedByTarget`:** these two variants are +/// semantically identical — both describe a source row that has no matching +/// target row. `NotMatched` is the SQL standard short form (used by +/// Snowflake, Postgres, SQL Server); `NotMatchedByTarget` is BigQuery's +/// explicit form added for symmetry with `NotMatchedBySource`. Downstream +/// consumers (planners, table providers, optimizers) MUST treat the two +/// variants identically. +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Hash)] +pub enum MergeIntoClauseKind { + /// `WHEN MATCHED` + Matched, + /// `WHEN NOT MATCHED` — see type-level note for the equivalence with + /// [`NotMatchedByTarget`](Self::NotMatchedByTarget). + NotMatched, + /// `WHEN NOT MATCHED BY TARGET` — see type-level note for the + /// equivalence with [`NotMatched`](Self::NotMatched). + NotMatchedByTarget, + /// `WHEN NOT MATCHED BY SOURCE` + NotMatchedBySource, +} + +impl MergeIntoClauseKind { + /// True if this clause fires on a source row that has no matching target + /// row. Returns `true` for both [`NotMatched`](Self::NotMatched) and + /// [`NotMatchedByTarget`](Self::NotMatchedByTarget) (see the type-level + /// note explaining why those two variants are semantically identical). + /// + /// Prefer this predicate over hand-written `matches!` arms so the + /// `NotMatched`/`NotMatchedByTarget` equivalence is enforced in one place. + pub fn is_not_matched_by_target(&self) -> bool { + matches!(self, Self::NotMatched | Self::NotMatchedByTarget) + } + + /// Collapse the SQL-spelling variants into the canonical three semantic + /// categories: [`Matched`](Self::Matched), + /// [`NotMatchedByTarget`](Self::NotMatchedByTarget) (covering both + /// "NOT MATCHED" spellings), and + /// [`NotMatchedBySource`](Self::NotMatchedBySource). + /// + /// Use this in downstream `match` expressions when the SQL spelling + /// distinction does not matter — e.g. in planners, optimizers, or + /// table-provider dispatch. + pub fn canonical(self) -> Self { + match self { + Self::NotMatched => Self::NotMatchedByTarget, + other => other, + } + } +} + +/// The action for a single WHEN clause. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] +pub enum MergeIntoAction { + /// `UPDATE SET col1 = expr1, col2 = expr2, ...`, stored as + /// `(column_name, value_expr)` pairs. + Update(Vec<(String, Expr)>), + /// `INSERT (col1, col2, ...) VALUES (expr1, expr2, ...)`. `columns` may + /// be empty, meaning all columns. + Insert { + columns: Vec, + values: Vec, + }, + Delete, +} + fn make_count_schema() -> DFSchemaRef { Arc::new( Schema::new(vec![Field::new("count", DataType::UInt64, false)]) @@ -298,3 +495,103 @@ fn make_count_schema() -> DFSchemaRef { .unwrap(), ) } + +#[cfg(test)] +mod tests { + use super::*; + use crate::{col, lit}; + + #[test] + fn write_op_merge_into_name_and_display() { + let op = WriteOp::MergeInto(Box::new(MergeIntoOp { + on: col("id").eq(col("source_id")), + clauses: vec![MergeIntoClause { + kind: MergeIntoClauseKind::Matched, + predicate: Some(col("qty").gt(lit(0_i64))), + action: MergeIntoAction::Update(vec![( + "qty".to_string(), + col("source_qty"), + )]), + }], + })); + assert_eq!(op.name(), "MergeInto"); + assert_eq!(format!("{op}"), "MergeInto"); + } + + #[test] + fn merge_into_clause_kind_is_not_matched_by_target() { + assert!(!MergeIntoClauseKind::Matched.is_not_matched_by_target()); + assert!(MergeIntoClauseKind::NotMatched.is_not_matched_by_target()); + assert!(MergeIntoClauseKind::NotMatchedByTarget.is_not_matched_by_target()); + assert!(!MergeIntoClauseKind::NotMatchedBySource.is_not_matched_by_target()); + } + + #[test] + fn merge_into_clause_kind_canonical_collapses_not_matched() { + assert_eq!( + MergeIntoClauseKind::NotMatched.canonical(), + MergeIntoClauseKind::NotMatchedByTarget + ); + assert_eq!( + MergeIntoClauseKind::NotMatchedByTarget.canonical(), + MergeIntoClauseKind::NotMatchedByTarget + ); + assert_eq!( + MergeIntoClauseKind::Matched.canonical(), + MergeIntoClauseKind::Matched + ); + assert_eq!( + MergeIntoClauseKind::NotMatchedBySource.canonical(), + MergeIntoClauseKind::NotMatchedBySource + ); + } + + #[test] + fn merge_into_op_exprs_round_trip() { + let op = MergeIntoOp { + on: col("id").eq(col("source_id")), + clauses: vec![ + MergeIntoClause { + kind: MergeIntoClauseKind::Matched, + predicate: Some(col("qty").gt(lit(0_i64))), + action: MergeIntoAction::Update(vec![ + ("qty".to_string(), col("source_qty")), + ("price".to_string(), col("source_price")), + ]), + }, + MergeIntoClause { + kind: MergeIntoClauseKind::NotMatched, + predicate: None, + action: MergeIntoAction::Insert { + columns: vec!["id".to_string(), "qty".to_string()], + values: vec![col("source_id"), col("source_qty")], + }, + }, + MergeIntoClause { + kind: MergeIntoClauseKind::NotMatchedBySource, + predicate: Some(col("active").eq(lit(true))), + action: MergeIntoAction::Delete, + }, + ], + }; + let exprs = op.exprs(); + assert_eq!(exprs.len(), 7); + + let owned: Vec = exprs.into_iter().cloned().collect(); + let rebuilt = op.with_new_exprs(owned).unwrap(); + assert_eq!(op, rebuilt); + } + + #[test] + fn merge_into_op_with_new_exprs_length_mismatch() { + let op = MergeIntoOp { + on: col("id").eq(col("source_id")), + clauses: vec![], + }; + let err = op.with_new_exprs(vec![]).unwrap_err(); + assert!( + err.to_string().contains("expected 1 expressions, got 0"), + "unexpected error: {err}" + ); + } +} diff --git a/datafusion/expr/src/logical_plan/mod.rs b/datafusion/expr/src/logical_plan/mod.rs index e0e51d7e470c3..4766c3f33379f 100644 --- a/datafusion/expr/src/logical_plan/mod.rs +++ b/datafusion/expr/src/logical_plan/mod.rs @@ -36,7 +36,10 @@ pub use ddl::{ CreateFunctionBody, CreateIndex, CreateMemoryTable, CreateView, DdlStatement, DropCatalogSchema, DropFunction, DropTable, DropView, OperateFunctionArg, }; -pub use dml::{DmlStatement, WriteOp}; +pub use dml::{ + DmlStatement, MergeIntoAction, MergeIntoClause, MergeIntoClauseKind, MergeIntoOp, + WriteOp, +}; pub use plan::{ Aggregate, Analyze, ColumnUnnestList, DescribeTable, Distinct, DistinctOn, EmptyRelation, Explain, ExplainOption, Extension, FetchType, Filter, Join, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 3608c81878d17..beeace1d1461e 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -39,7 +39,7 @@ use crate::expr_rewriter::{ }; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; -use crate::logical_plan::{DmlStatement, Statement}; +use crate::logical_plan::{DmlStatement, Statement, WriteOp}; use crate::utils::{ enumerate_grouping_sets, exprlist_to_fields, find_out_reference_exprs, grouping_set_expr_count, grouping_set_to_exprlist, merge_schema, split_conjunction, @@ -810,12 +810,20 @@ impl LogicalPlan { op, .. }) => { - self.assert_no_expressions(expr)?; let input = self.only_input(inputs)?; + let op = match op { + WriteOp::MergeInto(merge_op) => { + WriteOp::MergeInto(Box::new(merge_op.with_new_exprs(expr)?)) + } + other => { + self.assert_no_expressions(expr)?; + other.clone() + } + }; Ok(LogicalPlan::Dml(DmlStatement::new( table_name.clone(), Arc::clone(target), - op.clone(), + op, Arc::new(input), ))) } @@ -1186,12 +1194,20 @@ impl LogicalPlan { options, .. }) => { - self.assert_no_expressions(expr)?; + let exec_columns = if expr.is_empty() { + columns.clone() + } else { + expr.into_iter() + .map(|e| match e { + Expr::Column(c) => Ok(c), + other => internal_err!( + "Expected Expr::Column for Unnest exec_columns, got {other:?}" + ), + }) + .collect::>>()? + }; let input = self.only_input(inputs)?; - // Update schema with unnested column type. - let new_plan = - unnest_with_options(input, columns.clone(), options.clone())?; - Ok(new_plan) + Ok(unnest_with_options(input, exec_columns, options.clone())?) } } } @@ -2073,6 +2089,7 @@ impl LogicalPlan { filter, join_constraint, join_type, + null_aware, .. }) => { let join_expr: Vec = @@ -2081,6 +2098,8 @@ impl LogicalPlan { .as_ref() .map(|expr| format!(" Filter: {expr}")) .unwrap_or_else(|| "".to_string()); + let null_aware_expr = + if *null_aware { " null_aware" } else { "" }; let join_type = if filter.is_none() && keys.is_empty() && *join_type == JoinType::Inner @@ -2100,15 +2119,17 @@ impl LogicalPlan { filter_expr )?; } + write!(f, "{null_aware_expr}")?; Ok(()) } JoinConstraint::Using => { write!( f, - "{} Join: Using {}{}", + "{} Join: Using {}{}{}", join_type, join_expr.join(", "), filter_expr, + null_aware_expr, ) } } @@ -6597,4 +6618,53 @@ mod tests { Ok(()) } + + #[test] + fn test_unnest_with_new_exprs_accepts_expressions() -> Result<()> { + use crate::LogicalPlanBuilder; + use arrow::datatypes::{DataType, Field, Schema}; + + let schema = Schema::new(vec![ + Field::new("list_col", DataType::new_list(DataType::Int32, true), true), + Field::new("other_col", DataType::Int32, true), + ]); + let plan = table_scan(Some("t"), &schema, None)?.build()?; + let unnest_plan = LogicalPlanBuilder::from(plan) + .unnest_column("list_col")? + .build()?; + + let exprs = unnest_plan.expressions(); + assert!(!exprs.is_empty(), "Unnest should expose exec_columns"); + assert_eq!(exprs.len(), 1); + assert!(matches!(&exprs[0], Expr::Column(c) if c.name == "list_col")); + + let inputs: Vec = + unnest_plan.inputs().into_iter().cloned().collect(); + let rebuilt = unnest_plan.with_new_exprs(exprs, inputs)?; + assert_eq!(rebuilt.schema(), unnest_plan.schema()); + + Ok(()) + } + + #[test] + fn test_unnest_with_new_exprs_empty_preserves_columns() -> Result<()> { + use crate::LogicalPlanBuilder; + use arrow::datatypes::{DataType, Field, Schema}; + + let schema = Schema::new(vec![ + Field::new("list_col", DataType::new_list(DataType::Int32, true), true), + Field::new("other_col", DataType::Int32, true), + ]); + let plan = table_scan(Some("t"), &schema, None)?.build()?; + let unnest_plan = LogicalPlanBuilder::from(plan) + .unnest_column("list_col")? + .build()?; + + let inputs: Vec = + unnest_plan.inputs().into_iter().cloned().collect(); + let rebuilt = unnest_plan.with_new_exprs(vec![], inputs)?; + assert_eq!(rebuilt.schema(), unnest_plan.schema()); + + Ok(()) + } } diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index cba2dac24b610..c4c1d743b58b6 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -37,13 +37,15 @@ //! * [`LogicalPlan::with_new_exprs`]: Create a new plan with different expressions //! * [`LogicalPlan::expressions`]: Return a copy of the plan's expressions +use std::sync::Arc; + use crate::logical_plan::plan::RangePartitioning; use crate::{ Aggregate, Analyze, CreateMemoryTable, CreateView, DdlStatement, Distinct, DistinctOn, DmlStatement, Execute, Explain, Expr, Extension, Filter, Join, Limit, LogicalPlan, Partitioning, Prepare, Projection, RecursiveQuery, Repartition, Sort, Statement, Subquery, SubqueryAlias, TableScan, Union, Unnest, UserDefinedLogicalNode, - Values, Window, dml::CopyTo, + Values, Window, WriteOp, builder::unnest_with_options, dml::CopyTo, }; use datafusion_common::tree_node::TreeNodeRefContainer; @@ -478,6 +480,10 @@ impl LogicalPlan { } _ => Ok(TreeNodeRecursion::Continue), }, + LogicalPlan::Dml(DmlStatement { + op: WriteOp::MergeInto(merge_op), + .. + }) => merge_op.exprs().apply_ref_elements(f), // plans without expressions LogicalPlan::EmptyRelation(_) | LogicalPlan::RecursiveQuery(_) @@ -686,9 +692,60 @@ impl LogicalPlan { _ => Transformed::no(stmt), } .update_data(LogicalPlan::Statement), + LogicalPlan::Unnest(Unnest { + input, + exec_columns, + options, + .. + }) => { + let exprs: Vec = + exec_columns.into_iter().map(Expr::Column).collect(); + exprs.map_elements(f)?.map_data(|mapped_exprs| { + let new_columns = mapped_exprs + .into_iter() + .map(|e| match e { + Expr::Column(c) => Ok(c), + other => internal_err!( + "Expected Expr::Column for Unnest exec_columns, got {other:?}" + ), + }) + .collect::>>()?; + // Rebuild through `unnest_with_options` so the derived + // `list_type_columns`, `struct_type_columns`, + // `dependency_indices`, and `schema` are recomputed from + // the (possibly rewritten) columns rather than carried over + // stale. This keeps `map_expressions` consistent with + // `with_new_exprs`. + unnest_with_options( + Arc::unwrap_or_clone(input), + new_columns, + options, + ) + })? + } + LogicalPlan::Dml(DmlStatement { + table_name, + target, + op: WriteOp::MergeInto(merge_op), + input, + output_schema, + }) => { + let owned_exprs: Vec = + merge_op.exprs().into_iter().cloned().collect(); + owned_exprs.map_elements(f)?.transform_data(|new_exprs| { + Ok(Transformed::no(LogicalPlan::Dml(DmlStatement { + table_name, + target, + op: WriteOp::MergeInto(Box::new( + merge_op.with_new_exprs(new_exprs)?, + )), + input, + output_schema, + }))) + })? + } // plans without expressions LogicalPlan::EmptyRelation(_) - | LogicalPlan::Unnest(_) | LogicalPlan::RecursiveQuery(_) | LogicalPlan::Subquery(_) | LogicalPlan::SubqueryAlias(_) diff --git a/datafusion/ffi/src/lib.rs b/datafusion/ffi/src/lib.rs index 4df6c4b570f34..fd2ac58576b09 100644 --- a/datafusion/ffi/src/lib.rs +++ b/datafusion/ffi/src/lib.rs @@ -36,6 +36,7 @@ pub mod ffi_option; pub mod insert_op; pub mod physical_expr; pub mod physical_optimizer; +pub mod placement; pub mod plan_properties; pub mod proto; pub mod record_batch_stream; diff --git a/datafusion/ffi/src/placement.rs b/datafusion/ffi/src/placement.rs new file mode 100644 index 0000000000000..837f0e3aad647 --- /dev/null +++ b/datafusion/ffi/src/placement.rs @@ -0,0 +1,72 @@ +// 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. + +use datafusion_expr::ExpressionPlacement; + +#[expect(non_camel_case_types)] +#[repr(u8)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum FFI_ExpressionPlacement { + Literal, + Column, + MoveTowardsLeafNodes, + KeepInPlace, +} + +impl From for FFI_ExpressionPlacement { + fn from(value: ExpressionPlacement) -> Self { + match value { + ExpressionPlacement::Literal => Self::Literal, + ExpressionPlacement::Column => Self::Column, + ExpressionPlacement::MoveTowardsLeafNodes => Self::MoveTowardsLeafNodes, + ExpressionPlacement::KeepInPlace => Self::KeepInPlace, + } + } +} + +impl From for ExpressionPlacement { + fn from(value: FFI_ExpressionPlacement) -> Self { + match value { + FFI_ExpressionPlacement::Literal => Self::Literal, + FFI_ExpressionPlacement::Column => Self::Column, + FFI_ExpressionPlacement::MoveTowardsLeafNodes => Self::MoveTowardsLeafNodes, + FFI_ExpressionPlacement::KeepInPlace => Self::KeepInPlace, + } + } +} + +#[cfg(test)] +mod tests { + use datafusion::logical_expr::ExpressionPlacement; + + use super::FFI_ExpressionPlacement; + + fn test_round_trip_placement(placement: ExpressionPlacement) { + let ffi_placement: FFI_ExpressionPlacement = placement.into(); + let round_trip: ExpressionPlacement = ffi_placement.into(); + + assert_eq!(placement, round_trip); + } + + #[test] + fn test_all_round_trip_placement() { + test_round_trip_placement(ExpressionPlacement::Literal); + test_round_trip_placement(ExpressionPlacement::Column); + test_round_trip_placement(ExpressionPlacement::MoveTowardsLeafNodes); + test_round_trip_placement(ExpressionPlacement::KeepInPlace); + } +} diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index 03b3a7ab246c7..dcd0910ecb4e9 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -90,6 +90,8 @@ pub struct ForeignLibraryModule { pub create_timezone_udf: extern "C" fn() -> FFI_ScalarUDF, + pub create_placement_udf: extern "C" fn() -> FFI_ScalarUDF, + pub create_table_function: extern "C" fn(FFI_LogicalExtensionCodec) -> FFI_TableFunction, @@ -251,6 +253,7 @@ pub extern "C" fn datafusion_ffi_get_module() -> ForeignLibraryModule { create_scalar_udf: create_ffi_abs_func, create_nullary_udf: create_ffi_random_func, create_timezone_udf: udf_udaf_udwf::create_timezone_func, + create_placement_udf: udf_udaf_udwf::create_placement_func, create_table_function: create_ffi_table_func, create_sum_udaf: create_ffi_sum_func, create_stddev_udaf: create_ffi_stddev_func, diff --git a/datafusion/ffi/src/tests/udf_udaf_udwf.rs b/datafusion/ffi/src/tests/udf_udaf_udwf.rs index 399a2cc6be5cd..b393f5db3a506 100644 --- a/datafusion/ffi/src/tests/udf_udaf_udwf.rs +++ b/datafusion/ffi/src/tests/udf_udaf_udwf.rs @@ -21,8 +21,8 @@ use arrow_schema::DataType; use datafusion_catalog::TableFunctionImpl; use datafusion_common::ScalarValue; use datafusion_expr::{ - AggregateUDF, ColumnarValue, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, - Volatility, WindowUDF, + AggregateUDF, ColumnarValue, ExpressionPlacement, ScalarFunctionArgs, ScalarUDF, + ScalarUDFImpl, Signature, Volatility, WindowUDF, }; use datafusion_functions::math::abs::AbsFunc; use datafusion_functions::math::random::RandomFunc; @@ -112,6 +112,56 @@ pub(crate) extern "C" fn create_timezone_func() -> FFI_ScalarUDF { udf.into() } +#[derive(Debug, PartialEq, Eq, Hash)] +struct PlacementUDF { + signature: Signature, +} + +impl ScalarUDFImpl for PlacementUDF { + fn name(&self) -> &str { + "placement_udf" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type( + &self, + _arg_types: &[DataType], + ) -> datafusion_common::Result { + Ok(DataType::Int64) + } + + fn invoke_with_args( + &self, + _args: ScalarFunctionArgs, + ) -> datafusion_common::Result { + datafusion_common::internal_err!("placement_udf is not meant to be invoked") + } + + fn placement(&self, args: &[ExpressionPlacement]) -> ExpressionPlacement { + // Push to the leaves only for a (Column, Literal) pairing, so the + // test catches dropped, reordered, or truncated arguments. + if matches!( + args, + [ExpressionPlacement::Column, ExpressionPlacement::Literal] + ) { + ExpressionPlacement::MoveTowardsLeafNodes + } else { + ExpressionPlacement::KeepInPlace + } + } +} + +pub(crate) extern "C" fn create_placement_func() -> FFI_ScalarUDF { + let udf: Arc = Arc::new(ScalarUDF::from(PlacementUDF { + signature: Signature::uniform(1, vec![DataType::Int64], Volatility::Immutable), + })); + + udf.into() +} + pub(crate) extern "C" fn create_ffi_table_func( codec: FFI_LogicalExtensionCodec, ) -> FFI_TableFunction { diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 1600bef39da45..272afdb6abfb1 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -64,7 +64,6 @@ pub struct FFI_GroupsAccumulator { accumulator: &mut Self, values: SVec, group_indices: SVec, - opt_filter: FFI_Option, total_num_groups: usize, ) -> FFI_Result<()>, @@ -195,21 +194,14 @@ unsafe extern "C" fn merge_batch_fn_wrapper( accumulator: &mut FFI_GroupsAccumulator, values: SVec, group_indices: SVec, - opt_filter: FFI_Option, total_num_groups: usize, ) -> FFI_Result<()> { unsafe { let accumulator = accumulator.inner_mut(); let values = sresult_return!(process_values(values)); let group_indices: Vec = group_indices.into_iter().collect(); - let opt_filter = sresult_return!(process_opt_filter(opt_filter)); - sresult!(accumulator.merge_batch( - &values, - &group_indices, - opt_filter.as_ref(), - total_num_groups - )) + sresult!(accumulator.merge_batch(&values, &group_indices, total_num_groups)) } } @@ -379,7 +371,6 @@ impl GroupsAccumulator for ForeignGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { unsafe { @@ -388,20 +379,11 @@ impl GroupsAccumulator for ForeignGroupsAccumulator { .map(WrappedArray::try_from) .collect::, ArrowError>>()?; let group_indices = group_indices.iter().cloned().collect(); - let opt_filter = opt_filter - .map(|bool_array| to_ffi(&bool_array.to_data())) - .transpose()? - .map(|(array, schema)| WrappedArray { - array, - schema: WrappedSchema(schema), - }) - .into(); df_result!((self.accumulator.merge_batch)( &mut self.accumulator, values.into_iter().collect(), group_indices, - opt_filter, total_num_groups )) } @@ -517,8 +499,7 @@ mod tests { let second_states = vec![make_array(create_array!(Boolean, vec![false]).to_data())]; - let opt_filter = create_array!(Boolean, vec![true]); - foreign_accum.merge_batch(&second_states, &[0], Some(opt_filter.as_ref()), 1)?; + foreign_accum.merge_batch(&second_states, &[0], 1)?; let groups_bool = foreign_accum.evaluate(EmitTo::All)?; assert_eq!(groups_bool.len(), 1); assert_eq!( diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index ff18a30e4ba19..4fc22e859f9fb 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -28,8 +28,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::{DataFusionError, Result, internal_err}; use datafusion_expr::type_coercion::functions::fields_with_udf; use datafusion_expr::{ - ColumnarValue, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, - Signature, + ColumnarValue, ExpressionPlacement, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, + ScalarUDFImpl, Signature, }; use return_type_args::{ FFI_ReturnFieldArgs, ForeignReturnFieldArgs, ForeignReturnFieldArgsOwned, @@ -41,6 +41,7 @@ use stabby::vec::Vec as SVec; use crate::arrow_wrappers::{WrappedArray, WrappedSchema}; use crate::config::FFI_ConfigOptions; use crate::expr::columnar_value::FFI_ColumnarValue; +use crate::placement::FFI_ExpressionPlacement; use crate::util::{ FFI_Result, rvec_wrapped_to_vec_datatype, vec_datatype_to_rvec_wrapped, }; @@ -91,6 +92,14 @@ pub struct FFI_ScalarUDF { arg_types: SVec, ) -> FFI_Result>, + /// FFI equivalent to the `placement` of a [`ScalarUDFImpl`]. Returns the + /// placement hint for the underlying [`ScalarUDF`] given each argument's + /// placement. Infallible, so it returns the value directly, not an `FFI_Result`. + pub placement: unsafe extern "C" fn( + udf: &Self, + args: SVec, + ) -> FFI_ExpressionPlacement, + /// Used to create a clone on the provider of the udf. This should /// only need to be called by the receiver of the udf. pub clone: unsafe extern "C" fn(udf: &Self) -> Self, @@ -157,6 +166,18 @@ unsafe extern "C" fn coerce_types_fn_wrapper( sresult!(vec_datatype_to_rvec_wrapped(&return_types)) } +unsafe extern "C" fn placement_fn_wrapper( + udf: &FFI_ScalarUDF, + args: SVec, +) -> FFI_ExpressionPlacement { + let args = args + .into_iter() + .map(ExpressionPlacement::from) + .collect::>(); + + udf.inner().placement(&args).into() +} + unsafe extern "C" fn invoke_with_args_fn_wrapper( udf: &FFI_ScalarUDF, args: SVec, @@ -250,6 +271,7 @@ impl From> for FFI_ScalarUDF { invoke_with_args: invoke_with_args_fn_wrapper, return_field_from_args: return_field_from_args_fn_wrapper, coerce_types: coerce_types_fn_wrapper, + placement: placement_fn_wrapper, clone: clone_fn_wrapper, release: release_fn_wrapper, private_data: Box::into_raw(private_data) as *mut c_void, @@ -427,12 +449,59 @@ impl ScalarUDFImpl for ForeignScalarUDF { Ok(rvec_wrapped_to_vec_datatype(&result_types)?) } } + + fn placement(&self, args: &[ExpressionPlacement]) -> ExpressionPlacement { + let args = args + .iter() + .map(|p| FFI_ExpressionPlacement::from(*p)) + .collect::>(); + + let result = unsafe { (self.udf.placement)(&self.udf, args) }; + + result.into() + } } #[cfg(test)] mod tests { use super::*; + #[derive(Debug, PartialEq, Eq, Hash)] + struct PlacementUDF { + signature: Signature, + } + + impl ScalarUDFImpl for PlacementUDF { + fn name(&self) -> &str { + "placement_udf" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Int64) + } + + fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { + internal_err!("placement_udf is not meant to be invoked") + } + + fn placement(&self, args: &[ExpressionPlacement]) -> ExpressionPlacement { + // Push to the leaves only for a (Column, Literal) pairing, so the + // test catches dropped, reordered, or truncated arguments. + if matches!( + args, + [ExpressionPlacement::Column, ExpressionPlacement::Literal] + ) { + ExpressionPlacement::MoveTowardsLeafNodes + } else { + ExpressionPlacement::KeepInPlace + } + } + } + #[test] fn test_round_trip_scalar_udf() -> Result<()> { let original_udf = datafusion::functions::math::abs::AbsFunc::new(); @@ -467,4 +536,42 @@ mod tests { Ok(()) } + + #[test] + fn test_ffi_udf_placement_round_trip() -> Result<()> { + use datafusion_expr::Volatility; + + let original_udf = Arc::new(ScalarUDF::from(PlacementUDF { + signature: Signature::uniform( + 1, + vec![DataType::Int64], + Volatility::Immutable, + ), + })); + + let mut ffi_udf = FFI_ScalarUDF::from(original_udf); + + // Force the foreign path so the call travels through the FFI vtable + // rather than downcasting back to the original local type. + ffi_udf.library_marker_id = crate::mock_foreign_marker_id; + let foreign_udf: Arc = (&ffi_udf).into(); + assert!(foreign_udf.is::()); + + // Without the plumbing the override is dropped and every call is + // KeepInPlace. The three cases also check the arguments survive the + // round trip in order. + assert_eq!( + foreign_udf + .placement(&[ExpressionPlacement::Column, ExpressionPlacement::Literal]), + ExpressionPlacement::MoveTowardsLeafNodes + ); + assert_eq!( + foreign_udf + .placement(&[ExpressionPlacement::Literal, ExpressionPlacement::Column]), + ExpressionPlacement::KeepInPlace + ); + assert_eq!(foreign_udf.placement(&[]), ExpressionPlacement::KeepInPlace); + + Ok(()) + } } diff --git a/datafusion/ffi/tests/ffi_udf.rs b/datafusion/ffi/tests/ffi_udf.rs index 6e6cb31f53133..dffaf83c479b1 100644 --- a/datafusion/ffi/tests/ffi_udf.rs +++ b/datafusion/ffi/tests/ffi_udf.rs @@ -23,7 +23,7 @@ mod tests { use arrow::datatypes::DataType; use datafusion::common::record_batch; use datafusion::error::Result; - use datafusion::logical_expr::{ScalarUDF, ScalarUDFImpl}; + use datafusion::logical_expr::{ExpressionPlacement, ScalarUDF, ScalarUDFImpl}; use datafusion::prelude::{SessionContext, col}; use datafusion_execution::config::SessionConfig; use datafusion_expr::lit; @@ -91,6 +91,31 @@ mod tests { Ok(()) } + /// This test validates that a producer's `placement` override survives the + /// FFI boundary instead of collapsing to the default `KeepInPlace`. + #[tokio::test] + async fn test_scalar_udf_placement() -> Result<()> { + let module = get_module()?; + + let ffi_placement_func = (module.create_placement_udf)(); + let foreign_func: Arc = (&ffi_placement_func).into(); + + // The override pushes to the leaves only for (Column, Literal), so these + // also check the arguments cross the boundary in order. + assert_eq!( + foreign_func + .placement(&[ExpressionPlacement::Column, ExpressionPlacement::Literal]), + ExpressionPlacement::MoveTowardsLeafNodes + ); + assert_eq!( + foreign_func + .placement(&[ExpressionPlacement::Literal, ExpressionPlacement::Column]), + ExpressionPlacement::KeepInPlace + ); + + Ok(()) + } + #[tokio::test] async fn test_config_on_scalar_udf() -> Result<()> { let module = get_module()?; diff --git a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs index d370d59c90012..60fe0388c430f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/count_distinct/groups.rs @@ -160,7 +160,6 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - _opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> datafusion_common::Result<()> { debug_assert_eq!(values.len(), 1); diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index ad2a21bb4733c..b412b4ffe09f2 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -375,13 +375,12 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { self.invoke_per_accumulator( values, group_indices, - opt_filter, + None, total_num_groups, |accumulator, values_to_accumulate| { accumulator.merge_batch(values_to_accumulate)?; diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs index d1d8924a2c3e8..afb1dec24a484 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs @@ -132,11 +132,10 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { // update / merge are the same - self.update_batch(values, group_indices, opt_filter, total_num_groups) + self.update_batch(values, group_indices, None, total_num_groups) } fn size(&self) -> usize { 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 a81b89e1e46f1..474899d8f3c6a 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 @@ -131,11 +131,10 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { // update / merge are the same - self.update_batch(values, group_indices, opt_filter, total_num_groups) + self.update_batch(values, group_indices, None, total_num_groups) } /// Converts an input batch directly to a state batch diff --git a/datafusion/functions-aggregate/benches/first_last.rs b/datafusion/functions-aggregate/benches/first_last.rs index 1d18e1c7dcd44..8f28e126a4009 100644 --- a/datafusion/functions-aggregate/benches/first_last.rs +++ b/datafusion/functions-aggregate/benches/first_last.rs @@ -235,7 +235,6 @@ fn merge_bench( Arc::clone(&is_set), ], &group_indices, - opt_filter, num_groups, ) .unwrap(), diff --git a/datafusion/functions-aggregate/src/approx_distinct.rs b/datafusion/functions-aggregate/src/approx_distinct.rs index 38b902964f546..0c5a438454092 100644 --- a/datafusion/functions-aggregate/src/approx_distinct.rs +++ b/datafusion/functions-aggregate/src/approx_distinct.rs @@ -705,15 +705,8 @@ impl GroupsAccumulator for HllGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - // Since aggregate filter should be applied in partial stage, in final stage there should be no filter - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { - assert!( - opt_filter.is_none(), - "aggregate filter should be applied in partial stage, there should be no filter in final stage" - ); - self.ensure_groups(total_num_groups); let states = downcast_value!(values[0], BinaryArray); let mut delta: isize = 0; diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 8ed3fbf8c3d26..1dd111f9182c9 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -732,7 +732,6 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - _opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 1, "one argument to merge_batch"); @@ -2019,7 +2018,7 @@ mod tests { // Merge acc2's state into acc1 let state = acc2.state(EmitTo::All)?; - acc1.merge_batch(&state, &[0, 1], None, 2)?; + acc1.merge_batch(&state, &[0, 1], 2)?; // Another update_batch on acc1 after the merge let values: ArrayRef = Arc::new(Int32Array::from(vec![5, 6])); @@ -2088,7 +2087,7 @@ mod tests { // Feed state into a new accumulator via merge_batch let mut acc2 = ArrayAggGroupsAccumulator::new(DataType::Int32, false); - acc2.merge_batch(&state, &[0, 0, 1], None, 2)?; + acc2.merge_batch(&state, &[0, 0, 1], 2)?; // Group 0 received rows 0 ([1]) and 1 ([NULL]) → [1, NULL] let vals = eval_i32_lists(&mut acc2, EmitTo::All)?; @@ -2118,7 +2117,7 @@ mod tests { // Feed state into a new accumulator via merge_batch let mut acc2 = ArrayAggGroupsAccumulator::new(DataType::Int32, true); - acc2.merge_batch(&state, &[0, 0, 1, 1], None, 2)?; + acc2.merge_batch(&state, &[0, 0, 1, 1], 2)?; // Group 0: received [1] and null (skipped) → [1] let vals = eval_i32_lists(&mut acc2, EmitTo::All)?; diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index ddeb9b0870a16..06c76946343dc 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -908,7 +908,6 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 2, "two arguments to merge_batch"); @@ -920,7 +919,7 @@ where self.null_state.accumulate( group_indices, partial_counts, - opt_filter, + None, total_num_groups, |group_index, partial_count| { // SAFETY: group_index is guaranteed to be in bounds @@ -934,7 +933,7 @@ where self.null_state.accumulate( group_indices, partial_sums, - opt_filter, + None, total_num_groups, |group_index, new_value: ::Native| { // SAFETY: group_index is guaranteed to be in bounds diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index 2621fcf0bf3c7..7fcf4bb61ffad 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -281,6 +281,10 @@ impl Accumulator for CorrelationAccumulator { self.stddev2.retract_batch(&values[1..2])?; Ok(()) } + + fn supports_retract_batch(&self) -> bool { + true + } } #[derive(Default)] @@ -489,7 +493,6 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { // Resize vectors to accommodate total number of groups @@ -508,11 +511,6 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { let partial_sum_xx = values[4].as_primitive::(); let partial_sum_yy = values[5].as_primitive::(); - assert!( - opt_filter.is_none(), - "aggregate filter should be applied in partial stage, there should be no filter in final stage" - ); - accumulate_correlation_states( group_indices, ( diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index eab36d4951a9c..f0ce8c82a1bb2 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -665,8 +665,6 @@ impl GroupsAccumulator for CountGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - // Since aggregate filter should be applied in partial stage, in final stage there should be no filter - _opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 1, "one argument to merge_batch"); diff --git a/datafusion/functions-aggregate/src/covariance.rs b/datafusion/functions-aggregate/src/covariance.rs index 18d602ab33940..bd7c8a039076a 100644 --- a/datafusion/functions-aggregate/src/covariance.rs +++ b/datafusion/functions-aggregate/src/covariance.rs @@ -305,6 +305,14 @@ impl Accumulator for CovarianceAccumulator { _ => continue, }; + if self.count <= 1 { + self.count = 0; + self.mean1 = 0.0; + self.mean2 = 0.0; + self.algo_const = 0.0; + continue; + } + let new_count = self.count - 1; let delta1 = self.mean1 - value1; let new_mean1 = delta1 / new_count as f64 + self.mean1; @@ -373,4 +381,8 @@ impl Accumulator for CovarianceAccumulator { fn size(&self) -> usize { size_of_val(self) } + + fn supports_retract_batch(&self) -> bool { + true + } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 1935f29c4cfe8..cecb277cb844a 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -671,7 +671,6 @@ impl GroupsAccumulator for FirstLastGroupsAccumulator, total_num_groups: usize, ) -> Result<()> { self.resize_states(total_num_groups); @@ -690,7 +689,7 @@ impl GroupsAccumulator for FirstLastGroupsAccumulator().unwrap(); + // group 0 keeps merged value=1 (ordering=1). + // group 1 keeps merged value=-6 (ordering=-6 < 6, so -6 is "first"). + // group 2 had no merged value (is_set=false), so update_batch value=6 wins. let expect: PrimitiveArray = - Int64Array::from(vec![Some(1), Some(6), Some(6), None]); + Int64Array::from(vec![Some(1), Some(-6), Some(6), None]); assert_eq!(eval_result, &expect); @@ -1680,7 +1677,7 @@ mod tests { group_acc.compute_size_of_orderings() ); - group_acc.merge_batch(&s, &Vec::from_iter(0..s[0].len()), None, 100)?; + group_acc.merge_batch(&s, &Vec::from_iter(0..s[0].len()), 100)?; assert_eq!( group_acc.size_of_orderings, group_acc.compute_size_of_orderings() @@ -1753,12 +1750,7 @@ mod tests { ]; assert_eq!(state, expected_state); - group_acc.merge_batch( - &state, - &[0, 1, 2], - Some(&BooleanArray::from(vec![true, false, false])), - 3, - )?; + group_acc.merge_batch(&state, &[0, 1, 2], 3)?; val_with_orderings.clear(); val_with_orderings.push(Arc::new(Int64Array::from(vec![66, 6]))); @@ -1769,6 +1761,10 @@ mod tests { let binding = group_acc.evaluate(EmitTo::All)?; let eval_result = binding.as_any().downcast_ref::().unwrap(); + // group 0: merged value=1 (ordering=1, is_set=true), update not called. + // group 1: merged value=-6 (ordering=-6, is_set=true); update ordering=66 > -6 + // → LAST_VALUE keeps the higher ordering, so group 1 becomes 66. + // group 2: is_set=false after merge; update_batch sets it to 6. let expect: PrimitiveArray = Int64Array::from(vec![Some(1), Some(66), Some(6), None]); diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index e7e7d03937f12..4a0da10f51845 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -393,8 +393,6 @@ impl GroupsAccumulator for MedianGroupsAccumulator, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 1, "one argument to merge_batch"); 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 b56c2106e32b5..7a3c605d82e4d 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -309,11 +309,10 @@ impl GroupsAccumulator for MinMaxBytesAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { // min/max are their own states (no transition needed) - self.update_batch(values, group_indices, opt_filter, total_num_groups) + self.update_batch(values, group_indices, None, total_num_groups) } fn convert_to_state( diff --git a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs index 7c94e7f5738be..10580ac18d3ec 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs @@ -134,11 +134,10 @@ impl GroupsAccumulator for MinMaxStructAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { // min/max are their own states (no transition needed) - self.update_batch(values, group_indices, opt_filter, total_num_groups) + self.update_batch(values, group_indices, None, total_num_groups) } fn convert_to_state( diff --git a/datafusion/functions-aggregate/src/percentile_cont.rs b/datafusion/functions-aggregate/src/percentile_cont.rs index 714988bde2acf..e8e6fd127e65d 100644 --- a/datafusion/functions-aggregate/src/percentile_cont.rs +++ b/datafusion/functions-aggregate/src/percentile_cont.rs @@ -537,8 +537,6 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - // Since aggregate filter should be applied in partial stage, in final stage there should be no filter - _opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 1, "one argument to merge_batch"); diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 68e38a3b8db07..f0482b23d12a7 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -329,11 +329,10 @@ impl GroupsAccumulator for StddevGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&arrow::array::BooleanArray>, total_num_groups: usize, ) -> Result<()> { self.variance - .merge_batch(values, group_indices, opt_filter, total_num_groups) + .merge_batch(values, group_indices, total_num_groups) } fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index f0757818afb93..6b0665f479d78 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -413,11 +413,10 @@ impl GroupsAccumulator for StringAggGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { // State is always LargeUtf8, which update_batch already handles. - self.update_batch(values, group_indices, opt_filter, total_num_groups) + self.update_batch(values, group_indices, None, total_num_groups) } fn convert_to_state( @@ -898,7 +897,7 @@ mod tests { // Simulate a second accumulator's state (LargeUtf8 partial strings) let partial_state: ArrayRef = Arc::new(LargeStringArray::from(vec!["c,d", "e"])); - acc.merge_batch(&[partial_state], &[0, 1], None, 2)?; + acc.merge_batch(&[partial_state], &[0, 1], 2)?; let result = evaluate_groups(&mut acc, EmitTo::All); assert_eq!( diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index c3c2e5e0b9677..1a1f9c59a2964 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -303,13 +303,18 @@ impl AggregateUDFImpl for Sum { args: AccumulatorArgs, ) -> Result> { if args.is_distinct { - // distinct path: use our sliding‐window distinct‐sum - macro_rules! helper_distinct { - ($t:ty, $dt:expr) => { - Ok(Box::new(SlidingDistinctSumAccumulator::try_new(&$dt)?)) - }; + // distinct path: [`SlidingDistinctSumAccumulator`] only implements + // Int64, so gate the supported type here rather than dispatching + // through `downcast_sum!`, which accepts every SUM type + match args.return_field.data_type() { + DataType::Int64 => Ok(Box::new(SlidingDistinctSumAccumulator::try_new( + &DataType::Int64, + )?)), + _ => not_impl_err!( + "SUM(DISTINCT) over sliding window frames is only supported for Int64, got {}", + args.expr_fields[0].data_type() + ), } - downcast_sum!(args, helper_distinct) } else { // non‐distinct path: existing sliding sum macro_rules! helper { @@ -525,7 +530,9 @@ impl SlidingDistinctSumAccumulator { pub fn try_new(data_type: &DataType) -> Result { // TODO support other numeric types if *data_type != DataType::Int64 { - return exec_err!("SlidingDistinctSumAccumulator only supports Int64"); + return exec_err!( + "SlidingDistinctSumAccumulator only supports Int64, got {data_type}" + ); } Ok(Self { counts: HashMap::default(), diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index ce3e00b9ffd91..551fcfe120352 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -348,6 +348,13 @@ impl Accumulator for VarianceAccumulator { fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { let arr = as_float64_array(&values[0])?; for value in arr.iter().flatten() { + if self.count <= 1 { + self.count = 0; + self.mean = 0.0; + self.m2 = 0.0; + continue; + } + let new_count = self.count - 1; let delta1 = self.mean - value; let new_mean = delta1 / new_count as f64 + self.mean; @@ -521,8 +528,6 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { &mut self, values: &[ArrayRef], group_indices: &[usize], - // Since aggregate filter should be applied in partial stage, in final stage there should be no filter - _opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 3, "two arguments to merge_batch"); @@ -666,8 +671,8 @@ mod tests { Arc::new(Float64Array::from(vec![1.0])), ]; let mut acc = VarianceGroupsAccumulator::new(StatsType::Sample); - acc.merge_batch(&state_1, &[0], None, 1)?; - acc.merge_batch(&state_2, &[0], None, 1)?; + acc.merge_batch(&state_1, &[0], 1)?; + acc.merge_batch(&state_2, &[0], 1)?; let result = acc.evaluate(EmitTo::All)?; let result = result.as_any().downcast_ref::().unwrap(); assert_eq!(result.len(), 1); diff --git a/datafusion/functions/src/core/file_row_index.rs b/datafusion/functions/src/core/file_row_index.rs new file mode 100644 index 0000000000000..7b2667a8b8768 --- /dev/null +++ b/datafusion/functions/src/core/file_row_index.rs @@ -0,0 +1,96 @@ +// 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. + +//! Implementation of the `file_row_index` scalar function. + +use arrow::datatypes::DataType; +use datafusion_common::utils::take_function_args; +use datafusion_common::{Result, exec_err}; +use datafusion_doc::Documentation; +use datafusion_expr::{ + ColumnarValue, ExpressionPlacement, ScalarFunctionArgs, ScalarUDFImpl, Signature, + Volatility, +}; +use datafusion_macros::user_doc; + +/// Scalar UDF implementation for `file_row_index()`. +/// +/// File sources that can expose per-file row indexes rewrite this placeholder +/// function into a source-provided physical expression. Direct evaluation +/// returns an error because there is no file context outside a scan. +#[user_doc( + doc_section(label = "Other Functions"), + description = r#"Returns the zero-based row offset within the source file +that produced the current row. + +The value is scoped to one file, so rows from different files in the same scan +can have the same row index. This function is intended to be rewritten at +file-scan time. If the input file is not known (for example, if this function +is evaluated outside a file scan, or was not pushed down into one), direct +evaluation returns an error. +"#, + syntax_example = "file_row_index()", + sql_example = r#"```sql +SELECT file_row_index() FROM t; +```"# +)] +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct FileRowIndexFunc { + signature: Signature, +} + +impl Default for FileRowIndexFunc { + fn default() -> Self { + Self::new() + } +} + +impl FileRowIndexFunc { + pub fn new() -> Self { + Self { + signature: Signature::nullary(Volatility::Volatile), + } + } +} + +impl ScalarUDFImpl for FileRowIndexFunc { + fn name(&self) -> &str { + "file_row_index" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, args: &[DataType]) -> Result { + let [] = take_function_args(self.name(), args)?; + Ok(DataType::Int64) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + let [] = take_function_args(self.name(), args.args)?; + exec_err!("file_row_index() is source dependent and cannot be evaluated directly") + } + + fn placement(&self, _args: &[ExpressionPlacement]) -> ExpressionPlacement { + ExpressionPlacement::MoveTowardsLeafNodes + } + + fn documentation(&self) -> Option<&Documentation> { + self.doc() + } +} diff --git a/datafusion/functions/src/core/mod.rs b/datafusion/functions/src/core/mod.rs index 5657f9d88810c..4665eca99ebef 100644 --- a/datafusion/functions/src/core/mod.rs +++ b/datafusion/functions/src/core/mod.rs @@ -28,6 +28,7 @@ pub mod arrowtypeof; pub mod cast_to_type; pub mod coalesce; pub mod expr_ext; +pub mod file_row_index; pub mod getfield; pub mod greatest; mod greatest_least_utils; @@ -67,6 +68,7 @@ make_udf_function!(version::VersionFunc, version); make_udf_function!(arrow_metadata::ArrowMetadataFunc, arrow_metadata); make_udf_function!(with_metadata::WithMetadataFunc, with_metadata); make_udf_function!(arrow_field::ArrowFieldFunc, arrow_field); +make_udf_function!(file_row_index::FileRowIndexFunc, file_row_index); pub mod expr_fn { use datafusion_expr::{Expr, Literal}; @@ -143,6 +145,9 @@ pub mod expr_fn { union_tag, "Returns the name of the currently selected field in the union", arg1 + ),( + file_row_index, + "Returns the offset of the row within its source file", )); #[doc = "Returns the value of the field with the given name from the struct"] @@ -196,5 +201,6 @@ pub fn functions() -> Vec> { union_tag(), version(), r#struct(), + file_row_index(), ] } diff --git a/datafusion/functions/src/datetime/date_bin.rs b/datafusion/functions/src/datetime/date_bin.rs index 38b491e42bcbd..06ffd8ba5b3c6 100644 --- a/datafusion/functions/src/datetime/date_bin.rs +++ b/datafusion/functions/src/datetime/date_bin.rs @@ -34,9 +34,7 @@ use arrow::datatypes::{ use arrow::error::ArrowError; use arrow::temporal_conversions::NANOSECONDS_IN_DAY; use datafusion_common::cast::as_primitive_array; -use datafusion_common::{ - DataFusionError, Result, ScalarValue, exec_err, not_impl_err, plan_err, -}; +use datafusion_common::{Result, ScalarValue, exec_err, not_impl_err, plan_err}; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ @@ -420,14 +418,44 @@ fn date_bin_months_interval(stride_months: i64, source: i64, origin: i64) -> Res } fn to_utc_date_time(nanos: i64) -> Result> { - let secs = nanos / NANOS_PER_SEC; - let nsec = (nanos % NANOS_PER_SEC) as u32; + // Keep negative sub-second values normalized as seconds + non-negative nanos. + let secs = nanos.div_euclid(NANOS_PER_SEC); + let nsec = nanos.rem_euclid(NANOS_PER_SEC) as u32; match DateTime::from_timestamp(secs, nsec) { Some(dt) => Ok(dt), None => exec_err!("Invalid timestamp value"), } } +fn timestamp_scale() -> i64 { + match T::UNIT { + Nanosecond => 1, + Microsecond => NANOS_PER_MICRO, + Millisecond => NANOS_PER_MILLI, + Second => NANOSECONDS, + } +} + +// Scale to nanoseconds and report overflow as a normal error. +fn checked_scale_to_nanos(x: i64, scale: i64) -> Result { + match x.checked_mul(scale) { + Some(scaled) => Ok(scaled), + None => exec_err!("date_bin timestamp value {x} * scale {scale} overflows i64"), + } +} + +fn validate_time_stride(stride: &Interval) -> Result<()> { + match stride { + Interval::Months(m) if *m > 0 => { + exec_err!("DATE_BIN stride for TIME input must be less than 1 day") + } + Interval::Nanoseconds(ns) if *ns >= NANOSECONDS_IN_DAY => { + exec_err!("DATE_BIN stride for TIME input must be less than 1 day") + } + _ => Ok(()), + } +} + // Supported intervals: // 1. IntervalDayTime: this means that the stride is in days, hours, minutes, seconds and milliseconds // We will assume month interval won't be converted into this type @@ -498,83 +526,20 @@ fn date_bin_impl( (*v, false) } ColumnarValue::Scalar(ScalarValue::Time32Millisecond(Some(v))) => { - match stride { - Interval::Months(m) => { - if m > 0 { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - Interval::Nanoseconds(ns) => { - if ns >= NANOSECONDS_IN_DAY { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - } - - (*v as i64 * NANOS_PER_MILLI, true) + validate_time_stride(&stride)?; + // TIME origins can come from reinterpret casts, so scale defensively. + (checked_scale_to_nanos(*v as i64, NANOS_PER_MILLI)?, true) } ColumnarValue::Scalar(ScalarValue::Time32Second(Some(v))) => { - match stride { - Interval::Months(m) => { - if m > 0 { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - Interval::Nanoseconds(ns) => { - if ns >= NANOSECONDS_IN_DAY { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - } - - (*v as i64 * NANOS_PER_SEC, true) + validate_time_stride(&stride)?; + (checked_scale_to_nanos(*v as i64, NANOS_PER_SEC)?, true) } ColumnarValue::Scalar(ScalarValue::Time64Microsecond(Some(v))) => { - match stride { - Interval::Months(m) => { - if m > 0 { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - Interval::Nanoseconds(ns) => { - if ns >= NANOSECONDS_IN_DAY { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - } - - (*v * NANOS_PER_MICRO, true) + validate_time_stride(&stride)?; + (checked_scale_to_nanos(*v, NANOS_PER_MICRO)?, true) } ColumnarValue::Scalar(ScalarValue::Time64Nanosecond(Some(v))) => { - match stride { - Interval::Months(m) => { - if m > 0 { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - Interval::Nanoseconds(ns) => { - if ns >= NANOSECONDS_IN_DAY { - return exec_err!( - "DATE_BIN stride for TIME input must be less than 1 day" - ); - } - } - } - + validate_time_stride(&stride)?; (*v, true) } ColumnarValue::Scalar(v) => { @@ -597,91 +562,49 @@ fn date_bin_impl( return exec_err!("DATE_BIN stride must be non-zero"); } - fn timestamp_scale() -> i64 { - match T::UNIT { - Nanosecond => 1, - Microsecond => NANOS_PER_MICRO, - Millisecond => NANOS_PER_MILLI, - Second => NANOSECONDS, - } - } - - fn timestamp_scale_overflow_error(x: i64) -> DataFusionError { - DataFusionError::Execution(format!( - "DATE_BIN source timestamp {x} cannot be represented in nanoseconds" - )) + fn transform_scalar_with_stride( + value: Option, + origin: i64, + stride: i64, + stride_fn: BinFunction, + ) -> Option { + let scale = timestamp_scale::(); + value + .and_then(|val| val.checked_mul(scale)) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| binned / scale) } Ok(match array { ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(v, tz_opt)) => { - let scale = timestamp_scale::(); ColumnarValue::Scalar(ScalarValue::TimestampNanosecond( - match *v { - Some(val) => { - let scaled = val - .checked_mul(scale) - .ok_or_else(|| timestamp_scale_overflow_error(val))?; - match stride_fn(stride, scaled, origin) { - Ok(result) => Some(result / scale), - Err(_) => None, - } - } - None => None, - }, + transform_scalar_with_stride::( + *v, origin, stride, stride_fn, + ), tz_opt.clone(), )) } ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond(v, tz_opt)) => { - let scale = timestamp_scale::(); ColumnarValue::Scalar(ScalarValue::TimestampMicrosecond( - match *v { - Some(val) => { - let scaled = val - .checked_mul(scale) - .ok_or_else(|| timestamp_scale_overflow_error(val))?; - match stride_fn(stride, scaled, origin) { - Ok(result) => Some(result / scale), - Err(_) => None, - } - } - None => None, - }, + transform_scalar_with_stride::( + *v, origin, stride, stride_fn, + ), tz_opt.clone(), )) } ColumnarValue::Scalar(ScalarValue::TimestampMillisecond(v, tz_opt)) => { - let scale = timestamp_scale::(); ColumnarValue::Scalar(ScalarValue::TimestampMillisecond( - match *v { - Some(val) => { - let scaled = val - .checked_mul(scale) - .ok_or_else(|| timestamp_scale_overflow_error(val))?; - match stride_fn(stride, scaled, origin) { - Ok(result) => Some(result / scale), - Err(_) => None, - } - } - None => None, - }, + transform_scalar_with_stride::( + *v, origin, stride, stride_fn, + ), tz_opt.clone(), )) } ColumnarValue::Scalar(ScalarValue::TimestampSecond(v, tz_opt)) => { - let scale = timestamp_scale::(); ColumnarValue::Scalar(ScalarValue::TimestampSecond( - match *v { - Some(val) => { - let scaled = val - .checked_mul(scale) - .ok_or_else(|| timestamp_scale_overflow_error(val))?; - match stride_fn(stride, scaled, origin) { - Ok(result) => Some(result / scale), - Err(_) => None, - } - } - None => None, - }, + transform_scalar_with_stride::( + *v, origin, stride, stride_fn, + ), tz_opt.clone(), )) } @@ -689,39 +612,30 @@ fn date_bin_impl( if !is_time { return exec_err!("DATE_BIN with Time32 source requires Time32 origin"); } - let result = v.and_then(|x| { - match stride_fn(stride, x as i64 * NANOS_PER_MILLI, origin) { - Ok(binned_nanos) => { - let nanos = binned_nanos % (NANOSECONDS_IN_DAY); - Some((nanos / NANOS_PER_MILLI) as i32) - } - Err(_) => None, - } - }); + let result = v + .and_then(|x| (x as i64).checked_mul(NANOS_PER_MILLI)) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| ((binned % NANOSECONDS_IN_DAY) / NANOS_PER_MILLI) as i32); ColumnarValue::Scalar(ScalarValue::Time32Millisecond(result)) } ColumnarValue::Scalar(ScalarValue::Time32Second(v)) => { if !is_time { return exec_err!("DATE_BIN with Time32 source requires Time32 origin"); } - let result = v.and_then(|x| { - match stride_fn(stride, x as i64 * NANOS_PER_SEC, origin) { - Ok(binned_nanos) => { - let nanos = binned_nanos % (NANOSECONDS_IN_DAY); - Some((nanos / NANOS_PER_SEC) as i32) - } - Err(_) => None, - } - }); + let result = v + .and_then(|x| (x as i64).checked_mul(NANOS_PER_SEC)) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| ((binned % NANOSECONDS_IN_DAY) / NANOS_PER_SEC) as i32); ColumnarValue::Scalar(ScalarValue::Time32Second(result)) } ColumnarValue::Scalar(ScalarValue::Time64Nanosecond(v)) => { if !is_time { return exec_err!("DATE_BIN with Time64 source requires Time64 origin"); } - let result = v.and_then(|x| match stride_fn(stride, x, origin) { - Ok(binned_nanos) => Some(binned_nanos % (NANOSECONDS_IN_DAY)), - Err(_) => None, + let result = v.and_then(|x| { + stride_fn(stride, x, origin) + .map(|binned| binned % NANOSECONDS_IN_DAY) + .ok() }); ColumnarValue::Scalar(ScalarValue::Time64Nanosecond(result)) } @@ -729,14 +643,10 @@ fn date_bin_impl( if !is_time { return exec_err!("DATE_BIN with Time64 source requires Time64 origin"); } - let result = - v.and_then(|x| match stride_fn(stride, x * NANOS_PER_MICRO, origin) { - Ok(binned_nanos) => { - let nanos = binned_nanos % (NANOSECONDS_IN_DAY); - Some(nanos / NANOS_PER_MICRO) - } - Err(_) => None, - }); + let result = v + .and_then(|x| x.checked_mul(NANOS_PER_MICRO)) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| (binned % NANOSECONDS_IN_DAY) / NANOS_PER_MICRO); ColumnarValue::Scalar(ScalarValue::Time64Microsecond(result)) } ColumnarValue::Array(array) => { @@ -753,22 +663,12 @@ fn date_bin_impl( let array = as_primitive_array::(array)?; let scale = timestamp_scale::(); - let values = array - .iter() - .map(|val| match val { - Some(val) => { - let scaled = val - .checked_mul(scale) - .ok_or_else(|| timestamp_scale_overflow_error(val))?; - Ok(stride_fn(stride, scaled, origin) - .ok() - .map(|binned| binned / scale)) - } - None => Ok(None), - }) - .collect::>>()?; - - let result = PrimitiveArray::::from_iter(values); + // Per-row errors become NULL, matching scalar behavior. + let result: PrimitiveArray = array.unary_opt(|val| { + val.checked_mul(scale) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| binned / scale) + }); let array = result.with_timezone_opt(tz_opt.clone()); Ok(ColumnarValue::Array(Arc::new(array))) @@ -803,14 +703,15 @@ fn date_bin_impl( } let array = array.as_primitive::(); let result: PrimitiveArray = - array.try_unary(|x| { - stride_fn(stride, x as i64 * NANOS_PER_MILLI, origin) - .map(|binned_nanos| { - let nanos = binned_nanos % (NANOSECONDS_IN_DAY); - (nanos / NANOS_PER_MILLI) as i32 + array.unary_opt(|x| { + (x as i64) + .checked_mul(NANOS_PER_MILLI) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| { + ((binned % NANOSECONDS_IN_DAY) / NANOS_PER_MILLI) + as i32 }) - .map_err(|e| ArrowError::ComputeError(e.to_string())) - })?; + }); ColumnarValue::Array(Arc::new(result)) } Time32(Second) => { @@ -820,15 +721,14 @@ fn date_bin_impl( ); } let array = array.as_primitive::(); - let result: PrimitiveArray = - array.try_unary(|x| { - stride_fn(stride, x as i64 * NANOS_PER_SEC, origin) - .map(|binned_nanos| { - let nanos = binned_nanos % (NANOSECONDS_IN_DAY); - (nanos / NANOS_PER_SEC) as i32 - }) - .map_err(|e| ArrowError::ComputeError(e.to_string())) - })?; + let result: PrimitiveArray = array.unary_opt(|x| { + (x as i64) + .checked_mul(NANOS_PER_SEC) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| { + ((binned % NANOSECONDS_IN_DAY) / NANOS_PER_SEC) as i32 + }) + }); ColumnarValue::Array(Arc::new(result)) } Time64(Microsecond) => { @@ -839,14 +739,13 @@ fn date_bin_impl( } let array = array.as_primitive::(); let result: PrimitiveArray = - array.try_unary(|x| { - stride_fn(stride, x * NANOS_PER_MICRO, origin) - .map(|binned_nanos| { - let nanos = binned_nanos % (NANOSECONDS_IN_DAY); - nanos / NANOS_PER_MICRO + array.unary_opt(|x| { + x.checked_mul(NANOS_PER_MICRO) + .and_then(|scaled| stride_fn(stride, scaled, origin).ok()) + .map(|binned| { + (binned % NANOSECONDS_IN_DAY) / NANOS_PER_MICRO }) - .map_err(|e| ArrowError::ComputeError(e.to_string())) - })?; + }); ColumnarValue::Array(Arc::new(result)) } Time64(Nanosecond) => { @@ -857,11 +756,11 @@ fn date_bin_impl( } let array = array.as_primitive::(); let result: PrimitiveArray = - array.try_unary(|x| { + array.unary_opt(|x| { stride_fn(stride, x, origin) .map(|binned_nanos| binned_nanos % (NANOSECONDS_IN_DAY)) - .map_err(|e| ArrowError::ComputeError(e.to_string())) - })?; + .ok() + }); ColumnarValue::Array(Arc::new(result)) } _ => { @@ -917,6 +816,31 @@ mod tests { DateBinFunc::new().invoke_with_args(args) } + fn assert_null_scalar(value: ColumnarValue, expected_type: DataType) { + let ColumnarValue::Scalar(value) = value else { + panic!("expected scalar, got {value:?}"); + }; + assert_eq!(value.data_type(), expected_type); + assert!(value.is_null(), "expected NULL, got {value:?}"); + } + + fn assert_array_null_then_valid(value: ColumnarValue, expected_type: DataType) { + let ColumnarValue::Array(array) = value else { + panic!("expected array, got {value:?}"); + }; + assert_eq!(array.data_type(), &expected_type); + assert!(array.is_null(0), "expected NULL at row 0"); + assert!(array.is_valid(1), "expected valid value at row 1"); + } + + fn assert_overflow_error(result: Result) { + let err = result.expect_err("expected overflow error"); + assert!( + err.strip_backtrace().contains("overflows i64"), + "unexpected error: {err}" + ); + } + #[test] fn test_date_bin() { let return_field = &Arc::new(Field::new( @@ -1433,6 +1357,97 @@ mod tests { } } + #[test] + fn test_date_bin_scale_overflow_returns_null() { + // Scaling non-nanosecond timestamps to nanoseconds can overflow. + use arrow::array::{ + ArrayRef, TimestampMicrosecondArray, TimestampMillisecondArray, + TimestampSecondArray, + }; + + let scalar_cases = [ + ScalarValue::TimestampSecond(Some(i64::MAX), None), + ScalarValue::TimestampMillisecond(Some(i64::MAX), None), + ScalarValue::TimestampMicrosecond(Some(i64::MAX), None), + ]; + for source in scalar_cases { + let expected_type = source.data_type(); + let return_field = Arc::new(Field::new("f", expected_type.clone(), true)); + let args = vec![ + ColumnarValue::Scalar(ScalarValue::new_interval_dt(1, 0)), + ColumnarValue::Scalar(source), + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(0), None)), + ]; + let result = invoke_date_bin_with_args(args, 1, &return_field) + .unwrap_or_else(|e| panic!("expected Ok for {expected_type}, got {e:?}")); + assert_null_scalar(result, expected_type); + } + + let array_cases: Vec = vec![ + Arc::new(TimestampSecondArray::from(vec![Some(i64::MAX), Some(0)])), + Arc::new(TimestampMillisecondArray::from(vec![ + Some(i64::MAX), + Some(0), + ])), + Arc::new(TimestampMicrosecondArray::from(vec![ + Some(i64::MAX), + Some(0), + ])), + ]; + for array in array_cases { + let dt = array.data_type().clone(); + let return_field = Arc::new(Field::new("f", dt.clone(), true)); + let args = vec![ + ColumnarValue::Scalar(ScalarValue::new_interval_dt(1, 0)), + ColumnarValue::Array(array), + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(0), None)), + ]; + let result = invoke_date_bin_with_args(args, 2, &return_field) + .unwrap_or_else(|e| panic!("expected Ok for {dt:?}, got {e:?}")); + assert_array_null_then_valid(result, dt); + } + } + + #[test] + fn test_date_bin_time64_micro_overflow_handling() { + // Time64(Microsecond) can hold out-of-range values after reinterpret casts. + use arrow::array::Time64MicrosecondArray; + + let data_type = DataType::Time64(TimeUnit::Microsecond); + let return_field = &Arc::new(Field::new("f", data_type.clone(), true)); + let stride = || ColumnarValue::Scalar(ScalarValue::new_interval_dt(0, 1000)); + let origin = || ColumnarValue::Scalar(ScalarValue::Time64Microsecond(Some(0))); + + // Out-of-range source values are per-row data, so they become NULL. + let args = vec![ + stride(), + ColumnarValue::Scalar(ScalarValue::Time64Microsecond(Some(i64::MAX))), + origin(), + ]; + let result = invoke_date_bin_with_args(args, 1, return_field).unwrap(); + assert_null_scalar(result, data_type.clone()); + + let array = Arc::new(Time64MicrosecondArray::from(vec![Some(i64::MAX), Some(0)])); + let args = vec![stride(), ColumnarValue::Array(array), origin()]; + let result = invoke_date_bin_with_args(args, 2, return_field).unwrap(); + assert_array_null_then_valid(result, data_type); + + let bad_origin = + || ColumnarValue::Scalar(ScalarValue::Time64Microsecond(Some(i64::MAX))); + + // Out-of-range origins are shared inputs, so they return an error. + let args = vec![ + stride(), + ColumnarValue::Scalar(ScalarValue::Time64Microsecond(Some(0))), + bad_origin(), + ]; + assert_overflow_error(invoke_date_bin_with_args(args, 1, return_field)); + + let array = Arc::new(Time64MicrosecondArray::from(vec![Some(0), Some(1)])); + let args = vec![stride(), ColumnarValue::Array(array), bad_origin()]; + assert_overflow_error(invoke_date_bin_with_args(args, 2, return_field)); + } + #[test] fn test_date_bin_compute_distance_rem_overflow() { // Regression for #22215: `time_diff % stride` panics with "attempt to diff --git a/datafusion/functions/src/math/common.rs b/datafusion/functions/src/math/common.rs new file mode 100644 index 0000000000000..9bb6f6fe1e35c --- /dev/null +++ b/datafusion/functions/src/math/common.rs @@ -0,0 +1,320 @@ +// 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. + +use arrow::array::ArrowNativeTypeOp; +use arrow::error::ArrowError; +use num_traits::{CheckedMul, CheckedNeg, Signed}; +use std::fmt::Display; +use std::mem::swap; +use std::ops::RemAssign; + +/// A gcd helper to compute GCD using Euclidean GCD algorithm +/// on non-negative numbers (scalars and decimals) +fn gcd_helper(a: T, b: T) -> Result +where + T: ArrowNativeTypeOp + RemAssign + CheckedNeg, +{ + debug_assert!(a >= T::ZERO); + debug_assert!(b >= T::ZERO); + let (mut a, mut b) = if a > b { (a, b) } else { (b, a) }; + + while b != T::ZERO { + swap(&mut a, &mut b); + b %= a; + } + + Ok(a) +} + +/// Computes gcd of two unsigned integers using Binary GCD algorithm +/// Faster, works with integers only +pub(crate) fn unsigned_gcd(mut a: u64, mut b: u64) -> u64 { + if a == 0 { + return b; + } + if b == 0 { + return a; + } + + let shift = (a | b).trailing_zeros(); + a >>= a.trailing_zeros(); + loop { + b >>= b.trailing_zeros(); + if a > b { + swap(&mut a, &mut b); + } + b -= a; + if b == 0 { + return a << shift; + } + } +} + +/// Computes gcd of two signed numbers (integers or decimals), +/// checking for output integer overflow +pub(crate) fn gcd_signed(x: T, y: T) -> Result +where + T: ArrowNativeTypeOp + RemAssign + Signed + CheckedNeg, +{ + // Make absolute values, keeping type + let a = if x.is_positive() { + x + } else { + x.checked_neg() + .ok_or_else(|| ArrowError::ComputeError("Signed integer overflow".into()))? + }; + let b = if y.is_positive() { + y + } else { + y.checked_neg() + .ok_or_else(|| ArrowError::ComputeError("Signed integer overflow".into()))? + }; + // Call with signed numbers + gcd_helper(a, b) +} + +/// Computes gcd of two signed integers +pub(crate) fn gcd_signed_int(x: i64, y: i64) -> Result { + let a = x.unsigned_abs(); + let b = y.unsigned_abs(); + + // Call with unsigned numbers + let r = unsigned_gcd(a, b); + // gcd(i64::MIN, i64::MIN) = u64::MIN.unsigned_abs() cannot fit into i64 + r.try_into().map_err(|_| { + ArrowError::ComputeError(format!("Signed integer overflow in GCD({x}, {y})")) + }) +} + +/// Computes lcm of two signed numbers (integers or decimals) +pub(crate) fn lcm_signed(x: T, y: T) -> Result +where + T: ArrowNativeTypeOp + RemAssign + Signed + CheckedNeg + CheckedMul + Display, +{ + if x == T::ZERO || y == T::ZERO { + return Ok(T::ZERO); + } + + // Make absolute values, keeping type + let a = if x.is_positive() { + x + } else { + x.checked_neg() + .ok_or_else(|| ArrowError::ComputeError("Signed integer overflow".into()))? + }; + let b = if y.is_positive() { + y + } else { + y.checked_neg() + .ok_or_else(|| ArrowError::ComputeError("Signed integer overflow".into()))? + }; + // Call with signed numbers + let gcd = gcd_helper(a, b)?; + // gcd is not zero since both a and b are not zero, so the division is safe. + (a / gcd).checked_mul(&b).ok_or_else(|| { + ArrowError::ComputeError(format!("Signed integer overflow in LCM({x}, {y})")) + }) +} + +/// Computes lcm of two signed integers, +/// checking for output integer overflow +pub(crate) fn lcm_signed_int(x: i64, y: i64) -> Result { + if x == 0 || y == 0 { + return Ok(0); + } + + let a = x.unsigned_abs(); + let b = y.unsigned_abs(); + + let gcd = gcd_helper::(a, b)?; + // gcd is not zero since both a and b are not zero, so the division is safe. + (a / gcd) + .checked_mul(b) + .and_then(|v| i64::try_from(v).ok()) + .ok_or_else(|| { + ArrowError::ComputeError(format!("Signed integer overflow in LCM({x}, {y})")) + }) +} + +#[cfg(test)] +mod tests { + use super::*; + use arrow_buffer::i256; + + const GCD_COMMON_TEST_CASES: [(i64, i64, i64); 18] = [ + // Basic cases + (48, 18, 6), + (54, 24, 6), + (100, 50, 50), + (17, 19, 1), + (21, 14, 7), + // Edge cases with 0 + (0, 0, 0), + (0, 5, 5), + (10, 0, 10), + // Same numbers + (7, 7, 7), + (100, 100, 100), + // One is 1 + (1, 1, 1), + (1, 100, 1), + (999, 1, 1), + // Large numbers + (1000000, 500000, 500000), + (123456, 789012, 12), + (999999, 111111, 111111), + // Powers of 2 + (64, 128, 64), + (1024, 2048, 1024), + ]; + + const LCM_COMMON_TEST_CASES: [(i64, i64, i64); 18] = [ + // Basic cases + (48, 18, 144), + (54, 24, 216), + (100, 50, 100), + (17, 19, 323), + (21, 14, 42), + // Edge cases with 0 + (0, 0, 0), + (0, 5, 0), + (10, 0, 0), + // Same numbers + (7, 7, 7), + (100, 100, 100), + // One is 1 + (1, 1, 1), + (1, 100, 100), + (999, 1, 999), + // Large numbers + (1_000_000, 500_000, 1_000_000), + (123_456, 789_012, 8_117_355_456), + (999_999, 111_111, 999_999), + // Powers of 2 + (64, 128, 128), + (1024, 2048, 2048), + ]; + + #[test] + fn test_gcd_i64() { + let test_cases: Vec<(i64, i64, i64)> = [ + GCD_COMMON_TEST_CASES.into(), + vec![ + // Max value cases + (1, i64::MAX, 1), + (i64::MAX, 1, 1), + (i64::MAX, i64::MAX, i64::MAX), + ], + ] + .concat(); + + // Success cases + for (a, b, expected) in test_cases { + let actual_euclidean = gcd_signed(a, b).expect("should succeed"); + assert_eq!( + actual_euclidean, expected, + "gcd_signed({a}, {b}) expected {expected}, actual {actual_euclidean}" + ); + let actual_binary: i64 = + unsigned_gcd(a.try_into().unwrap(), b.try_into().unwrap()) + .try_into() + .expect("overflow"); + assert_eq!( + actual_binary, expected, + "unsigned_gcd({a}, {b}) expected {expected}, actual {actual_binary}" + ); + } + } + + #[test] + fn test_gcd_decimal() { + let test_cases: Vec<(i256, i256, i256)> = [ + GCD_COMMON_TEST_CASES + .iter() + .map(|&(a, b, c)| (i256::from(a), i256::from(b), i256::from(c))) + .collect(), + vec![ + (i256::from(1), i256::MAX, i256::from(1)), + (i256::MAX, i256::from(1), i256::from(1)), + (i256::MAX, i256::MAX, i256::MAX), + ], + ] + .concat(); + + // Success cases + for (a, b, expected) in test_cases { + let actual = gcd_signed(a, b).expect("should succeed"); + assert_eq!( + actual, expected, + "euclid_gcd({a}, {b}) expected {expected}, actual {actual}" + ); + } + } + + #[test] + fn test_lcm_i64() { + let test_cases: Vec<(i64, i64, i64)> = [ + LCM_COMMON_TEST_CASES.into(), + vec![ + // Negative inputs - LCM is always non-negative + (-6, 4, 12), + (-4, -6, 12), + // Max value cases + (1, i64::MAX, i64::MAX), + (i64::MAX, 1, i64::MAX), + (i64::MAX, i64::MAX, i64::MAX), + ], + ] + .concat(); + + for (a, b, expected) in test_cases { + let actual = lcm_signed_int(a, b).expect("should succeed"); + assert_eq!( + actual, expected, + "lcm_signed_int({a}, {b}) expected {expected}, actual {actual}" + ); + } + } + + #[test] + fn test_lcm_decimal() { + let test_cases: Vec<(i256, i256, i256)> = [ + LCM_COMMON_TEST_CASES + .iter() + .map(|&(a, b, c)| (i256::from(a), i256::from(b), i256::from(c))) + .collect(), + vec![ + // Negative inputs - LCM is always non-negative + (i256::from(-6_i64), i256::from(4_i64), i256::from(12_i64)), + (i256::from(-4_i64), i256::from(-6_i64), i256::from(12_i64)), + // Max value cases + (i256::from(1_i64), i256::MAX, i256::MAX), + (i256::MAX, i256::from(1_i64), i256::MAX), + (i256::MAX, i256::MAX, i256::MAX), + ], + ] + .concat(); + + for (a, b, expected) in test_cases { + let actual = lcm_signed(a, b).expect("should succeed"); + assert_eq!( + actual, expected, + "lcm_signed({a}, {b}) expected {expected}, actual {actual}" + ); + } + } +} diff --git a/datafusion/functions/src/math/gcd.rs b/datafusion/functions/src/math/gcd.rs index 8b92c454d9b4c..aeddc3f27c409 100644 --- a/datafusion/functions/src/math/gcd.rs +++ b/datafusion/functions/src/math/gcd.rs @@ -17,16 +17,22 @@ use arrow::array::{ArrayRef, AsArray, PrimitiveArray}; use arrow::compute::try_binary; -use arrow::datatypes::{DataType, Int64Type}; -use arrow::error::ArrowError; -use std::mem::swap; +use arrow::datatypes::{ + DataType, Decimal32Type, Decimal64Type, Decimal128Type, Decimal256Type, Int64Type, +}; use std::sync::Arc; -use datafusion_common::{Result, ScalarValue, exec_err, internal_datafusion_err}; +use crate::math::common::{gcd_signed, gcd_signed_int, unsigned_gcd}; +use crate::utils::calculate_binary_decimal_math_cast; +use datafusion_common::utils::take_function_args; +use datafusion_common::{ + Result, ScalarValue, exec_err, internal_datafusion_err, plan_err, +}; use datafusion_expr::{ ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_expr_common::type_coercion::binary::decimal_coercion; use datafusion_macros::user_doc; #[user_doc( @@ -58,11 +64,7 @@ impl Default for GcdFunc { impl GcdFunc { pub fn new() -> Self { Self { - signature: Signature::uniform( - 2, - vec![DataType::Int64], - Volatility::Immutable, - ), + signature: Signature::user_defined(Volatility::Immutable), } } } @@ -76,37 +78,123 @@ impl ScalarUDFImpl for GcdFunc { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::Int64) + fn return_type(&self, arg_types: &[DataType]) -> Result { + Ok(arg_types[0].clone()) + } + + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + let [arg1, arg2] = take_function_args(self.name(), arg_types)?; + + let coerced_type = match (arg1, arg2) { + (DataType::Null, _) | (_, DataType::Null) => Ok(DataType::Int64), + (lhs, rhs) if lhs.is_integer() && rhs.is_integer() => Ok(DataType::Int64), + (lhs, rhs) if lhs.is_decimal() || rhs.is_decimal() => { + decimal_coercion(lhs, rhs).map(Ok).unwrap_or_else(|| { + plan_err!( + "Unsupported argument types {lhs:?} and {rhs:?} for function {}", + self.name() + ) + }) + } + (lhs, rhs) => { + plan_err!( + "Unsupported argument types {lhs:?} and {rhs:?} for function {}", + self.name() + ) + } + }?; + Ok(vec![coerced_type.clone(), coerced_type]) } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + let number_rows = args.number_rows; let args: [ColumnarValue; 2] = args.args.try_into().map_err(|_| { internal_datafusion_err!("Expected 2 arguments for function gcd") })?; - match args { - [ColumnarValue::Array(a), ColumnarValue::Array(b)] => { - compute_gcd_for_arrays(&a, &b) + if args[0].data_type() == DataType::Int64 { + // Optimized path for both integers + match args { + [ColumnarValue::Array(a), ColumnarValue::Array(b)] => { + compute_gcd_for_arrays(&a, &b) + } + [ + ColumnarValue::Scalar(ScalarValue::Int64(a)), + ColumnarValue::Scalar(ScalarValue::Int64(b)), + ] => match (a, b) { + (Some(a), Some(b)) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( + Some(gcd_signed_int(a, b)?), + ))), + _ => Ok(ColumnarValue::Scalar(ScalarValue::Int64(None))), + }, + [ + ColumnarValue::Array(a), + ColumnarValue::Scalar(ScalarValue::Int64(b)), + ] => compute_gcd_with_scalar(&a, b), + [ + ColumnarValue::Scalar(ScalarValue::Int64(a)), + ColumnarValue::Array(b), + ] => compute_gcd_with_scalar(&b, a), + _ => exec_err!("Unsupported argument types for function gcd"), } - [ - ColumnarValue::Scalar(ScalarValue::Int64(a)), - ColumnarValue::Scalar(ScalarValue::Int64(b)), - ] => match (a, b) { - (Some(a), Some(b)) => Ok(ColumnarValue::Scalar(ScalarValue::Int64( - Some(compute_gcd(a, b)?), - ))), - _ => Ok(ColumnarValue::Scalar(ScalarValue::Int64(None))), - }, - [ - ColumnarValue::Array(a), - ColumnarValue::Scalar(ScalarValue::Int64(b)), - ] => compute_gcd_with_scalar(&a, b), - [ - ColumnarValue::Scalar(ScalarValue::Int64(a)), - ColumnarValue::Array(b), - ] => compute_gcd_with_scalar(&b, a), - _ => exec_err!("Unsupported argument types for function gcd"), + } else { + // Decimal path: convert left to array and use generic helper + let left = args[0].to_array(number_rows)?; + let right = &args[1]; + + let arr: ArrayRef = match (left.data_type(), right.data_type()) { + ( + lhs @ DataType::Decimal32(precision, scale), + rhs @ DataType::Decimal32(_, _), + ) if *lhs == rhs => calculate_binary_decimal_math_cast::< + Decimal32Type, + Decimal32Type, + Decimal32Type, + _, + >( + &left, right, gcd_signed, *precision, *scale, lhs + )?, + ( + lhs @ DataType::Decimal64(precision, scale), + rhs @ DataType::Decimal64(_, _), + ) if *lhs == rhs => calculate_binary_decimal_math_cast::< + Decimal64Type, + Decimal64Type, + Decimal64Type, + _, + >( + &left, right, gcd_signed, *precision, *scale, lhs + )?, + ( + lhs @ DataType::Decimal128(precision, scale), + rhs @ DataType::Decimal128(_, _), + ) if *lhs == rhs => calculate_binary_decimal_math_cast::< + Decimal128Type, + Decimal128Type, + Decimal128Type, + _, + >( + &left, right, gcd_signed, *precision, *scale, lhs + )?, + ( + lhs @ DataType::Decimal256(precision, scale), + rhs @ DataType::Decimal256(_, _), + ) if *lhs == rhs => calculate_binary_decimal_math_cast::< + Decimal256Type, + Decimal256Type, + Decimal256Type, + _, + >( + &left, right, gcd_signed, *precision, *scale, lhs + )?, + (lhs, rhs) => { + exec_err!( + "Unsupported data types {lhs:?} and {rhs:?} for function {}", + self.name() + ) + }?, + }; + Ok(ColumnarValue::Array(arr)) } } @@ -118,7 +206,7 @@ impl ScalarUDFImpl for GcdFunc { fn compute_gcd_for_arrays(a: &ArrayRef, b: &ArrayRef) -> Result { let a = a.as_primitive::(); let b = b.as_primitive::(); - try_binary(a, b, compute_gcd) + try_binary(a, b, gcd_signed_int) .map(|arr: PrimitiveArray| { ColumnarValue::Array(Arc::new(arr) as ArrayRef) }) @@ -141,44 +229,37 @@ fn compute_gcd_with_scalar(arr: &ArrayRef, scalar: Option) -> Result { let result: PrimitiveArray = - prim.try_unary(|val| compute_gcd(val, scalar_value))?; + prim.try_unary(|val| gcd_signed_int(val, scalar_value))?; Ok(ColumnarValue::Array(Arc::new(result) as ArrayRef)) } None => Ok(ColumnarValue::Scalar(ScalarValue::Int64(None))), } } -/// Computes gcd of two unsigned integers using Binary GCD algorithm. -pub(super) fn unsigned_gcd(mut a: u64, mut b: u64) -> u64 { - if a == 0 { - return b; - } - if b == 0 { - return a; - } +#[cfg(test)] +mod tests { + use super::*; - let shift = (a | b).trailing_zeros(); - a >>= a.trailing_zeros(); - loop { - b >>= b.trailing_zeros(); - if a > b { - swap(&mut a, &mut b); - } - b -= a; - if b == 0 { - return a << shift; - } - } -} + #[test] + fn test_coercion() { + let mut coerced = GcdFunc::new() + .coerce_types(&[DataType::Int64, DataType::Int32]) + .expect("coercion should succeed"); + assert_eq!(coerced, vec![DataType::Int64, DataType::Int64]); -/// Computes greatest common divisor using Binary GCD algorithm. -pub fn compute_gcd(x: i64, y: i64) -> Result { - let a = x.unsigned_abs(); - let b = y.unsigned_abs(); - let r = unsigned_gcd(a, b); - // The result can be up to 2^63 (e.g. gcd(i64::MIN, 0) or - // gcd(i64::MIN, i64::MIN)), which does not fit into i64. - r.try_into().map_err(|_| { - ArrowError::ComputeError(format!("Signed integer overflow in GCD({x}, {y})")) - }) + coerced = GcdFunc::new() + .coerce_types(&[DataType::Decimal128(10, 2), DataType::Int32]) + .expect("coercion should succeed"); + + assert_eq!( + coerced, + vec![DataType::Decimal128(12, 2), DataType::Decimal128(12, 2)] + ); + + coerced = GcdFunc::new() + .coerce_types(&[DataType::Decimal128(10, 2), DataType::Null]) + .expect("coercion should succeed"); + + assert_eq!(coerced, vec![DataType::Int64, DataType::Int64]); + } } diff --git a/datafusion/functions/src/math/lcm.rs b/datafusion/functions/src/math/lcm.rs index 9398e9f8d6e00..245dba0ba3938 100644 --- a/datafusion/functions/src/math/lcm.rs +++ b/datafusion/functions/src/math/lcm.rs @@ -15,25 +15,22 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - -use arrow::array::{ArrayRef, AsArray, PrimitiveArray}; -use arrow::compute::try_binary; -use arrow::datatypes::DataType; -use arrow::datatypes::DataType::Int64; -use arrow::datatypes::Int64Type; +use arrow::array::ArrayRef; +use arrow::datatypes::{ + DataType, Decimal32Type, Decimal64Type, Decimal128Type, Decimal256Type, Int64Type, +}; -use arrow::error::ArrowError; -use datafusion_common::{Result, exec_err}; +use crate::math::common::{lcm_signed, lcm_signed_int}; +use crate::utils::{calculate_binary_decimal_math_cast, calculate_binary_math}; +use datafusion_common::utils::take_function_args; +use datafusion_common::{Result, exec_err, plan_err}; use datafusion_expr::{ ColumnarValue, Documentation, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_expr_common::type_coercion::binary::decimal_coercion; use datafusion_macros::user_doc; -use super::gcd::unsigned_gcd; -use crate::utils::make_scalar_function; - #[user_doc( doc_section(label = "Math Functions"), description = "Returns the least common multiple of `expression_x` and `expression_y`. Returns 0 if either input is zero.", @@ -62,9 +59,8 @@ impl Default for LcmFunc { impl LcmFunc { pub fn new() -> Self { - use DataType::*; Self { - signature: Signature::uniform(2, vec![Int64], Volatility::Immutable), + signature: Signature::user_defined(Volatility::Immutable), } } } @@ -78,49 +74,100 @@ impl ScalarUDFImpl for LcmFunc { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(Int64) + fn return_type(&self, arg_types: &[DataType]) -> Result { + Ok(arg_types[0].clone()) + } + + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + let [arg1, arg2] = take_function_args(self.name(), arg_types)?; + + let coerced_type = match (arg1, arg2) { + (DataType::Null, _) | (_, DataType::Null) => Ok(DataType::Int64), + (lhs, rhs) if lhs.is_integer() && rhs.is_integer() => Ok(DataType::Int64), + (lhs, rhs) if lhs.is_decimal() || rhs.is_decimal() => { + decimal_coercion(lhs, rhs).map(Ok).unwrap_or_else(|| { + plan_err!( + "Unsupported argument types {lhs:?} and {rhs:?} for function {}", + self.name() + ) + }) + } + (lhs, rhs) => { + plan_err!( + "Unsupported argument types {lhs:?} and {rhs:?} for function {}", + self.name() + ) + } + }?; + Ok(vec![coerced_type.clone(), coerced_type]) } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { - make_scalar_function(lcm, vec![])(&args.args) + let left = &args.args[0].to_array(args.number_rows)?; + let right = &args.args[1]; + + let arr: ArrayRef = match (left.data_type(), right.data_type()) { + (DataType::Int64, _) => calculate_binary_math::< + Int64Type, + Int64Type, + Int64Type, + _, + >(&left, right, lcm_signed_int)?, + ( + lhs @ DataType::Decimal32(precision, scale), + rhs @ DataType::Decimal32(_, _), + ) if *lhs == rhs => { + calculate_binary_decimal_math_cast::< + Decimal32Type, + Decimal32Type, + Decimal32Type, + _, + >(&left, right, lcm_signed, *precision, *scale, lhs)? + } + ( + lhs @ DataType::Decimal64(precision, scale), + rhs @ DataType::Decimal64(_, _), + ) if *lhs == rhs => { + calculate_binary_decimal_math_cast::< + Decimal64Type, + Decimal64Type, + Decimal64Type, + _, + >(&left, right, lcm_signed, *precision, *scale, lhs)? + } + ( + lhs @ DataType::Decimal128(precision, scale), + rhs @ DataType::Decimal128(_, _), + ) if *lhs == rhs => { + calculate_binary_decimal_math_cast::< + Decimal128Type, + Decimal128Type, + Decimal128Type, + _, + >(&left, right, lcm_signed, *precision, *scale, lhs)? + } + ( + lhs @ DataType::Decimal256(precision, scale), + rhs @ DataType::Decimal256(_, _), + ) if *lhs == rhs => { + calculate_binary_decimal_math_cast::< + Decimal256Type, + Decimal256Type, + Decimal256Type, + _, + >(&left, right, lcm_signed, *precision, *scale, lhs)? + } + (lhs, rhs) => { + return exec_err!( + "Unsupported data types {lhs:?} and {rhs:?} for function {}", + self.name() + ); + } + }; + Ok(ColumnarValue::Array(arr)) } fn documentation(&self) -> Option<&Documentation> { self.doc() } } - -/// Lcm SQL function -fn lcm(args: &[ArrayRef]) -> Result { - let compute_lcm = |x: i64, y: i64| -> Result { - if x == 0 || y == 0 { - return Ok(0); - } - - // lcm(x, y) = |x| * |y| / gcd(|x|, |y|) - let a = x.unsigned_abs(); - let b = y.unsigned_abs(); - let gcd = unsigned_gcd(a, b); - // gcd is not zero since both a and b are not zero, so the division is safe. - (a / gcd) - .checked_mul(b) - .and_then(|v| i64::try_from(v).ok()) - .ok_or_else(|| { - ArrowError::ComputeError(format!( - "Signed integer overflow in LCM({x}, {y})" - )) - }) - }; - - match args[0].data_type() { - Int64 => { - let arg1 = args[0].as_primitive::(); - let arg2 = args[1].as_primitive::(); - - let result: PrimitiveArray = try_binary(arg1, arg2, compute_lcm)?; - Ok(Arc::new(result) as ArrayRef) - } - other => exec_err!("Unsupported data type {other:?} for function lcm"), - } -} diff --git a/datafusion/functions/src/math/mod.rs b/datafusion/functions/src/math/mod.rs index 1754ccb43488a..a5d45380ecf0a 100644 --- a/datafusion/functions/src/math/mod.rs +++ b/datafusion/functions/src/math/mod.rs @@ -25,6 +25,7 @@ use std::sync::Arc; pub mod abs; pub mod bounds; pub mod ceil; +mod common; pub mod cot; mod decimal; pub mod factorial; diff --git a/datafusion/functions/src/math/round.rs b/datafusion/functions/src/math/round.rs index 78016c0f52f71..aacc8820a8cb6 100644 --- a/datafusion/functions/src/math/round.rs +++ b/datafusion/functions/src/math/round.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::{calculate_binary_decimal_math, calculate_binary_math}; +use crate::utils::{calculate_binary_decimal_math_cast, calculate_binary_math}; use arrow::array::ArrayRef; use arrow::datatypes::DataType::{ @@ -486,7 +486,7 @@ fn round_columnar( } (Decimal32(input_precision, scale), Decimal32(precision, new_scale)) => { // reduce scale to reclaim integer precision - let result = calculate_binary_decimal_math::< + let result = calculate_binary_decimal_math_cast::< Decimal32Type, Int32Type, Decimal32Type, @@ -518,11 +518,12 @@ fn round_columnar( }, *precision, *new_scale, + &DataType::Int32, )?; result as _ } (Decimal64(input_precision, scale), Decimal64(precision, new_scale)) => { - let result = calculate_binary_decimal_math::< + let result = calculate_binary_decimal_math_cast::< Decimal64Type, Int32Type, Decimal64Type, @@ -551,11 +552,12 @@ fn round_columnar( }, *precision, *new_scale, + &DataType::Int32, )?; result as _ } (Decimal128(input_precision, scale), Decimal128(precision, new_scale)) => { - let result = calculate_binary_decimal_math::< + let result = calculate_binary_decimal_math_cast::< Decimal128Type, Int32Type, Decimal128Type, @@ -584,11 +586,12 @@ fn round_columnar( }, *precision, *new_scale, + &DataType::Int32, )?; result as _ } (Decimal256(input_precision, scale), Decimal256(precision, new_scale)) => { - let result = calculate_binary_decimal_math::< + let result = calculate_binary_decimal_math_cast::< Decimal256Type, Int32Type, Decimal256Type, @@ -617,6 +620,7 @@ fn round_columnar( }, *precision, *new_scale, + &DataType::Int32, )?; result as _ } diff --git a/datafusion/functions/src/utils.rs b/datafusion/functions/src/utils.rs index b9bde1454994c..39683e9a6afa2 100644 --- a/datafusion/functions/src/utils.rs +++ b/datafusion/functions/src/utils.rs @@ -133,6 +133,72 @@ pub fn calculate_binary_math( right: &ColumnarValue, fun: F, ) -> Result>> +where + L: ArrowPrimitiveType, + R: ArrowPrimitiveType, + O: ArrowPrimitiveType, + F: Fn(L::Native, R::Native) -> Result, + R::Native: TryFrom, +{ + calculate_binary_math_cast::(left, right, fun, &R::DATA_TYPE) +} + +/// Computes a binary math function for input arrays using a specified function +/// and applies rescaling to given precision and scale. +/// Generic types: +/// - `L`: Left array decimal type +/// - `R`: Right array primitive type +/// - `O`: Output array decimal type +/// - `F`: Functor computing `fun(l: L, r: R) -> Result` +#[deprecated( + since = "55.0.0", + note = "Use `calculate_binary_decimal_math_cast` instead" +)] +pub fn calculate_binary_decimal_math( + left: &dyn Array, + right: &ColumnarValue, + fun: F, + precision: u8, + scale: i8, +) -> Result>> +where + L: DecimalType, + R: ArrowPrimitiveType, + O: DecimalType, + F: Fn(L::Native, R::Native) -> Result, + R::Native: TryFrom, +{ + calculate_binary_decimal_math_cast::( + left, + right, + fun, + precision, + scale, + &R::DATA_TYPE, + ) +} + +/// Computes a binary math function for input arrays using a specified function. +/// +/// It casts the right operand to `cast_target` instead of the default `R::DATA_TYPE` to preserve +/// the right operand scale. +/// +/// # Type Parameters +/// - `L`: Left array primitive type +/// - `R`: Right array primitive type +/// - `O`: Output array primitive type +/// - `F`: Functor computing `fun(l: L, r: R) -> Result` +/// # Arguments +/// - `left`: Left input array +/// - `right`: Right input array or scalar value +/// - `fun`: Function of type `F` +/// - `cast_target`: Data type to cast right operand to before applying function +fn calculate_binary_math_cast( + left: &dyn Array, + right: &ColumnarValue, + fun: F, + cast_target: &DataType, +) -> Result>> where L: ArrowPrimitiveType, R: ArrowPrimitiveType, @@ -141,7 +207,7 @@ where R::Native: TryFrom, { let left = left.as_primitive::(); - let right = right.cast_to(&R::DATA_TYPE, None)?; + let right = right.cast_to(cast_target, None)?; let result = match right { ColumnarValue::Scalar(scalar) => { if scalar.is_null() { @@ -152,8 +218,7 @@ where let right = R::Native::try_from(scalar.clone()).map_err(|_| { DataFusionError::NotImplemented(format!( "Cannot convert scalar value {} to {}", - &scalar, - R::DATA_TYPE + &scalar, cast_target )) })?; left.try_unary::<_, O, _>(|lvalue| fun(lvalue, right))? @@ -168,18 +233,30 @@ where } /// Computes a binary math function for input arrays using a specified function -/// and apply rescaling to given precision and scale. -/// Generic types: +/// and applies rescaling to given precision and scale. +/// +/// It casts the right operand to `cast_target` instead of the default `R::DATA_TYPE` to preserve +/// the right operand scale. +/// +/// # Type Parameters /// - `L`: Left array decimal type /// - `R`: Right array primitive type /// - `O`: Output array decimal type /// - `F`: Functor computing `fun(l: L, r: R) -> Result` -pub fn calculate_binary_decimal_math( +/// # Arguments +/// - `left`: Left input array +/// - `right`: Right input array or scalar value +/// - `fun`: Function of type `F` +/// - `precision`: Precision to apply to output decimal array +/// - `scale`: Scale to apply to output decimal array +/// - `cast_target`: Data type to cast right operand to before applying function +pub fn calculate_binary_decimal_math_cast( left: &dyn Array, right: &ColumnarValue, fun: F, precision: u8, scale: i8, + cast_target: &DataType, ) -> Result>> where L: DecimalType, @@ -188,7 +265,8 @@ where F: Fn(L::Native, R::Native) -> Result, R::Native: TryFrom, { - let result_array = calculate_binary_math::(left, right, fun)?; + let result_array = + calculate_binary_math_cast::(left, right, fun, cast_target)?; Ok(Arc::new( result_array .as_ref() diff --git a/datafusion/optimizer/src/extract_leaf_expressions.rs b/datafusion/optimizer/src/extract_leaf_expressions.rs index 185f9d045f10f..b855f224c420b 100644 --- a/datafusion/optimizer/src/extract_leaf_expressions.rs +++ b/datafusion/optimizer/src/extract_leaf_expressions.rs @@ -21,7 +21,7 @@ //! [`ExtractLeafExpressions`] (pass 1) and [`PushDownLeafProjections`] (pass 2). use indexmap::{IndexMap, IndexSet}; -use std::collections::HashMap; +use std::collections::{BTreeSet, HashMap}; use std::sync::Arc; use datafusion_common::alias::AliasGenerator; @@ -827,12 +827,8 @@ fn split_and_push_projection( let original_schema = proj.schema.as_ref(); let mut recovery_exprs: Vec = Vec::with_capacity(proj.expr.len()); - let mut needs_recovery = false; let mut has_new_extractions = false; let mut proj_exprs_captured: usize = 0; - // Track standalone column expressions (Case B) to detect column refs - // from extracted aliases (Case A) that aren't also standalone expressions. - let mut standalone_columns: IndexSet = IndexSet::new(); for (expr, (qualifier, field)) in proj.expr.iter().zip(original_schema.iter()) { if let Expr::Alias(alias) = expr @@ -854,7 +850,6 @@ fn split_and_push_projection( } else if let Expr::Column(col) = expr { // Plain column pass-through — track it in the extractor extractors[0].columns_needed.insert(col.clone()); - standalone_columns.insert(col.clone()); recovery_exprs.push(expr.clone()); proj_exprs_captured += 1; } else { @@ -875,7 +870,6 @@ fn split_and_push_projection( original_name != &expr_name }; let recovery_expr = if needs_alias { - needs_recovery = true; transformed_expr .clone() .alias_qualified(qualifier.cloned(), original_name) @@ -883,14 +877,6 @@ fn split_and_push_projection( transformed_expr.clone() }; - // If the expression was transformed (i.e., has extracted sub-parts), - // it differs from what the pushed projection outputs → needs recovery. - // Also, any non-column, non-__datafusion_extracted expression needs recovery - // because the pushed extraction projection won't output it directly. - if transformed.transformed || !matches!(expr, Expr::Column(_)) { - needs_recovery = true; - } - recovery_exprs.push(recovery_expr); } } @@ -913,17 +899,6 @@ fn split_and_push_projection( return Ok(None); } - // If columns_needed has entries that aren't standalone projection columns - // (i.e., they came from column refs inside extracted aliases), a merge - // into an inner projection will widen the schema with those extra columns, - // requiring a recovery projection to restore the original schema. - if columns_needed - .iter() - .any(|c| !standalone_columns.contains(c)) - { - needs_recovery = true; - } - // ── Phase 2: Push down ────────────────────────────────────────────── let proj_input = Arc::clone(&proj.input); let pushed = push_extraction_pairs( @@ -959,6 +934,37 @@ fn split_and_push_projection( } }; + // The recovery projection restores the original projection's output. We need + // it whenever `base_plan` no longer exposes the same set of output column + // names, which happens two ways: + // * a column is *renamed* — a transformed expression now surfaces as its + // internal `__datafusion_extracted_*` alias instead of the original name; + // * a column is *leaked* — pushing the projection down widens `base_plan` + // with an inner extraction projection's *other* extracted aliases bubbling + // up through a Filter. A schema-caching parent like SubqueryAlias then + // keeps a stale schema (see `map_children` in `logical_plan/tree_node.rs`) + // and the later `optimize_projections` pass fails to resolve columns. + // + // Both are captured by comparing the *set of unqualified field names*. We + // compare by unqualified name rather than the full qualified schema on + // purpose: extracted aliases are globally unique, so name-only comparison is + // unambiguous for them, while it ignores the benign column reordering and the + // `SubqueryAlias` re-qualification (`sub.__datafusion_extracted_1` vs + // `__datafusion_extracted_1`) that a qualified/ordered comparison would + // spuriously treat as drift, stacking redundant recovery projections. + let base_names: BTreeSet<&str> = base_plan + .schema() + .fields() + .iter() + .map(|f| f.name().as_str()) + .collect(); + let original_names: BTreeSet<&str> = original_schema + .fields() + .iter() + .map(|f| f.name().as_str()) + .collect(); + let needs_recovery = base_names != original_names; + // Wrap with recovery projection if the output schema changed if needs_recovery { let recovery = LogicalPlan::Projection(Projection::try_new( @@ -1145,7 +1151,6 @@ fn try_push_into_inputs( // Unnest may output a column with the same name but different value/type // than its input column. Name-based routing cannot distinguish those. - // On top of that Unnest can't go through the `node.with_new_exprs(node.expressions(), new_inputs)` rebuild if matches!(node, LogicalPlan::Unnest(_)) { return Ok(None); } @@ -1618,9 +1623,10 @@ mod tests { ## After Pushdown Projection: __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")) - Filter: __datafusion_extracted_1 = Utf8("active") - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2 - TableScan: test projection=[user] + Projection: test.user, __datafusion_extracted_2 + Filter: __datafusion_extracted_1 = Utf8("active") + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2 + TableScan: test projection=[user] ## Optimized Projection: __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")) @@ -1681,13 +1687,17 @@ mod tests { TableScan: test projection=[user] ## After Pushdown + Projection: test.user, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("label")) + Projection: test.user, __datafusion_extracted_2 + Filter: __datafusion_extracted_1 > Int32(150) + Projection: leaf_udf(test.user, Utf8("value")) AS __datafusion_extracted_1, test.user, leaf_udf(test.user, Utf8("label")) AS __datafusion_extracted_2 + TableScan: test projection=[user] + + ## Optimized Projection: test.user, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("label")) Filter: __datafusion_extracted_1 > Int32(150) Projection: leaf_udf(test.user, Utf8("value")) AS __datafusion_extracted_1, test.user, leaf_udf(test.user, Utf8("label")) AS __datafusion_extracted_2 TableScan: test projection=[user] - - ## Optimized - (same as after pushdown) "#) } @@ -1892,19 +1902,15 @@ mod tests { TableScan: test projection=[id, user] ## After Pushdown - Projection: test.id, test.user - Filter: __datafusion_extracted_1 IS NOT NULL - Filter: __datafusion_extracted_2 = Utf8("active") - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2, test.id, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_1 - TableScan: test projection=[id, user] - - ## Optimized Projection: test.id, test.user Filter: __datafusion_extracted_1 IS NOT NULL Projection: test.id, test.user, __datafusion_extracted_1 Filter: __datafusion_extracted_2 = Utf8("active") Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2, test.id, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_1 TableScan: test projection=[id, user] + + ## Optimized + (same as after pushdown) "#) } @@ -2013,9 +2019,10 @@ mod tests { ## After Pushdown Projection: __datafusion_extracted_1 AS leaf_udf(test.user,Utf8("name")), COUNT(Int32(1)) Aggregate: groupBy=[[__datafusion_extracted_1]], aggr=[[COUNT(Int32(1))]] - Filter: __datafusion_extracted_2 = Utf8("active") - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_1 - TableScan: test projection=[user] + Projection: test.user, __datafusion_extracted_1 + Filter: __datafusion_extracted_2 = Utf8("active") + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_1 + TableScan: test projection=[user] ## Optimized Projection: __datafusion_extracted_1 AS leaf_udf(test.user,Utf8("name")), COUNT(Int32(1)) @@ -2092,19 +2099,15 @@ mod tests { TableScan: test projection=[a, b, c] ## After Pushdown - Projection: test.a, test.b, test.c - Filter: __datafusion_extracted_1 = Int32(2) - Filter: __datafusion_extracted_2 = Int32(1) - Projection: leaf_udf(test.a, Utf8("x")) AS __datafusion_extracted_2, test.a, test.b, test.c, leaf_udf(test.b, Utf8("y")) AS __datafusion_extracted_1 - TableScan: test projection=[a, b, c] - - ## Optimized Projection: test.a, test.b, test.c Filter: __datafusion_extracted_1 = Int32(2) Projection: test.a, test.b, test.c, __datafusion_extracted_1 Filter: __datafusion_extracted_2 = Int32(1) Projection: leaf_udf(test.a, Utf8("x")) AS __datafusion_extracted_2, test.a, test.b, test.c, leaf_udf(test.b, Utf8("y")) AS __datafusion_extracted_1 TableScan: test projection=[a, b, c] + + ## Optimized + (same as after pushdown) "#) } @@ -2314,21 +2317,15 @@ mod tests { ## After Pushdown Projection: test.id, test.user, right.id, right.user Filter: __datafusion_extracted_1 = Utf8("active") - Inner Join: __datafusion_extracted_2 = __datafusion_extracted_3 - Projection: leaf_udf(test.user, Utf8("id")) AS __datafusion_extracted_2, test.id, test.user, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1 - TableScan: test projection=[id, user] - Projection: leaf_udf(right.user, Utf8("id")) AS __datafusion_extracted_3, right.id, right.user - TableScan: right projection=[id, user] - - ## Optimized - Projection: test.id, test.user, right.id, right.user - Filter: __datafusion_extracted_1 = Utf8("active") - Projection: test.id, test.user, __datafusion_extracted_1, right.id, right.user + Projection: test.id, test.user, right.id, right.user, __datafusion_extracted_1 Inner Join: __datafusion_extracted_2 = __datafusion_extracted_3 Projection: leaf_udf(test.user, Utf8("id")) AS __datafusion_extracted_2, test.id, test.user, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1 TableScan: test projection=[id, user] Projection: leaf_udf(right.user, Utf8("id")) AS __datafusion_extracted_3, right.id, right.user TableScan: right projection=[id, user] + + ## Optimized + (same as after pushdown) "#) } @@ -2681,10 +2678,11 @@ mod tests { ## After Pushdown Projection: __datafusion_extracted_2 AS leaf_udf(sub.user,Utf8("name")) - Filter: __datafusion_extracted_1 = Utf8("active") - SubqueryAlias: sub - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2, test.user - TableScan: test projection=[user] + Projection: sub.user, __datafusion_extracted_2 + Filter: __datafusion_extracted_1 = Utf8("active") + SubqueryAlias: sub + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2, test.user + TableScan: test projection=[user] ## Optimized Projection: __datafusion_extracted_2 AS leaf_udf(sub.user,Utf8("name")) @@ -2856,9 +2854,10 @@ mod tests { ## After Pushdown Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")) - Filter: __datafusion_extracted_1 = Utf8("active") - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2 - TableScan: test projection=[id, user] + Projection: test.id, test.user, __datafusion_extracted_2 + Filter: __datafusion_extracted_1 = Utf8("active") + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2 + TableScan: test projection=[id, user] ## Optimized Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")) @@ -2893,9 +2892,10 @@ mod tests { ## After Pushdown Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("status")) - Filter: __datafusion_extracted_1 > Int32(5) - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2 - TableScan: test projection=[id, user] + Projection: test.id, test.user, __datafusion_extracted_2 + Filter: __datafusion_extracted_1 > Int32(5) + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2 + TableScan: test projection=[id, user] ## Optimized Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("status")) @@ -2947,11 +2947,12 @@ mod tests { ## After Pushdown Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")), __datafusion_extracted_3 AS leaf_udf(right.user,Utf8("status")) - Left Join: Filter: test.id = right.id AND __datafusion_extracted_1 > Int32(5) - Projection: leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2, test.id, test.user - TableScan: test projection=[id, user] - Projection: leaf_udf(right.user, Utf8("status")) AS __datafusion_extracted_1, right.id, right.user, leaf_udf(right.user, Utf8("status")) AS __datafusion_extracted_3 - TableScan: right projection=[id, user] + Projection: test.id, test.user, right.id, right.user, __datafusion_extracted_2, __datafusion_extracted_3 + Left Join: Filter: test.id = right.id AND __datafusion_extracted_1 > Int32(5) + Projection: leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2, test.id, test.user + TableScan: test projection=[id, user] + Projection: leaf_udf(right.user, Utf8("status")) AS __datafusion_extracted_1, right.id, right.user, leaf_udf(right.user, Utf8("status")) AS __datafusion_extracted_3 + TableScan: right projection=[id, user] ## Optimized Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")), __datafusion_extracted_3 AS leaf_udf(right.user,Utf8("status")) @@ -2992,9 +2993,10 @@ mod tests { ## After Pushdown Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")), __datafusion_extracted_3 AS leaf_udf(test.user,Utf8("status")) - Filter: __datafusion_extracted_1 > Int32(5) - Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_3 - TableScan: test projection=[id, user] + Projection: test.id, test.user, __datafusion_extracted_2, __datafusion_extracted_3 + Filter: __datafusion_extracted_1 > Int32(5) + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("name")) AS __datafusion_extracted_2, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_3 + TableScan: test projection=[id, user] ## Optimized Projection: test.id, __datafusion_extracted_2 AS leaf_udf(test.user,Utf8("name")), __datafusion_extracted_3 AS leaf_udf(test.user,Utf8("status")) @@ -3043,16 +3045,15 @@ mod tests { Ok(()) } - /// Regression test for the `Assertion failed: expr.is_empty(): Unnest` - /// internal error. + /// Regression test: the optimizer must not push extractions through + /// `Unnest`. /// - /// `try_push_into_inputs` rebuilds the parent node via - /// `node.with_new_exprs(node.expressions(), new_inputs)`. For `Unnest`, - /// `apply_expressions` exposes the `exec_columns` as `Expr::Column`s - /// (so `expressions()` is **non-empty**), but `with_new_exprs` for - /// `Unnest` immediately calls `assert_no_expressions(expr)?` and errors - /// out. The optimizer should treat `Unnest` as a barrier and bail - /// instead of attempting to push through it. + /// `try_push_into_inputs` routes extracted pairs to inputs by column name. + /// `Unnest` can emit an output column with the same name as its input + /// column but a different value/type (the unnested element), so name-based + /// routing cannot tell the two apart. `try_push_into_inputs` therefore + /// treats `Unnest` as a barrier and bails instead of pushing through it + /// (see the `matches!(node, LogicalPlan::Unnest(_))` guard there). #[test] fn test_no_push_through_unnest() -> Result<()> { use arrow::datatypes::{DataType, Field, Schema}; @@ -3086,4 +3087,80 @@ mod tests { Ok(()) } + + /// Regression test: a leaf expression used in **both** the filter and the + /// projection, with the **bare base column** also projected, over a + /// `SubqueryAlias` whose projection emits an **extra column the outer query + /// never consumes** (`synth`). + /// + /// This reproduces a production failure where the leaf-pushdown passes drop + /// the bare passthrough column from an intermediate schema, causing the + /// subsequent `optimize_projections` run to fail with: + /// `Schema error: No field named __datafusion_extracted_N`. + /// + /// Equivalent SQL: + /// ```sql + /// CREATE VIEW v AS SELECT user, id, id + 1 AS synth FROM test; + /// SELECT user['status'], user, id FROM v WHERE user['status'] IS NOT NULL; + /// ``` + #[test] + fn test_subquery_alias_with_unconsumed_column() -> Result<()> { + let table_scan = test_table_scan_with_struct()?; + + // This is the plan shape *after* `push_down_filter` has run: it pushes + // the `leaf_udf(...)` filter down through the `SubqueryAlias` and below + // the view's inner projection. The filter and the outer projection now + // each contain the same leaf expression but are separated by the + // `SubqueryAlias`, so they extract into two *independent* aliases + // (`__datafusion_extracted_1` from the filter, `__datafusion_extracted_2` + // from the projection) instead of deduplicating into one. + // + // The view projects an extra `synth` column the outer query never + // consumes — without it the bug does not manifest. + let inner = LogicalPlanBuilder::from(table_scan) + .filter(leaf_udf(col("user"), "status").is_not_null())? + .project(vec![ + col("user"), + col("id"), + (col("id") + lit(1u32)).alias("synth"), + ])? + .alias("v")? + .build()?; + + // Outer projection: leaf expr + the bare base column + id. + let plan = LogicalPlanBuilder::from(inner) + .project(vec![ + leaf_udf(col("v.user"), "status"), + col("v.user"), + col("v.id"), + ])? + .build()?; + + // Run the leaf-pushdown passes followed by `optimize_projections`, + // exactly as the default optimizer schedules them. `optimize_projections` + // is what prunes the unused `synth` column and validates the plan; if the + // leaf passes drop the bare `v.user` passthrough column it fails with + // `Schema error: No field named __datafusion_extracted_N`. + let ctx = OptimizerContext::new(); + let optimizer = Optimizer::with_rules(vec![ + Arc::new(ExtractLeafExpressions::new()), + Arc::new(PushDownLeafProjections::new()), + Arc::new(OptimizeProjections::new()), + ]); + let optimized = optimizer.optimize(plan, &ctx, |_, _| {})?; + + // The bare `test.user` passthrough column is preserved and the view's + // output schema (`user`, `id`, `__datafusion_extracted_2`) is restored + // by a recovery projection, so `optimize_projections` succeeds. + insta::assert_snapshot!(format!("{optimized}"), @r#" + Projection: __datafusion_extracted_2 AS leaf_udf(v.user,Utf8("status")), v.user, v.id + SubqueryAlias: v + Projection: test.user, test.id, __datafusion_extracted_2 + Filter: __datafusion_extracted_1 IS NOT NULL + Projection: leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_1, test.id, test.user, leaf_udf(test.user, Utf8("status")) AS __datafusion_extracted_2 + TableScan: test projection=[id, user] + "#); + + Ok(()) + } } diff --git a/datafusion/physical-expr-adapter/src/lib.rs b/datafusion/physical-expr-adapter/src/lib.rs index ea4db19ee110e..fa14bc8b4d150 100644 --- a/datafusion/physical-expr-adapter/src/lib.rs +++ b/datafusion/physical-expr-adapter/src/lib.rs @@ -29,5 +29,6 @@ pub mod schema_rewriter; pub use schema_rewriter::{ BatchAdapter, BatchAdapterFactory, DefaultPhysicalExprAdapter, DefaultPhysicalExprAdapterFactory, PhysicalExprAdapter, PhysicalExprAdapterFactory, - replace_columns_with_literals, + expr_references_scalar_udf, replace_columns_with_literals, + rewrite_file_row_index_expr, rewrite_file_row_index_projection, }; diff --git a/datafusion/physical-expr-adapter/src/schema_rewriter.rs b/datafusion/physical-expr-adapter/src/schema_rewriter.rs index 56502ab8731a7..f287caf32ecda 100644 --- a/datafusion/physical-expr-adapter/src/schema_rewriter.rs +++ b/datafusion/physical-expr-adapter/src/schema_rewriter.rs @@ -25,16 +25,19 @@ use std::hash::Hash; use std::sync::Arc; use arrow::array::RecordBatch; -use arrow::datatypes::{DataType, FieldRef, SchemaRef}; +use arrow::datatypes::{DataType, Field, FieldRef, SchemaRef}; use datafusion_common::{ DataFusionError, Result, ScalarValue, exec_err, metadata::FieldMetadata, nested_struct::validate_data_type_compatibility, - tree_node::{Transformed, TransformedResult, TreeNode}, + tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion}, +}; +use datafusion_expr::ScalarUDFImpl; +use datafusion_functions::core::{ + file_row_index::FileRowIndexFunc, getfield::GetFieldFunc, }; -use datafusion_functions::core::getfield::GetFieldFunc; use datafusion_physical_expr::PhysicalExprSimplifier; -use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; +use datafusion_physical_expr::projection::{ProjectionExpr, ProjectionExprs, Projector}; use datafusion_physical_expr::{ ScalarFunctionExpr, expressions::{self, CastExpr, Column}, @@ -81,6 +84,114 @@ where .data() } +/// Return true if `expr` references scalar UDF `T`. +/// +/// This matches the concrete [`ScalarUDFImpl`] type rather than the function +/// name, so unrelated UDFs with the same name are not treated as matches. +pub fn expr_references_scalar_udf( + expr: &Arc, +) -> bool { + let mut found = false; + + expr.apply(|node| { + if ScalarFunctionExpr::try_downcast_func::(node.as_ref()).is_some() { + found = true; + return Ok(TreeNodeRecursion::Stop); + } + Ok(TreeNodeRecursion::Continue) + }) + .expect("Infallible traversal of PhysicalExpr tree failed"); + + found +} + +/// Rewrite occurrences of scalar UDF `T` in `expr` using `replacement`. +/// +/// The rewrite matches the concrete [`ScalarUDFImpl`] type rather than the +/// function name. `replacement` is called with each matching +/// [`ScalarFunctionExpr`] after its children have been rewritten. +fn rewrite_scalar_udf( + expr: Arc, + mut replacement: F, +) -> Result> +where + T: ScalarUDFImpl, + F: FnMut(&ScalarFunctionExpr) -> Result>, +{ + expr.transform_up(|node| { + if let Some(scalar_fn) = ScalarFunctionExpr::try_downcast_func::(node.as_ref()) + { + Ok(Transformed::yes(replacement(scalar_fn)?)) + } else { + Ok(Transformed::no(node)) + } + }) + .map(|transformed| transformed.data) +} + +/// Rewrite `file_row_index()` in `expr` to read from a source-provided +/// row-index column. +/// +/// `row_index_idx` is the index of `row_index_name` in the schema that the +/// rewritten expression will be evaluated against. The rewrite uses ordinary +/// physical expressions: a [`Column`] that reads the source row-index values +/// wrapped in a [`CastExpr`] that exposes the public `file_row_index: Int64` +/// return field without source-specific extension metadata. +pub fn rewrite_file_row_index_expr( + expr: Arc, + row_index_name: &str, + row_index_idx: usize, +) -> Result> { + rewrite_scalar_udf::(expr, |_| { + let source = Arc::new(Column::new(row_index_name, row_index_idx)); + let target_field = Arc::new(Field::new("file_row_index", DataType::Int64, true)); + Ok(Arc::new(CastExpr::new_with_target_field( + source, + target_field, + None, + ))) + }) +} + +/// Rewrite `file_row_index()` in a pushed projection to read from a +/// source-provided row-index column. +/// +/// +/// For example if `row_index_column` is `__datafusion_row_idx` this function rewrites all +/// instances of `file_row_index()` to `__datafusion_row_index` column references. +/// +/// `base_projection` is the current projection already pushed into a source. +/// The row-index source column is appended to that base projection if it is not +/// already present. `projection` is rewritten to read from the projected +/// row-index column and then merged on top of the extended base projection. +pub fn rewrite_file_row_index_projection( + base_projection: &ProjectionExprs, + projection: &ProjectionExprs, + row_index_col: &Column, +) -> Result { + let mut base_exprs = base_projection.as_ref().to_vec(); + let row_index_projection_idx = + base_projection.projected_column_position(row_index_col); + + // If the column doesn't exist in the projection yet + if row_index_projection_idx.is_none() { + base_exprs.push(ProjectionExpr { + expr: Arc::new(row_index_col.clone()), + alias: row_index_col.name().to_owned(), + }); + } + + let rewritten_projection = projection.clone().try_map_exprs(|expr| { + rewrite_file_row_index_expr( + expr, + row_index_col.name(), + row_index_projection_idx.unwrap_or(base_exprs.len() - 1), + ) + })?; + + ProjectionExprs::new(base_exprs).try_merge(&rewritten_projection) +} + /// Trait for adapting [`PhysicalExpr`] expressions to match a target schema. /// /// This is used in file scans to rewrite expressions so that they can be @@ -631,8 +742,8 @@ mod tests { RecordBatchOptions, StringArray, StringViewArray, StructArray, }; use arrow::datatypes::{Field, Fields, Schema}; - use datafusion_common::{assert_contains, record_batch}; - use datafusion_expr::Operator; + use datafusion_common::{assert_contains, config::ConfigOptions, record_batch}; + use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{Column, Literal, col}; fn assert_cast_expr(expr: &Arc) -> &CastExpr { @@ -648,6 +759,88 @@ mod tests { assert_eq!(inner_col.index(), index); } + fn file_row_index_expr() -> Arc { + Arc::new(ScalarFunctionExpr::new( + "file_row_index", + Arc::new(ScalarUDF::from(FileRowIndexFunc::new())), + vec![], + Arc::new(Field::new("file_row_index", DataType::Int64, true)), + Arc::new(ConfigOptions::default()), + )) + } + + #[test] + fn test_rewrite_scalar_udf_replaces_nested_typed_udf() -> Result<()> { + let expr = Arc::new(expressions::BinaryExpr::new( + file_row_index_expr(), + Operator::Plus, + expressions::lit(ScalarValue::Int64(Some(1))), + )) as Arc; + + let rewritten = rewrite_scalar_udf::(expr, |_| { + Ok(expressions::lit(ScalarValue::Int64(Some(7)))) + })?; + + let binary = rewritten + .downcast_ref::() + .expect("rewritten expression should remain binary"); + assert_eq!(binary.op(), &Operator::Plus); + + let left = binary + .left() + .downcast_ref::() + .expect("left side should be rewritten to a literal"); + assert_eq!(left.value(), &ScalarValue::Int64(Some(7))); + + let right = binary + .right() + .downcast_ref::() + .expect("right side should remain the original literal"); + assert_eq!(right.value(), &ScalarValue::Int64(Some(1))); + Ok(()) + } + + #[test] + fn test_rewrite_file_row_index_expr_to_source_column() -> Result<()> { + let expr = rewrite_file_row_index_expr( + file_row_index_expr(), + "__datafusion_file_row_index", + 2, + )?; + + let cast_expr = expr + .downcast_ref::() + .expect("file row index expression should be a cast"); + assert_eq!(cast_expr.cast_type(), &DataType::Int64); + let target_field = cast_expr.target_field(); + assert_eq!(target_field.name(), "file_row_index"); + assert_eq!(target_field.data_type(), &DataType::Int64); + assert!(target_field.is_nullable()); + assert!(target_field.metadata().is_empty()); + + let source = cast_expr + .expr() + .downcast_ref::() + .expect("source column"); + assert_eq!(source.name(), "__datafusion_file_row_index"); + assert_eq!(source.index(), 2); + + let input_schema = Schema::new(vec![ + Field::new("value", DataType::Int64, true), + Field::new("__datafusion_file_row_index", DataType::Int64, false) + .with_metadata(HashMap::from([( + "source".to_string(), + "virtual".to_string(), + )])), + ]); + let return_field = expr.return_field(&input_schema)?; + assert_eq!(return_field.name(), "file_row_index"); + assert_eq!(return_field.data_type(), &DataType::Int64); + assert!(return_field.is_nullable()); + assert!(return_field.metadata().is_empty()); + Ok(()) + } + fn stale_index_cast_schemas() -> (SchemaRef, SchemaRef) { let physical_schema = Arc::new(Schema::new(vec![ Field::new("b", DataType::Binary, true), diff --git a/datafusion/physical-expr/src/projection.rs b/datafusion/physical-expr/src/projection.rs index cee95685e8440..1f6a6eb08fb78 100644 --- a/datafusion/physical-expr/src/projection.rs +++ b/datafusion/physical-expr/src/projection.rs @@ -661,7 +661,7 @@ impl ProjectionExprs { for proj_expr in self.exprs.iter() { let expr = &proj_expr.expr; let col_stats = if let Some(col) = expr.downcast_ref::() { - stats.column_statistics[col.index()].clone() + column_statistics_at(&stats.column_statistics, col.index()) } else if let Some(literal) = expr.downcast_ref::() { // Handle literal expressions (constants) by calculating proper statistics let data_type = expr.data_type(output_schema)?; @@ -725,6 +725,60 @@ impl ProjectionExprs { stats.column_statistics = column_statistics; Ok(stats) } + + /// Returns the output position of `column` if this projection contains it. + /// + /// This only matches projection expressions that are exactly [`Column`] expressions. + /// Computed expressions, even if they reference `column`, do not match. The + /// comparison uses [`Column`] equality, so both the name and index must match. + /// If the same column appears more than once, this returns the first matching + /// position. + /// + /// # Example + /// + /// ```rust + /// use datafusion_common::ScalarValue; + /// use datafusion_physical_expr::expressions::{Column, Literal}; + /// use datafusion_physical_expr::projection::{ProjectionExpr, ProjectionExprs}; + /// use std::sync::Arc; + /// + /// let projection = ProjectionExprs::new([ + /// ProjectionExpr::new(Arc::new(Column::new("b", 1)), "b"), + /// ProjectionExpr::new( + /// Arc::new(Literal::new(ScalarValue::Int32(Some(42)))), + /// "answer", + /// ), + /// ProjectionExpr::new(Arc::new(Column::new("a", 0)), "a"), + /// ]); + /// + /// assert_eq!( + /// projection.projected_column_position(&Column::new("b", 1)), + /// Some(0) + /// ); + /// assert_eq!( + /// projection.projected_column_position(&Column::new("a", 0)), + /// Some(2) + /// ); + /// + /// // The literal projection is not a Column expression. + /// assert_eq!( + /// projection.projected_column_position(&Column::new("answer", 1)), + /// None + /// ); + /// + /// // Columns not present in the projection also return None. + /// assert_eq!( + /// projection.projected_column_position(&Column::new("c", 2)), + /// None + /// ); + /// ``` + pub fn projected_column_position(&self, column: &Column) -> Option { + self.iter().position(|expr| { + expr.expr + .downcast_ref::() + .is_some_and(|projected| projected == column) + }) + } } /// Propagate column statistics through CAST projections. Other expressions @@ -736,7 +790,7 @@ fn project_column_statistics_through_expr( column_stats: &[ColumnStatistics], ) -> ColumnStatistics { if let Some(col) = expr.downcast_ref::() { - return column_stats[col.index()].clone(); + return column_statistics_at(column_stats, col.index()); } let Some(cast_expr) = expr.downcast_ref::() else { return ColumnStatistics::new_unknown(); @@ -760,6 +814,16 @@ fn project_column_statistics_through_expr( } } +fn column_statistics_at( + column_stats: &[ColumnStatistics], + index: usize, +) -> ColumnStatistics { + column_stats + .get(index) + .cloned() + .unwrap_or_else(ColumnStatistics::new_unknown) +} + impl<'a> IntoIterator for &'a ProjectionExprs { type Item = &'a ProjectionExpr; type IntoIter = std::slice::Iter<'a, ProjectionExpr>; @@ -2202,6 +2266,43 @@ pub(crate) mod tests { Schema::new(vec![field_0, field_1, field_2]) } + #[test] + fn test_projected_column_position_returns_output_position() { + let projection = ProjectionExprs::new([ + ProjectionExpr::new(Arc::new(Column::new("col2", 2)), "col2"), + ProjectionExpr::new(Arc::new(Column::new("col0", 0)), "col0"), + ]); + + assert_eq!( + projection.projected_column_position(&Column::new("col2", 2)), + Some(0) + ); + assert_eq!( + projection.projected_column_position(&Column::new("col0", 0)), + Some(1) + ); + } + + #[test] + fn test_projected_column_position_returns_none_for_non_column_or_missing() { + let projection = ProjectionExprs::new([ + ProjectionExpr::new( + Arc::new(Literal::new(ScalarValue::Int64(Some(42)))), + "col1", + ), + ProjectionExpr::new(Arc::new(Column::new("col0", 0)), "col0"), + ]); + + assert_eq!( + projection.projected_column_position(&Column::new("col1", 1)), + None + ); + assert_eq!( + projection.projected_column_position(&Column::new("col2", 2)), + None + ); + } + #[test] fn test_stats_projection_columns_only() { let source = get_stats(); @@ -2913,6 +3014,57 @@ pub(crate) mod tests { byte_size: Precision::Absent, } ); + + Ok(()) + } + + #[test] + fn test_project_statistics_missing_column_stats_are_unknown() -> Result<()> { + let mut input_stats = get_stats(); + let input_schema = get_schema(); + input_stats.column_statistics.truncate(2); + + // The schema has col2, but the statistics do not. This can happen for + // source-provided virtual columns that are available at execution time + // but not represented in file-level statistics. + let projection = ProjectionExprs::new(vec![ + ProjectionExpr { + expr: Arc::new(Column::new("col2", 2)), + alias: "virtual_col".to_string(), + }, + ProjectionExpr { + expr: Arc::new(CastExpr::new( + Arc::new(Column::new("col2", 2)), + DataType::Float64, + None, + )), + alias: "casted_virtual_col".to_string(), + }, + ProjectionExpr { + expr: Arc::new(Column::new("col0", 0)), + alias: "physical_col".to_string(), + }, + ]); + + let output_stats = projection.project_statistics( + input_stats, + &projection.project_schema(&input_schema)?, + )?; + + assert_eq!(output_stats.column_statistics.len(), 3); + assert_eq!( + output_stats.column_statistics[0], + ColumnStatistics::new_unknown() + ); + assert_eq!( + output_stats.column_statistics[1], + ColumnStatistics::new_unknown() + ); + assert_eq!( + output_stats.column_statistics[2].max_value, + Precision::Exact(ScalarValue::Int64(Some(21))) + ); + Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/metrics.rs b/datafusion/physical-plan/src/aggregates/group_values/metrics.rs index a0934b976ea79..1c6285d793b88 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/metrics.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/metrics.rs @@ -19,6 +19,7 @@ use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, Time}; +#[derive(Clone)] pub(crate) struct GroupByMetrics { /// Time spent calculating the group IDs from the evaluated grouping columns. pub(crate) time_calculating_group_ids: Time, diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs index abc3aba88ad48..8625772e2c995 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs @@ -21,7 +21,6 @@ use crate::aggregates::group_values::multi_group_by::{ use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; use arrow::array::{ Array, ArrayRef, AsArray, BooleanBufferBuilder, ByteView, GenericByteViewArray, - make_view, }; use arrow::buffer::{Buffer, ScalarBuffer}; use arrow::datatypes::ByteViewType; @@ -176,9 +175,9 @@ impl ByteViewGroupValueBuilder { // copy them directly instead of going through value() → make_view(). self.views.extend(rows.iter().map(|&row| arr.views()[row])); } else { - // Slow path: some strings are non-inline (>12 bytes). - // Read views directly to avoid array.value(row) overhead and - // reuse the source view's prefix instead of recomputing it via make_view. + // Slow path: some strings may be non-inline (>12 bytes). + // Pre-reserve and delegate to do_append_val_inner which + // reads raw views directly and reuses source prefixes. self.views.try_reserve(rows.len()).map_err(|e| { datafusion_common::exec_datafusion_err!( "failed to reserve {0} views: {e}", @@ -186,33 +185,7 @@ impl ByteViewGroupValueBuilder { ) })?; for &row in rows { - let view = arr.views()[row]; - let len = view as u32; - if len <= 12 { - // This row happens to be inline; copy view directly. - self.views.push(view); - } else { - let src = ByteView::from(view); - // ensure_in_progress_big_enough must be called before computing - // new_buffer_index / new_offset — it may flush in_progress to completed. - self.ensure_in_progress_big_enough(len as usize); - let new_buffer_index = self.completed.len() as u32; - let new_offset = self.in_progress.len() as u32; - let src_buf = &arr.data_buffers()[src.buffer_index as usize]; - self.in_progress.extend_from_slice( - &src_buf[src.offset as usize - ..(src.offset + src.length) as usize], - ); - // Reuse prefix from the source view — avoids re-reading first 4 bytes. - let new_view = ByteView { - length: src.length, - prefix: src.prefix, - buffer_index: new_buffer_index, - offset: new_offset, - } - .as_u128(); - self.views.push(new_view); - } + self.do_append_val_inner(arr, row); } } } @@ -230,25 +203,33 @@ impl ByteViewGroupValueBuilder { where B: ByteViewType, { - let value: &[u8] = array.value(row).as_ref(); + // SAFETY: the caller ensures `row` is valid + let view = unsafe { *array.views().get_unchecked(row) }; + let len = view as u32; - let value_len = value.len(); - let view = if value_len <= 12 { - make_view(value, 0, 0) + if len <= 12 { + // Inline value: the view is already self-contained, push as-is. + self.views.push(view); } else { - // Ensure big enough block to hold the value firstly - self.ensure_in_progress_big_enough(value_len); - - // Append value - let buffer_index = self.completed.len(); - let offset = self.in_progress.len(); - self.in_progress.extend_from_slice(value); - - make_view(value, buffer_index as u32, offset as u32) - }; - - // Append view - self.views.push(view); + // Non-inline value: copy the buffer data and construct a new view + // that points into our own buffers, reusing the source prefix. + let src = ByteView::from(view); + self.ensure_in_progress_big_enough(len as usize); + let new_buffer_index = self.completed.len() as u32; + let new_offset = self.in_progress.len() as u32; + let src_buf = &array.data_buffers()[src.buffer_index as usize]; + self.in_progress.extend_from_slice( + &src_buf[src.offset as usize..(src.offset + src.length) as usize], + ); + let new_view = ByteView { + length: src.length, + prefix: src.prefix, + buffer_index: new_buffer_index, + offset: new_offset, + } + .as_u128(); + self.views.push(new_view); + } } fn ensure_in_progress_big_enough(&mut self, value_len: usize) { diff --git a/datafusion/physical-plan/src/aggregates/hash_aggregate.rs b/datafusion/physical-plan/src/aggregates/hash_aggregate.rs index 0c8593efd05bb..59ee09912f621 100644 --- a/datafusion/physical-plan/src/aggregates/hash_aggregate.rs +++ b/datafusion/physical-plan/src/aggregates/hash_aggregate.rs @@ -25,6 +25,7 @@ //! //! See issue for details: +use std::ops::ControlFlow; use std::sync::Arc; use std::task::{Context, Poll}; @@ -36,17 +37,20 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; use super::AggregateExec; -use super::hash_table::{AggregateHashTable, Final, Partial}; -use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput, SpillMetrics}; +use super::hash_table::{AggregateHashTable, Final, Partial, PartialSkip}; +use super::skip_partial::SkipAggregationProbe; +use crate::metrics::{ + BaselineMetrics, MetricBuilder, MetricCategory, RecordOutput, SpillMetrics, +}; use crate::stream::EmptyRecordBatchStream; use crate::{InputOrderMode, RecordBatchStream, SendableRecordBatchStream, metrics}; -/// Hash aggregation uses a 2-stage (partial and final) hash aggregation, this stream -/// is for the partial stage. +/// Hash aggregation is implemented in two stages: partial and final. This +/// stream implements the partial stage. /// /// # Example /// -/// select k, avg(v) from t group by k; +/// SELECT k, AVG(v) FROM t GROUP BY k; /// /// ## Plan /// AggregateExec(stage=final) @@ -55,15 +59,18 @@ use crate::{InputOrderMode, RecordBatchStream, SendableRecordBatchStream, metric /// /// ## Partial Stage Behavior /// Input: raw rows -/// Output: partial states for all groups (e.g. for avg(x), it's sum(x), count(x)) +/// Output: partial states for all groups (for example, `AVG(x)` emits `SUM(x)` +/// and `COUNT(x)`) /// /// ## Final Stage Behavior /// Input: partial states -/// Output: results for all groups (e.g. for avg(x), it's avg(x) calculated from the state) +/// Output: results for all groups (for example, `AVG(x)` calculated from the +/// state) /// /// # Optimization: DISTINCT LIMIT Soft Limit /// -/// This optimization applies to both [`PartialHashAggregateStream`] and [`FinalHashAggregateStream`] +/// This optimization applies to both [`PartialHashAggregateStream`] and +/// [`FinalHashAggregateStream`]. /// /// Unordered distinct queries such as: /// @@ -87,6 +94,17 @@ use crate::{InputOrderMode, RecordBatchStream, SendableRecordBatchStream, metric /// This operator does not guarantee an exact limit because a single batch can /// cross the threshold. The downstream limit operator enforces the exact result /// size. +/// +/// # Optimization: Partial Aggregation Skip +/// +/// Partial aggregation can be counterproductive for high-cardinality inputs, +/// where most rows create distinct groups. The stream probes the ratio of +/// accumulated groups to input rows while it is still aggregating. If the ratio +/// crosses the configured threshold and all aggregate accumulators can convert +/// raw inputs directly to partial state, the stream emits any already +/// accumulated groups, then switches to a skip state. In that state, each +/// remaining input batch is converted directly to partial aggregate state rows +/// without inserting the rows into the grouped hash table. pub(crate) struct PartialHashAggregateStream { /// Output schema: group columns followed by partial aggregate state columns. schema: SchemaRef, @@ -94,9 +112,6 @@ pub(crate) struct PartialHashAggregateStream { /// Input batches containing raw rows, not partial aggregate state. input: SendableRecordBatchStream, - /// Hash table state for this aggregate stream. - hash_table: AggregateHashTable, - /// Memory reservation for group keys and accumulators. reservation: MemoryReservation, @@ -106,15 +121,69 @@ pub(crate) struct PartialHashAggregateStream { /// Tracks partial aggregation row reduction, matching `GroupedHashAggregateStream`. reduction_factor: metrics::RatioMetrics, + /// Tracks whether partial aggregation should switch to direct state conversion. + skip_aggregation_probe: Option, + /// Optional soft limit on the number of groups to accumulate before output. /// /// Invariant: when this is `Some(..)`, the accumulators inside `hash_table` must /// be empty. See struct comments for details. group_values_soft_limit: Option, + + /// Tracks the high-level stream lifecycle. The hash table owns the lower-level + /// state for materializing and slicing output batches. + state: Option, +} + +/// States for partial hash aggregation processing. +enum PartialHashAggregateState { + ReadingInput { + hash_table: AggregateHashTable, + }, + ProducingOutput { + hash_table: AggregateHashTable, + /// If `None`, partial skip was never triggered and this state will + /// finish in `Done`. If `Some`, partial skip has triggered and the + /// stream will move to `SkippingAggregation` after these accumulated + /// groups are emitted. + skip_hash_table: Option>, + }, + SkippingAggregation { + hash_table: AggregateHashTable, + }, + Done, +} + +type PartialHashAggregatePoll = Poll>>; +type PartialHashAggregateStateTransition = ControlFlow< + (PartialHashAggregatePoll, PartialHashAggregateState), + PartialHashAggregateState, +>; + +impl PartialHashAggregateState { + fn hash_table(&self) -> &AggregateHashTable { + match self { + Self::ReadingInput { hash_table } + | Self::ProducingOutput { hash_table, .. } => hash_table, + Self::SkippingAggregation { .. } | Self::Done => { + unreachable!("state does not hold a partial hash table") + } + } + } + + fn hash_table_mut(&mut self) -> &mut AggregateHashTable { + match self { + Self::ReadingInput { hash_table } + | Self::ProducingOutput { hash_table, .. } => hash_table, + Self::SkippingAggregation { .. } | Self::Done => { + unreachable!("state does not hold a partial hash table") + } + } + } } -/// Hash aggregation uses a 2-stage (partial and final) hash aggregation, this stream -/// is for the final stage. +/// Hash aggregation is implemented in two stages: partial and final. This +/// stream implements the final stage. /// /// See [`PartialHashAggregateStream`] for details. pub(crate) struct FinalHashAggregateStream { @@ -124,17 +193,76 @@ pub(crate) struct FinalHashAggregateStream { /// Input batches containing partial aggregate state rows. input: SendableRecordBatchStream, - /// Hash table state for this aggregate stream. - hash_table: AggregateHashTable, - /// Execution metrics shared with the aggregate plan node. baseline_metrics: BaselineMetrics, /// Memory reservation for group keys and accumulators. reservation: MemoryReservation, - /// See comments for the same variable in [`PartialHashAggregateStream`] + /// See comments for the same variable in [`PartialHashAggregateStream`]. group_values_soft_limit: Option, + + /// Tracks the high-level stream lifecycle. The hash table owns the lower-level + /// state for materializing and slicing output batches. + state: Option, +} + +/// States for final hash aggregation processing. +// The typestate pattern is used in case the inner logic becomes more complex in +// the future. +enum FinalHashAggregateState { + ReadingInput { + hash_table: AggregateHashTable, + }, + ProducingOutput { + hash_table: AggregateHashTable, + }, + Done, +} + +type FinalHashAggregatePoll = Poll>>; +type FinalHashAggregateStateTransition = ControlFlow< + (FinalHashAggregatePoll, FinalHashAggregateState), + FinalHashAggregateState, +>; + +impl FinalHashAggregateState { + fn hash_table(&self) -> &AggregateHashTable { + match self { + Self::ReadingInput { hash_table } | Self::ProducingOutput { hash_table } => { + hash_table + } + Self::Done => unreachable!("Done state does not hold a hash table"), + } + } + + fn hash_table_mut(&mut self) -> &mut AggregateHashTable { + match self { + Self::ReadingInput { hash_table } | Self::ProducingOutput { hash_table } => { + hash_table + } + Self::Done => unreachable!("Done state does not hold a hash table"), + } + } + + fn into_hash_table(self) -> AggregateHashTable { + match self { + Self::ReadingInput { hash_table } | Self::ProducingOutput { hash_table } => { + hash_table + } + Self::Done => unreachable!("Done state does not hold a hash table"), + } + } + + fn into_producing_output(self) -> Self { + Self::ProducingOutput { + hash_table: self.into_hash_table(), + } + } + + fn into_done(self) -> Self { + Self::Done + } } impl PartialHashAggregateStream { @@ -163,6 +291,29 @@ impl PartialHashAggregateStream { Arc::clone(&schema), batch_size, )?; + let can_skip_aggregation = + agg.group_by.is_single() && hash_table.can_skip_aggregation(); + let skip_aggregation_probe = if can_skip_aggregation { + let options = &context.session_config().options().execution; + let probe_ratio_threshold = + options.skip_partial_aggregation_probe_ratio_threshold; + // A threshold >= 1.0 means the ratio (num_groups / input_rows) can + // never exceed it, so the feature is effectively disabled. + if probe_ratio_threshold >= 1.0 { + None + } else { + let skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) + .with_category(MetricCategory::Rows) + .counter("skipped_aggregation_rows", partition); + Some(SkipAggregationProbe::new( + options.skip_partial_aggregation_probe_rows_threshold, + probe_ratio_threshold, + skipped_aggregation_rows, + )) + } + } else { + None + }; let reservation = MemoryConsumer::new(format!("PartialHashAggregateStream[{partition}]")) @@ -171,106 +322,409 @@ impl PartialHashAggregateStream { Ok(Self { schema, input, - hash_table, baseline_metrics, reservation, reduction_factor, + skip_aggregation_probe, group_values_soft_limit: agg.limit_options().map(|config| config.limit()), + state: Some(PartialHashAggregateState::ReadingInput { hash_table }), }) } /// See comments in [`Self::group_values_soft_limit`] for details. - fn hit_soft_group_limit(&self) -> bool { + fn hit_soft_group_limit(&self, hash_table: &AggregateHashTable) -> bool { self.group_values_soft_limit - .is_some_and(|limit| limit <= self.hash_table.building_group_count()) + .is_some_and(|limit| limit <= hash_table.building_group_count()) } - fn start_output(&mut self) -> Result<()> { - let input_schema = self.input.schema(); - self.input = Box::pin(EmptyRecordBatchStream::new(input_schema)); - self.hash_table.start_output() + /// Updates skip aggregation probe state. + fn update_skip_aggregation_probe(&mut self, input_rows: usize, num_groups: usize) { + if let Some(probe) = self.skip_aggregation_probe.as_mut() { + probe.update_state(input_rows, num_groups); + } } -} -impl Stream for PartialHashAggregateStream { - type Item = Result; + /// Returns true if the aggregation probe indicates that aggregation + /// should be skipped. + fn should_skip_aggregation(&self) -> bool { + self.skip_aggregation_probe + .as_ref() + .is_some_and(|probe| probe.should_skip()) + } - fn poll_next( - mut self: std::pin::Pin<&mut Self>, + fn start_output( + &mut self, + hash_table: &mut AggregateHashTable, + close_input: bool, + ) -> Result<()> { + if close_input { + let input_schema = self.input.schema(); + self.input = Box::pin(EmptyRecordBatchStream::new(input_schema)); + } + hash_table.start_output() + } + + /// Handle ReadingInput state - aggregate input batches into the hash table. + /// + /// See comments at `poll_next()` for details. + /// + /// Returns the next operator state with control flow decision. + fn handle_reading_input( + &mut self, cx: &mut Context<'_>, - ) -> Poll> { - let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + mut original_state: PartialHashAggregateState, + ) -> PartialHashAggregateStateTransition { + debug_assert!(matches!( + &original_state, + PartialHashAggregateState::ReadingInput { .. } + )); + debug_assert!(original_state.hash_table().is_building()); - loop { - if self.hash_table.is_done() { - let _ = self.reservation.try_resize(0); - return Poll::Ready(None); - } else if self.hash_table.is_building() { - match self.input.poll_next_unpin(cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(Some(Ok(batch))) => { - let timer = elapsed_compute.timer(); - self.reduction_factor.add_total(batch.num_rows()); - let result = self.hash_table.aggregate_batch(&batch); - timer.done(); - - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); - } + match self.input.poll_next_unpin(cx) { + Poll::Pending => ControlFlow::Break((Poll::Pending, original_state)), + Poll::Ready(Some(Ok(batch))) => { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + let timer = elapsed_compute.timer(); + let input_rows = batch.num_rows(); + self.reduction_factor.add_total(input_rows); + let result = original_state.hash_table_mut().aggregate_batch(&batch); + timer.done(); - if self.hit_soft_group_limit() { - let timer = elapsed_compute.timer(); - let result = self.start_output(); - timer.done(); + if let Err(e) = result { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); + } - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); - } + if self.hit_soft_group_limit(original_state.hash_table()) { + let timer = elapsed_compute.timer(); + let result = self.start_output(original_state.hash_table_mut(), true); + timer.done(); - continue; - } + if let Err(e) = result { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); + } - // TODO: impl memory-limited aggr, when OOM directly send - // partial state to final aggregate stage - if let Err(e) = - self.reservation.try_resize(self.hash_table.memory_size()) - { - return Poll::Ready(Some(Err(e))); + let PartialHashAggregateState::ReadingInput { hash_table } = + original_state + else { + unreachable!("expected reading input state") + }; + return ControlFlow::Continue( + PartialHashAggregateState::ProducingOutput { + hash_table, + skip_hash_table: None, + }, + ); + } + + self.update_skip_aggregation_probe( + input_rows, + original_state.hash_table().building_group_count(), + ); + + // True branch: a decision has been made to skip partial aggregation. + if self.should_skip_aggregation() { + let timer = elapsed_compute.timer(); + let result = match original_state.hash_table().partial_skip_table() { + Ok(skip_hash_table) => self + .start_output(original_state.hash_table_mut(), false) + .map(|()| skip_hash_table), + Err(e) => Err(e), + }; + timer.done(); + + match result { + Ok(skip_hash_table) => { + let PartialHashAggregateState::ReadingInput { hash_table } = + original_state + else { + unreachable!("expected reading input state") + }; + + // Move to `ProducingOutput` first. Its `skip_hash_table` + // field moves the stream to skip-partial aggregation after + // the accumulated batches have been output. + return ControlFlow::Continue( + PartialHashAggregateState::ProducingOutput { + hash_table, + skip_hash_table: Some(skip_hash_table), + }, + ); + } + Err(e) => { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); } } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(e))); + } + + // TODO: impl memory-limited aggr, when OOM directly send + // partial state to final aggregate stage + if let Err(e) = self + .reservation + .try_resize(original_state.hash_table().memory_size()) + { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); + } + + ControlFlow::Continue(original_state) + } + Poll::Ready(Some(Err(e))) => { + ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)) + } + Poll::Ready(None) => { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + let timer = elapsed_compute.timer(); + let result = self.start_output(original_state.hash_table_mut(), true); + timer.done(); + + match result { + Ok(()) => { + let PartialHashAggregateState::ReadingInput { hash_table } = + original_state + else { + unreachable!("expected reading input state") + }; + ControlFlow::Continue( + PartialHashAggregateState::ProducingOutput { + hash_table, + skip_hash_table: None, + }, + ) } - Poll::Ready(None) => { - let timer = elapsed_compute.timer(); - let result = self.start_output(); - timer.done(); + Err(e) => { + ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)) + } + } + } + } + } - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); + /// Handle ProducingOutput state - emit partial aggregate state batches. + /// + /// See comments at `poll_next()` for details. + /// + /// Returns the next operator state with control flow decision. + fn handle_producing_output( + &mut self, + mut original_state: PartialHashAggregateState, + ) -> PartialHashAggregateStateTransition { + debug_assert!(matches!( + &original_state, + PartialHashAggregateState::ProducingOutput { .. } + )); + debug_assert!(!original_state.hash_table().is_building()); + + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + let timer = elapsed_compute.timer(); + let result = original_state.hash_table_mut().next_output_batch(); + timer.done(); + + match result { + Ok(Some(batch)) => { + let _ = self + .reservation + .try_resize(original_state.hash_table().memory_size()); + self.reduction_factor.add_part(batch.num_rows()); + debug_assert!(batch.num_rows() > 0); + let next_state = if original_state.hash_table().is_done() { + match original_state { + PartialHashAggregateState::ProducingOutput { + skip_hash_table: Some(hash_table), + .. + } => { + PartialHashAggregateState::SkippingAggregation { hash_table } } + PartialHashAggregateState::ProducingOutput { + skip_hash_table: None, + .. + } => PartialHashAggregateState::Done, + _ => unreachable!("expected producing output state"), } + } else { + original_state + }; + + ControlFlow::Break(( + Poll::Ready(Some(Ok(batch.record_output(&self.baseline_metrics)))), + next_state, + )) + } + Ok(None) => { + let _ = self.reservation.try_resize(0); + // If the previous `Aggregating` stage decided to skip partial + // aggregation, go to the `SkippingAggregation` stage; otherwise finish. + let next_state = match original_state { + PartialHashAggregateState::ProducingOutput { + skip_hash_table: Some(hash_table), + .. + } => PartialHashAggregateState::SkippingAggregation { hash_table }, + PartialHashAggregateState::ProducingOutput { + skip_hash_table: None, + .. + } => PartialHashAggregateState::Done, + _ => unreachable!("expected producing output state"), + }; + ControlFlow::Continue(next_state) + } + Err(e) => ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)), + } + } + + /// Handle SkippingAggregation state - convert raw input directly to partial states. + /// + /// See comments at `poll_next()` for details. + /// + /// Returns the next operator state with control flow decision. + fn handle_skipping_aggregation( + &mut self, + cx: &mut Context<'_>, + mut original_state: PartialHashAggregateState, + ) -> PartialHashAggregateStateTransition { + debug_assert!(matches!( + &original_state, + PartialHashAggregateState::SkippingAggregation { .. } + )); + + match self.input.poll_next_unpin(cx) { + Poll::Pending => ControlFlow::Break((Poll::Pending, original_state)), + Poll::Ready(Some(Ok(batch))) => { + if let Some(probe) = self.skip_aggregation_probe.as_mut() { + probe.record_skipped(&batch); } - } else { + + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); let timer = elapsed_compute.timer(); - let result = self.hash_table.next_output_batch(); + let result = match &mut original_state { + PartialHashAggregateState::SkippingAggregation { hash_table } => { + hash_table.convert_batch_to_state(&batch) + } + _ => unreachable!("expected skipping aggregation state"), + }; timer.done(); match result { - Ok(Some(batch)) => { - let _ = - self.reservation.try_resize(self.hash_table.memory_size()); - self.reduction_factor.add_part(batch.num_rows()); - debug_assert!(batch.num_rows() > 0); - return Poll::Ready(Some(Ok( - batch.record_output(&self.baseline_metrics) - ))); + Ok(batch) => ControlFlow::Break(( + Poll::Ready(Some( + Ok(batch.record_output(&self.baseline_metrics)), + )), + original_state, + )), + Err(e) => { + ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)) } - Ok(None) => { - let _ = self.reservation.try_resize(0); - return Poll::Ready(None); - } - Err(e) => return Poll::Ready(Some(Err(e))), + } + } + Poll::Ready(Some(Err(e))) => { + ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)) + } + Poll::Ready(None) => { + let input_schema = self.input.schema(); + self.input = Box::pin(EmptyRecordBatchStream::new(input_schema)); + ControlFlow::Continue(PartialHashAggregateState::Done) + } + } + } +} + +impl Stream for PartialHashAggregateStream { + type Item = Result; + + /// Entry point for the partial hash aggregate state machine. + /// + /// See comments in [`PartialHashAggregateStream`] for high-level ideas. + /// + /// State transition graph: + /// + /// ```text + /// (start) + /// -> ReadingInput + /// The stream starts by polling input and aggregating batches into the + /// in-memory hash table. + /// + /// ReadingInput + /// -> ReadingInput + /// Aggregate one batch, update the inner aggregate hash table, and + /// continue with the next input batch. + /// -> ProducingOutput(skip=None) + /// Input was exhausted, or the soft group limit was reached. Move to + /// the next state to start outputting. + /// -> ProducingOutput(skip=Some) + /// Partial skip aggregation was triggered. First move to the + /// `ProducingOutput` state to drain the accumulated state, then move to + /// the `SkippingAggregation` state to convert input directly to partial + /// state without aggregation. + /// + /// ProducingOutput(skip=None) + /// -> ProducingOutput(skip=None) + /// One accumulated output batch was yielded, repeat to continue producing + /// output incrementally. + /// -> Done + /// All accumulated output was emitted. + /// + /// ProducingOutput(skip=Some) + /// -> ProducingOutput(skip=Some) + /// One accumulated output batch was yielded, repeat to continue producing + /// output incrementally. + /// -> SkippingAggregation + /// All accumulated output was emitted. Continue by converting raw + /// input batches directly to partial aggregate state. + /// + /// SkippingAggregation + /// -> SkippingAggregation + /// One `convert_to_state` batch was yielded; repeat to continue + /// processing. + /// -> Done + /// Input was exhausted. + /// + /// Done + /// -> (end) + /// ``` + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + loop { + let cur_state = self + .state + .take() + .expect("PartialHashAggregateStream state should not be None"); + + let next_state = match cur_state { + state @ PartialHashAggregateState::ReadingInput { .. } => { + self.handle_reading_input(cx, state) + } + state @ PartialHashAggregateState::ProducingOutput { .. } => { + self.handle_producing_output(state) + } + state @ PartialHashAggregateState::SkippingAggregation { .. } => { + self.handle_skipping_aggregation(cx, state) + } + state @ PartialHashAggregateState::Done => { + let _ = self.reservation.try_resize(0); + self.state = Some(state); + return Poll::Ready(None); + } + }; + + match next_state { + ControlFlow::Continue(next_state) => { + self.state = Some(next_state); + continue; + } + ControlFlow::Break((poll, next_state)) => { + self.state = Some(next_state); + return poll; } } } @@ -317,101 +771,217 @@ impl FinalHashAggregateStream { Ok(Self { schema, input, - hash_table, baseline_metrics, reservation, group_values_soft_limit: agg.limit_options().map(|config| config.limit()), + state: Some(FinalHashAggregateState::ReadingInput { hash_table }), }) } /// See comments in [`Self::group_values_soft_limit`] for details. - fn hit_soft_group_limit(&self) -> bool { + fn hit_soft_group_limit(&self, hash_table: &AggregateHashTable) -> bool { self.group_values_soft_limit - .is_some_and(|limit| limit <= self.hash_table.building_group_count()) + .is_some_and(|limit| limit <= hash_table.building_group_count()) } - fn start_output(&mut self) -> Result<()> { + fn start_output(&mut self, hash_table: &mut AggregateHashTable) -> Result<()> { let input_schema = self.input.schema(); self.input = Box::pin(EmptyRecordBatchStream::new(input_schema)); - self.hash_table.start_output() + hash_table.start_output() } -} - -impl Stream for FinalHashAggregateStream { - type Item = Result; - fn poll_next( - mut self: std::pin::Pin<&mut Self>, + /// Handle ReadingInput state - aggregate partial state batches into the hash table. + /// + /// See comments at `poll_next()` for details. + /// + /// Returns the next operator state with control flow decision. + fn handle_reading_input( + &mut self, cx: &mut Context<'_>, - ) -> Poll> { - let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); - - loop { - if self.hash_table.is_done() { - let _ = self.reservation.try_resize(0); - return Poll::Ready(None); - } else if self.hash_table.is_building() { - match self.input.poll_next_unpin(cx) { - Poll::Pending => return Poll::Pending, - Poll::Ready(Some(Ok(batch))) => { - let timer = elapsed_compute.timer(); - let result = self.hash_table.aggregate_batch(&batch); - timer.done(); - - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); - } + mut original_state: FinalHashAggregateState, + ) -> FinalHashAggregateStateTransition { + debug_assert!(matches!( + &original_state, + FinalHashAggregateState::ReadingInput { .. } + )); + debug_assert!(original_state.hash_table().is_building()); - if self.hit_soft_group_limit() { - let timer = elapsed_compute.timer(); - let result = self.start_output(); - timer.done(); + match self.input.poll_next_unpin(cx) { + Poll::Pending => ControlFlow::Break((Poll::Pending, original_state)), + Poll::Ready(Some(Ok(batch))) => { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + let timer = elapsed_compute.timer(); + let result = original_state.hash_table_mut().aggregate_batch(&batch); + timer.done(); - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); - } + if let Err(e) = result { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); + } - continue; - } + if self.hit_soft_group_limit(original_state.hash_table()) { + let timer = elapsed_compute.timer(); + let result = self.start_output(original_state.hash_table_mut()); + timer.done(); - if let Err(e) = - self.reservation.try_resize(self.hash_table.memory_size()) - { - return Poll::Ready(Some(Err(e))); - } - } - Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(e))); + if let Err(e) = result { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); } - Poll::Ready(None) => { - let timer = elapsed_compute.timer(); - let result = self.start_output(); - timer.done(); - if let Err(e) = result { - return Poll::Ready(Some(Err(e))); - } - } + return ControlFlow::Continue(original_state.into_producing_output()); } - } else { + + if let Err(e) = self + .reservation + .try_resize(original_state.hash_table().memory_size()) + { + return ControlFlow::Break(( + Poll::Ready(Some(Err(e))), + original_state, + )); + } + + ControlFlow::Continue(original_state) + } + Poll::Ready(Some(Err(e))) => { + ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)) + } + Poll::Ready(None) => { + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); let timer = elapsed_compute.timer(); - let result = self.hash_table.next_output_batch(); + let result = self.start_output(original_state.hash_table_mut()); timer.done(); match result { - Ok(Some(batch)) => { - let _ = - self.reservation.try_resize(self.hash_table.memory_size()); - debug_assert!(batch.num_rows() > 0); - return Poll::Ready(Some(Ok( - batch.record_output(&self.baseline_metrics) - ))); + Ok(()) => { + ControlFlow::Continue(original_state.into_producing_output()) } - Ok(None) => { - let _ = self.reservation.try_resize(0); - return Poll::Ready(None); + Err(e) => { + ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)) } - Err(e) => return Poll::Ready(Some(Err(e))), + } + } + } + } + + /// Handle ProducingOutput state - emit final aggregate value batches. + /// + /// See comments at `poll_next()` for details. + /// + /// Returns the next operator state with control flow decision. + fn handle_producing_output( + &mut self, + mut original_state: FinalHashAggregateState, + ) -> FinalHashAggregateStateTransition { + debug_assert!(matches!( + &original_state, + FinalHashAggregateState::ProducingOutput { .. } + )); + debug_assert!(!original_state.hash_table().is_building()); + + let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); + let timer = elapsed_compute.timer(); + let result = original_state.hash_table_mut().next_output_batch(); + timer.done(); + + match result { + Ok(Some(batch)) => { + let _ = self + .reservation + .try_resize(original_state.hash_table().memory_size()); + debug_assert!(batch.num_rows() > 0); + let next_state = if original_state.hash_table().is_done() { + original_state.into_done() + } else { + original_state + }; + + ControlFlow::Break(( + Poll::Ready(Some(Ok(batch.record_output(&self.baseline_metrics)))), + next_state, + )) + } + Ok(None) => { + let _ = self.reservation.try_resize(0); + ControlFlow::Continue(original_state.into_done()) + } + Err(e) => ControlFlow::Break((Poll::Ready(Some(Err(e))), original_state)), + } + } +} + +impl Stream for FinalHashAggregateStream { + type Item = Result; + + /// Entry point for the final hash aggregate state machine. + /// + /// See comments in [`FinalHashAggregateStream`] for high-level ideas. + /// + /// State transition graph: + /// + /// ```text + /// (start) + /// -> ReadingInput + /// The stream starts by polling partial-state input and aggregating + /// those states into the final hash table. + /// + /// ReadingInput + /// -> ReadingInput + /// Aggregate one partial-state input batch, update the inner aggregate + /// hash table, and continue with the next input batch. + /// + /// -> ProducingOutput + /// Input was exhausted, or the soft group limit was reached. Move to + /// the next state to start outputting final aggregate values. + /// + /// ProducingOutput + /// -> ProducingOutput + /// One final output batch was yielded; repeat to continue producing + /// output incrementally. + /// + /// -> Done + /// All final output was emitted. + /// + /// Done + /// -> (end) + /// ``` + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + loop { + let cur_state = self + .state + .take() + .expect("FinalHashAggregateStream state should not be None"); + + let next_state = match cur_state { + state @ FinalHashAggregateState::ReadingInput { .. } => { + self.handle_reading_input(cx, state) + } + state @ FinalHashAggregateState::ProducingOutput { .. } => { + self.handle_producing_output(state) + } + state @ FinalHashAggregateState::Done => { + let _ = self.reservation.try_resize(0); + self.state = Some(state); + return Poll::Ready(None); + } + }; + + match next_state { + ControlFlow::Continue(next_state) => { + self.state = Some(next_state); + continue; + } + ControlFlow::Break((poll, next_state)) => { + self.state = Some(next_state); + return poll; } } } @@ -423,3 +993,274 @@ impl RecordBatchStream for FinalHashAggregateStream { Arc::clone(&self.schema) } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use super::*; + use crate::aggregates::{AggregateMode, PhysicalGroupBy}; + use crate::execution_plan::ExecutionPlan; + use crate::test::TestMemoryExec; + + use arrow::array::{Int32Array, Int64Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::Result; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_functions_aggregate::count::count_udaf; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_expr::expressions::col; + use futures::StreamExt; + + #[tokio::test] + async fn test_partial_hash_stream_double_emission_race_condition_bug() -> Result<()> { + // Fix for https://github.com/apache/datafusion/issues/18701 + // This test specifically proves that we have fixed double emission race condition + // where emit_early_if_necessary() and switch_to_skip_aggregation() + // both emit in the same loop iteration, causing data loss + + let schema = Arc::new(Schema::new(vec![ + Field::new("group_col", DataType::Int32, false), + Field::new("value_col", DataType::Int64, false), + ])); + + // Create data that will trigger BOTH conditions in the same iteration: + // 1. More groups than batch_size (triggers early emission when memory pressure hits) + // 2. High cardinality ratio (triggers skip aggregation) + let batch_size = 1024; // We'll set this in session config + let num_groups = batch_size + 100; // Slightly more than batch_size (1124 groups) + + // Create exactly 1 row per group = 100% cardinality ratio + let group_ids: Vec = (0..num_groups as i32).collect(); + let values: Vec = vec![1; num_groups]; + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids)), + Arc::new(Int64Array::from(values)), + ], + )?; + let input_partitions = vec![vec![batch]]; + + // Create constrained memory to trigger early emission but not completely fail + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1024, 1.0) // small enough to start but will trigger pressure + .build_arc()?; + + let mut task_ctx = TaskContext::default().with_runtime(runtime); + + // Configure to trigger BOTH conditions: + // 1. Low probe threshold (triggers skip probe after few rows) + // 2. Low ratio threshold (triggers skip aggregation immediately) + // 3. Set batch_size to 1024 so our 1124 groups will trigger early emission + // This creates the race condition where both emit paths are triggered + let mut session_config = task_ctx.session_config().clone(); + session_config = session_config.set( + "datafusion.execution.batch_size", + &datafusion_common::ScalarValue::UInt64(Some(1024)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &datafusion_common::ScalarValue::UInt64(Some(50)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &datafusion_common::ScalarValue::Float64(Some(0.8)), + ); + task_ctx = task_ctx.with_session_config(session_config); + let task_ctx = Arc::new(task_ctx); + + // Create aggregate: COUNT(*) GROUP BY group_col + let group_expr = vec![(col("group_col", &schema)?, "group_col".to_string())]; + let aggr_expr = vec![Arc::new( + AggregateExprBuilder::new(count_udaf(), vec![col("value_col", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("count_value") + .build()?, + )]; + + let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; + let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); + + // Use Partial mode where the race condition occurs + let aggregate_exec = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new_single(group_expr), + aggr_expr, + vec![None], + exec, + Arc::clone(&schema), + )?; + + // Execute and collect results + let mut stream = + PartialHashAggregateStream::new(&aggregate_exec, &Arc::clone(&task_ctx), 0)?; + let mut results = Vec::new(); + + while let Some(result) = stream.next().await { + let batch = result?; + results.push(batch); + } + + // Count total groups emitted + let mut total_output_groups = 0; + for batch in &results { + total_output_groups += batch.num_rows(); + } + + assert_eq!( + total_output_groups, num_groups, + "Unexpected number of groups", + ); + + Ok(()) + } + + #[tokio::test] + async fn test_partial_hash_stream_skip_aggregation_probe_not_locked_until_skip() + -> Result<()> { + // Test that the probe is not locked until we actually decide to skip. + // This allows us to continue evaluating the skip condition across multiple batches. + // + // Scenario: + // - Batch 1: Hits rows threshold but NOT ratio threshold (low cardinality) -> don't skip + // - Batch 2: Now hits ratio threshold (high cardinality) -> skip + // + // Without the fix, the probe would be locked after batch 1, preventing the skip + // decision from being made on batch 2. + + let schema = Arc::new(Schema::new(vec![ + Field::new("group_col", DataType::Int32, false), + Field::new("value_col", DataType::Int32, false), + ])); + + // Configure thresholds: + // - probe_rows_threshold: 100 rows + // - probe_ratio_threshold: 0.8 (80%) + let probe_rows_threshold = 100; + let probe_ratio_threshold = 0.8; + + // Batch 1: 100 rows with only 10 unique groups + // Ratio: 10/100 = 0.1 (10%) < 0.8 -> should NOT skip + // This will hit the rows threshold but not the ratio threshold + let batch1_rows = 100; + let batch1_groups = 10; + let mut group_ids_batch1 = Vec::new(); + for i in 0..batch1_rows { + group_ids_batch1.push((i % batch1_groups) as i32); + } + let values_batch1: Vec = vec![1; batch1_rows]; + + let batch1 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids_batch1)), + Arc::new(Int32Array::from(values_batch1)), + ], + )?; + + // Batch 2: 360 rows with 360 unique NEW groups (starting from group 10) + // After batch 2, total: 460 rows, 370 groups + // Ratio: 370/460 is about 0.804 (80.4%) > 0.8 -> SHOULD decide to skip + let batch2_rows = 360; + let batch2_groups = 360; + let group_ids_batch2: Vec = (batch1_groups..(batch1_groups + batch2_groups)) + .map(|x| x as i32) + .collect(); + let values_batch2: Vec = vec![1; batch2_rows]; + + let batch2 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids_batch2)), + Arc::new(Int32Array::from(values_batch2)), + ], + )?; + + // Batch 3: This batch should be skipped since we decided to skip after batch 2 + // 100 rows with 100 unique groups (continuing from where batch 2 left off) + let batch3_rows = 100; + let batch3_groups = 100; + let batch3_start_group = batch1_groups + batch2_groups; + let group_ids_batch3: Vec = (batch3_start_group + ..(batch3_start_group + batch3_groups)) + .map(|x| x as i32) + .collect(); + let values_batch3: Vec = vec![1; batch3_rows]; + + let batch3 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids_batch3)), + Arc::new(Int32Array::from(values_batch3)), + ], + )?; + + let input_partitions = vec![vec![batch1, batch2, batch3]]; + + let runtime = RuntimeEnvBuilder::default().build_arc()?; + let mut task_ctx = TaskContext::default().with_runtime(runtime); + + // Configure skip aggregation settings + let mut session_config = task_ctx.session_config().clone(); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &datafusion_common::ScalarValue::UInt64(Some(probe_rows_threshold)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &datafusion_common::ScalarValue::Float64(Some(probe_ratio_threshold)), + ); + task_ctx = task_ctx.with_session_config(session_config); + let task_ctx = Arc::new(task_ctx); + + // Create aggregate: COUNT(*) GROUP BY group_col + let group_expr = vec![(col("group_col", &schema)?, "group_col".to_string())]; + let aggr_expr = vec![Arc::new( + AggregateExprBuilder::new(count_udaf(), vec![col("value_col", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("count_value") + .build()?, + )]; + + let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; + let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); + + // Use Partial mode + let aggregate_exec = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new_single(group_expr), + aggr_expr, + vec![None], + exec, + Arc::clone(&schema), + )?; + + // Execute and collect results + let mut stream = + PartialHashAggregateStream::new(&aggregate_exec, &Arc::clone(&task_ctx), 0)?; + let mut results = Vec::new(); + + while let Some(result) = stream.next().await { + let batch = result?; + results.push(batch); + } + + // Check that skip aggregation actually happened. + // The key metric is skipped_aggregation_rows. + let metrics = aggregate_exec.metrics().unwrap(); + let skipped_rows = metrics + .sum_by_name("skipped_aggregation_rows") + .map(|m| m.as_usize()) + .unwrap_or(0); + + // We expect batch 3's rows to be skipped (100 rows) + assert_eq!( + skipped_rows, batch3_rows, + "Expected batch 3's rows ({batch3_rows}) to be skipped", + ); + + Ok(()) + } +} diff --git a/datafusion/physical-plan/src/aggregates/hash_table.rs b/datafusion/physical-plan/src/aggregates/hash_table.rs index 87f16d0eebe6f..e6b2fa22c137f 100644 --- a/datafusion/physical-plan/src/aggregates/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/hash_table.rs @@ -22,9 +22,10 @@ use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray, new_null_array}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::{Result, internal_err}; +use datafusion_common::{Result, assert_eq_or_internal_err, internal_err}; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::{EmitTo, GroupsAccumulator}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use super::group_values::{GroupByMetrics, GroupValues, new_group_values}; use super::order::GroupOrdering; @@ -34,10 +35,11 @@ use super::{ group_id_array, max_duplicate_ordinal, }; use crate::PhysicalExpr; -use crate::metrics::{MetricBuilder, MetricCategory}; /// Marker for raw rows -> partial state aggregation. pub(super) struct Partial; +/// Marker for raw rows -> partial state conversion without aggregation. +pub(super) struct PartialSkip; /// Marker for partial state -> final value aggregation. pub(super) struct Final; @@ -76,6 +78,10 @@ pub(super) struct AggregateHashTable { } struct HashAggregateAccumulator { + /// Aggregate expression used to create a fresh accumulator for related + /// hash tables, such as the partial-skip table. + aggregate_expr: Arc, + /// Arguments to pass to this accumulator. /// /// Example: `CORR(x, y)` stores two expressions here, while `SUM(x)` stores one. @@ -147,17 +153,29 @@ enum AggregateHashTableState { impl HashAggregateAccumulator { fn new( + aggregate_expr: Arc, arguments: Vec>, filter: Option>, accumulator: Box, ) -> Self { Self { + aggregate_expr, arguments, filter, accumulator, } } + fn empty_like(&self) -> Result { + let accumulator = create_group_accumulator(&self.aggregate_expr)?; + Ok(Self::new( + Arc::clone(&self.aggregate_expr), + self.arguments.clone(), + self.filter.clone(), + accumulator, + )) + } + fn evaluate(&self, batch: &RecordBatch) -> Result { let arguments = self .arguments @@ -203,12 +221,8 @@ impl HashAggregateAccumulator { total_num_groups: usize, ) -> Result<()> { debug_assert!(values.filter.is_none()); - self.accumulator.merge_batch( - &values.arguments, - group_indices, - None, - total_num_groups, - ) + self.accumulator + .merge_batch(&values.arguments, group_indices, total_num_groups) } fn evaluate_final(&mut self, emit_to: EmitTo) -> Result { @@ -223,6 +237,15 @@ impl HashAggregateAccumulator { self.accumulator.supports_convert_to_state() } + fn convert_to_state( + &mut self, + values: &EvaluatedHashAggregateAccumulator, + ) -> Result> { + let opt_filter = values.filter.as_ref().map(|filter| filter.as_boolean()); + self.accumulator + .convert_to_state(&values.arguments, opt_filter) + } + fn null_arguments(&self, input_schema: &SchemaRef) -> Result> { self.arguments .iter() @@ -272,6 +295,7 @@ impl AggregateHashTable { .map(|((agg_expr, arguments), filter)| { let accumulator = create_group_accumulator(agg_expr)?; Ok(HashAggregateAccumulator::new( + Arc::clone(agg_expr), arguments, filter, accumulator, @@ -342,6 +366,7 @@ impl AggregateHashTable { } } + /// How many distinct groups has been accumulated now. pub(super) fn building_group_count(&self) -> usize { self.state.building().group_values.len() } @@ -410,27 +435,49 @@ impl AggregateHashTable { output_schema: SchemaRef, batch_size: usize, ) -> Result { - let table = Self::new_with_filters( + Self::new_with_filters( agg, partition, output_schema, batch_size, agg.filter_expr.iter().cloned().collect(), - )?; + ) + } - if table - .state + pub(super) fn can_skip_aggregation(&self) -> bool { + self.state .building() .accumulators .iter() .all(|acc| acc.supports_convert_to_state()) - { - let _skipped_aggregation_rows = MetricBuilder::new(&agg.metrics) - .with_category(MetricCategory::Rows) - .counter("skipped_aggregation_rows", partition); - } + } + + /// In skip-partial-aggregation optimization, when a decision has made to skip + /// partial stage, build a typed hash table only for aggregation state conversion + /// row-by-row. + pub(super) fn partial_skip_table(&self) -> Result> { + let state = self.state.building(); + let group_schema = state.group_by.group_schema(&self.input_schema)?; + let group_values = new_group_values(group_schema, &GroupOrdering::None)?; + let accumulators = state + .accumulators + .iter() + .map(HashAggregateAccumulator::empty_like) + .collect::>>()?; - Ok(table) + Ok(AggregateHashTable { + group_by_metrics: self.group_by_metrics.clone(), + input_schema: Arc::clone(&self.input_schema), + output_schema: Arc::clone(&self.output_schema), + batch_size: self.batch_size, + state: AggregateHashTableState::Building(BuildingHashTableState { + group_by: Arc::clone(&state.group_by), + group_values, + batch_group_indices: Default::default(), + accumulators, + }), + _mode: PhantomData, + }) } pub(super) fn aggregate_batch(&mut self, batch: &RecordBatch) -> Result<()> { @@ -551,6 +598,40 @@ impl AggregateHashTable { } } +impl AggregateHashTable { + pub(super) fn convert_batch_to_state( + &mut self, + batch: &RecordBatch, + ) -> Result { + let evaluated_batch = self.evaluate_batch(batch)?; + + assert_eq_or_internal_err!( + evaluated_batch.grouping_set_args.len(), + 1, + "group_values expected to have single element" + ); + let mut output = evaluated_batch + .grouping_set_args + .into_iter() + .next() + .unwrap_or_default(); + + let state = self.state.building_mut(); + for (acc, values) in state + .accumulators + .iter_mut() + .zip(evaluated_batch.accumulator_args.iter()) + { + output.extend(acc.convert_to_state(values)?); + } + + Ok(RecordBatch::try_new( + Arc::clone(&self.output_schema), + output, + )?) + } +} + impl AggregateHashTable { pub(super) fn new( agg: &AggregateExec, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 940bdd41a88e4..54e44aa86d66c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -77,6 +77,7 @@ mod hash_table; mod no_grouping; pub mod order; mod row_hash; +mod skip_partial; mod topk; mod topk_stream; @@ -3988,6 +3989,8 @@ mod tests { Ok(()) } + // Migrated to PartialHashAggregateStream coverage below; + // kept here for the legacy GroupedHashAggregateStream implementation. #[tokio::test] async fn test_skip_aggregation_after_first_batch() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -4070,6 +4073,8 @@ mod tests { Ok(()) } + // Migrated to PartialHashAggregateStream coverage below; + // kept here for the legacy GroupedHashAggregateStream implementation. #[tokio::test] async fn test_skip_aggregation_after_threshold() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -4161,6 +4166,189 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_partial_hash_stream_skip_aggregation_after_first_batch() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, true), + Field::new("val", DataType::Int32, true), + ])); + + let group_by = + PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); + + let aggr_expr = vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("COUNT(val)")) + .build() + .map(Arc::new)?, + ]; + + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + ]; + + let input = + TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + schema, + )?); + + let session_config = SessionConfig::default() + .set_bool("datafusion.execution.enable_migration_aggregate", true) + .set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &ScalarValue::Int64(Some(2)), + ) + .set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &ScalarValue::Float64(Some(0.1)), + ); + + let ctx = Arc::new(TaskContext::default().with_session_config(session_config)); + let output = collect(aggregate_exec.execute(0, Arc::clone(&ctx))?).await?; + + allow_duplicates! { + assert_snapshot!(batches_to_sort_string(&output), @r" + +-----+-------------------+ + | key | COUNT(val)[count] | + +-----+-------------------+ + | 1 | 1 | + | 2 | 1 | + | 2 | 1 | + | 3 | 1 | + | 3 | 1 | + | 4 | 1 | + +-----+-------------------+ + "); + } + + let metrics = aggregate_exec.metrics().unwrap(); + let skipped_rows = metrics + .sum_by_name("skipped_aggregation_rows") + .map(|m| m.as_usize()) + .unwrap_or(0); + assert_eq!(skipped_rows, 3); + + Ok(()) + } + + #[tokio::test] + async fn test_partial_hash_stream_skip_aggregation_after_threshold() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, true), + Field::new("val", DataType::Int32, true), + ])); + + let group_by = + PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); + + let aggr_expr = vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("COUNT(val)")) + .build() + .map(Arc::new)?, + ]; + + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + ]; + + let input = + TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + schema, + )?); + + let session_config = SessionConfig::default() + .set_bool("datafusion.execution.enable_migration_aggregate", true) + .set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &ScalarValue::Int64(Some(5)), + ) + .set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &ScalarValue::Float64(Some(0.1)), + ); + + let ctx = Arc::new(TaskContext::default().with_session_config(session_config)); + let output = collect(aggregate_exec.execute(0, Arc::clone(&ctx))?).await?; + + allow_duplicates! { + assert_snapshot!(batches_to_sort_string(&output), @r" + +-----+-------------------+ + | key | COUNT(val)[count] | + +-----+-------------------+ + | 1 | 1 | + | 2 | 1 | + | 2 | 2 | + | 3 | 1 | + | 3 | 2 | + | 4 | 1 | + | 4 | 1 | + +-----+-------------------+ + "); + } + + let metrics = aggregate_exec.metrics().unwrap(); + let skipped_rows = metrics + .sum_by_name("skipped_aggregation_rows") + .map(|m| m.as_usize()) + .unwrap_or(0); + assert_eq!(skipped_rows, 3); + + Ok(()) + } + /// When `skip_partial_aggregation_probe_ratio_threshold` is set to 1.0, /// the feature must be effectively disabled: even with 100% cardinality /// (every row is a unique group), no rows should be skipped. diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index c3f73976c721a..d46faf9acc14a 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -22,6 +22,7 @@ use std::task::{Context, Poll}; use std::vec; use super::order::GroupOrdering; +use super::skip_partial::SkipAggregationProbe; use super::{AggregateExec, format_human_display}; use crate::aggregates::group_values::{GroupByMetrics, GroupValues, new_group_values}; use crate::aggregates::order::GroupOrderingFull; @@ -118,100 +119,6 @@ struct SpillState { // Metrics related to spilling are managed inside `spill_manager` } -/// Tracks if the aggregate should skip partial aggregations -/// -/// See "partial aggregation" discussion on [`GroupedHashAggregateStream`] -struct SkipAggregationProbe { - // ======================================================================== - // PROPERTIES: - // These fields are initialized at the start and remain constant throughout - // the execution. - // ======================================================================== - /// Aggregation ratio check performed when the number of input rows exceeds - /// this threshold (from `SessionConfig`) - probe_rows_threshold: usize, - /// Maximum ratio of `num_groups` to `input_rows` for continuing aggregation - /// (from `SessionConfig`). If the ratio exceeds this value, aggregation - /// is skipped and input rows are directly converted to output - probe_ratio_threshold: f64, - - // ======================================================================== - // STATES: - // Fields changes during execution. Can be buffer, or state flags that - // influence the execution in parent `GroupedHashAggregateStream` - // ======================================================================== - /// Number of processed input rows (updated during probing) - input_rows: usize, - /// Number of total group values for `input_rows` (updated during probing) - num_groups: usize, - - /// Flag indicating further data aggregation may be skipped (decision made - /// when probing complete) - should_skip: bool, - /// Flag indicating further updates of `SkipAggregationProbe` state won't - /// make any effect (set either while probing or on probing completion) - is_locked: bool, - - // ======================================================================== - // METRICS: - // ======================================================================== - /// Number of rows where state was output without aggregation. - /// - /// * If 0, all input rows were aggregated (should_skip was always false) - /// - /// * if greater than zero, the number of rows which were output directly - /// without aggregation - skipped_aggregation_rows: metrics::Count, -} - -impl SkipAggregationProbe { - fn new( - probe_rows_threshold: usize, - probe_ratio_threshold: f64, - skipped_aggregation_rows: metrics::Count, - ) -> Self { - Self { - input_rows: 0, - num_groups: 0, - probe_rows_threshold, - probe_ratio_threshold, - should_skip: false, - is_locked: false, - skipped_aggregation_rows, - } - } - - /// Updates `SkipAggregationProbe` state: - /// - increments the number of input rows - /// - replaces the number of groups with the new value - /// - on `probe_rows_threshold` exceeded calculates - /// aggregation ratio and sets `should_skip` flag - /// - if `should_skip` is set, locks further state updates - fn update_state(&mut self, input_rows: usize, num_groups: usize) { - if self.is_locked { - return; - } - self.input_rows += input_rows; - self.num_groups = num_groups; - if self.input_rows >= self.probe_rows_threshold { - self.should_skip = self.num_groups as f64 / self.input_rows as f64 - > self.probe_ratio_threshold; - // Set is_locked to true only if we have decided to skip, otherwise we can try to skip - // during processing the next record_batch. - self.is_locked = self.should_skip; - } - } - - fn should_skip(&self) -> bool { - self.should_skip - } - - /// Record the number of rows that were output directly without aggregation - fn record_skipped(&mut self, batch: &RecordBatch) { - self.skipped_aggregation_rows.add(batch.num_rows()); - } -} - /// Controls the behavior when an out-of-memory condition occurs. #[derive(PartialEq, Debug)] enum OutOfMemoryMode { @@ -1016,7 +923,7 @@ impl GroupedHashAggregateStream { // if aggregation is over intermediate states, // use merge - acc.merge_batch(values, group_indices, None, total_num_groups)?; + acc.merge_batch(values, group_indices, total_num_groups)?; } self.group_by_metrics .aggregation_time @@ -1204,17 +1111,31 @@ impl GroupedHashAggregateStream { // Prime each accumulator for the registered group count with no data. // // We build 1-row null arrays for each aggregate argument and pass them - // with an all-false filter. The filter ensures no row is accumulated - // into any group, which keeps every group in its "zero" initial state - // (NULL for SUM/AVG/MIN/MAX, 0 for COUNT). + // with an all-false filter to update_batch. The filter ensures no row + // is accumulated into any group, which keeps every group in its "zero" + // initial state (NULL for SUM/AVG/MIN/MAX, 0 for COUNT). // // Using a 1-row batch rather than 0 rows is required to avoid a fast // path in `NullState::accumulate` that treats "0 nulls in a 0-row // array" as "all groups have been seen", which would cause SUM to // return 0 instead of NULL. // - // Argument types are inferred directly from the expression metadata so - // we never need to construct a full `RecordBatch`. + // This path always runs in a Raw input mode, so `update_batch` (not + // `merge_batch`) is the right entry point: + // + // - `has_grouping_set()` can only be true for the Partial / Single / + // SinglePartitioned modes, whose `input_mode()` is `Raw`. The final + // modes rebuild their group-by via `PhysicalGroupBy::as_final()`, + // which clears `has_grouping_set`, so this method returns early for + // them and never reaches here. + // + // Since every row is filtered out, the actual data content never + // matters. The assert documents and guards the invariant above. + debug_assert_eq!( + self.mode.input_mode(), + AggregateInputMode::Raw, + "init_empty_grouping_sets must only run in a Raw input mode" + ); let total_groups = self.group_values.len(); let null_args: Vec> = self .aggregate_arguments @@ -1230,11 +1151,7 @@ impl GroupedHashAggregateStream { .collect::>>()?; let false_filter = BooleanArray::from(vec![false]); for (acc, args) in self.accumulators.iter_mut().zip(null_args.iter()) { - if self.mode.input_mode() == AggregateInputMode::Raw { - acc.update_batch(args, &[0], Some(&false_filter), total_groups)?; - } else { - acc.merge_batch(args, &[0], Some(&false_filter), total_groups)?; - } + acc.update_batch(args, &[0], Some(&false_filter), total_groups)?; } } @@ -1479,7 +1396,6 @@ impl GroupedHashAggregateStream { mod tests { use super::*; use crate::InputOrderMode; - use crate::execution_plan::ExecutionPlan; use crate::test::TestMemoryExec; use arrow::array::{Int32Array, Int64Array}; use arrow::datatypes::{DataType, Field, Schema}; @@ -1488,6 +1404,8 @@ mod tests { use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::col; + // Migrated to PartialHashAggregateStream coverage in hash_aggregate.rs; + // kept here for the legacy GroupedHashAggregateStream implementation. #[tokio::test] async fn test_double_emission_race_condition_bug() -> Result<()> { // Fix for https://github.com/apache/datafusion/issues/18701 @@ -1594,152 +1512,9 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_skip_aggregation_probe_not_locked_until_skip() -> Result<()> { - // Test that the probe is not locked until we actually decide to skip. - // This allows us to continue evaluating the skip condition across multiple batches. - // - // Scenario: - // - Batch 1: Hits rows threshold but NOT ratio threshold (low cardinality) -> don't skip - // - Batch 2: Now hits ratio threshold (high cardinality) -> skip - // - // Without the fix, the probe would be locked after batch 1, preventing the skip - // decision from being made on batch 2. - - let schema = Arc::new(Schema::new(vec![ - Field::new("group_col", DataType::Int32, false), - Field::new("value_col", DataType::Int32, false), - ])); - - // Configure thresholds: - // - probe_rows_threshold: 100 rows - // - probe_ratio_threshold: 0.8 (80%) - let probe_rows_threshold = 100; - let probe_ratio_threshold = 0.8; - - // Batch 1: 100 rows with only 10 unique groups - // Ratio: 10/100 = 0.1 (10%) < 0.8 -> should NOT skip - // This will hit the rows threshold but not the ratio threshold - let batch1_rows = 100; - let batch1_groups = 10; - let mut group_ids_batch1 = Vec::new(); - for i in 0..batch1_rows { - group_ids_batch1.push((i % batch1_groups) as i32); - } - let values_batch1: Vec = vec![1; batch1_rows]; - - let batch1 = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(group_ids_batch1)), - Arc::new(Int32Array::from(values_batch1)), - ], - )?; - - // Batch 2: 360 rows with 360 unique NEW groups (starting from group 10) - // After batch 2, total: 460 rows, 370 groups - // Ratio: 370/460 ≈ 0.804 (80.4%) > 0.8 -> SHOULD decide to skip - let batch2_rows = 360; - let batch2_groups = 360; - let group_ids_batch2: Vec = (batch1_groups..(batch1_groups + batch2_groups)) - .map(|x| x as i32) - .collect(); - let values_batch2: Vec = vec![1; batch2_rows]; - - let batch2 = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(group_ids_batch2)), - Arc::new(Int32Array::from(values_batch2)), - ], - )?; - - // Batch 3: This batch should be skipped since we decided to skip after batch 2 - // 100 rows with 100 unique groups (continuing from where batch 2 left off) - let batch3_rows = 100; - let batch3_groups = 100; - let batch3_start_group = batch1_groups + batch2_groups; - let group_ids_batch3: Vec = (batch3_start_group - ..(batch3_start_group + batch3_groups)) - .map(|x| x as i32) - .collect(); - let values_batch3: Vec = vec![1; batch3_rows]; - - let batch3 = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(group_ids_batch3)), - Arc::new(Int32Array::from(values_batch3)), - ], - )?; - - let input_partitions = vec![vec![batch1, batch2, batch3]]; - - let runtime = RuntimeEnvBuilder::default().build_arc()?; - let mut task_ctx = TaskContext::default().with_runtime(runtime); - - // Configure skip aggregation settings - let mut session_config = task_ctx.session_config().clone(); - session_config = session_config.set( - "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - &datafusion_common::ScalarValue::UInt64(Some(probe_rows_threshold)), - ); - session_config = session_config.set( - "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - &datafusion_common::ScalarValue::Float64(Some(probe_ratio_threshold)), - ); - task_ctx = task_ctx.with_session_config(session_config); - let task_ctx = Arc::new(task_ctx); - - // Create aggregate: COUNT(*) GROUP BY group_col - let group_expr = vec![(col("group_col", &schema)?, "group_col".to_string())]; - let aggr_expr = vec![Arc::new( - AggregateExprBuilder::new(count_udaf(), vec![col("value_col", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("count_value") - .build()?, - )]; - - let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; - let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); - - // Use Partial mode - let aggregate_exec = AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::new_single(group_expr), - aggr_expr, - vec![None], - exec, - Arc::clone(&schema), - )?; - - // Execute and collect results - let mut stream = - GroupedHashAggregateStream::new(&aggregate_exec, &Arc::clone(&task_ctx), 0)?; - let mut results = Vec::new(); - - while let Some(result) = stream.next().await { - let batch = result?; - results.push(batch); - } - - // Check that skip aggregation actually happened - // The key metric is skipped_aggregation_rows - let metrics = aggregate_exec.metrics().unwrap(); - let skipped_rows = metrics - .sum_by_name("skipped_aggregation_rows") - .map(|m| m.as_usize()) - .unwrap_or(0); - - // We expect batch 3's rows to be skipped (100 rows) - assert_eq!( - skipped_rows, batch3_rows, - "Expected batch 3's rows ({batch3_rows}) to be skipped", - ); - - Ok(()) - } - + // TODO: migrate to PartialHashAggregateStream when it supports + // InputOrderMode::PartiallySorted; kept here for the legacy + // GroupedHashAggregateStream implementation. #[tokio::test] async fn test_emit_early_with_partially_sorted() -> Result<()> { // Reproducer for #20445: EmitEarly with PartiallySorted panics in @@ -1823,25 +1598,4 @@ mod tests { Ok(()) } - - #[test] - fn test_skip_aggregation_probe_equality_does_not_skip() { - // When num_groups / input_rows == probe_ratio_threshold, the `>` boundary - // means we must NOT skip — equality is not sufficient to trigger skip. - let threshold_ratio = 0.5_f64; - let threshold_rows = 10_usize; - let mut probe = SkipAggregationProbe::new( - threshold_rows, - threshold_ratio, - metrics::Count::new(), - ); - - // 10 rows, 5 groups → ratio = 5/10 = 0.5 exactly equals threshold - probe.update_state(10, 5); - - assert!( - !probe.should_skip(), - "ratio == threshold should not trigger skip (boundary is exclusive)" - ); - } } diff --git a/datafusion/physical-plan/src/aggregates/skip_partial.rs b/datafusion/physical-plan/src/aggregates/skip_partial.rs new file mode 100644 index 0000000000000..903235f950f58 --- /dev/null +++ b/datafusion/physical-plan/src/aggregates/skip_partial.rs @@ -0,0 +1,305 @@ +// 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. + +use arrow::record_batch::RecordBatch; + +use crate::metrics; + +/// Tracks if the aggregate should skip partial aggregations +/// +/// See "partial aggregation" discussion on +/// [`crate::aggregates::row_hash::GroupedHashAggregateStream`]. +pub(super) struct SkipAggregationProbe { + // ======================================================================== + // PROPERTIES: + // These fields are initialized at the start and remain constant throughout + // the execution. + // ======================================================================== + /// Aggregation ratio check performed when the number of input rows exceeds + /// this threshold (from `SessionConfig`) + probe_rows_threshold: usize, + /// Maximum ratio of `num_groups` to `input_rows` for continuing aggregation + /// (from `SessionConfig`). If the ratio exceeds this value, aggregation + /// is skipped and input rows are directly converted to output + probe_ratio_threshold: f64, + + // ======================================================================== + // STATES: + // Fields changes during execution. Can be buffer, or state flags that + // influence the execution in parent `GroupedHashAggregateStream` + // ======================================================================== + /// Number of processed input rows (updated during probing) + input_rows: usize, + /// Number of total group values for `input_rows` (updated during probing) + num_groups: usize, + + /// Flag indicating further data aggregation may be skipped (decision made + /// when probing complete) + should_skip: bool, + /// Flag indicating further updates of `SkipAggregationProbe` state won't + /// make any effect (set either while probing or on probing completion) + is_locked: bool, + + // ======================================================================== + // METRICS: + // ======================================================================== + /// Number of rows where state was output without aggregation. + /// + /// * If 0, all input rows were aggregated (should_skip was always false) + /// + /// * if greater than zero, the number of rows which were output directly + /// without aggregation + skipped_aggregation_rows: metrics::Count, +} + +impl SkipAggregationProbe { + pub(super) fn new( + probe_rows_threshold: usize, + probe_ratio_threshold: f64, + skipped_aggregation_rows: metrics::Count, + ) -> Self { + Self { + input_rows: 0, + num_groups: 0, + probe_rows_threshold, + probe_ratio_threshold, + should_skip: false, + is_locked: false, + skipped_aggregation_rows, + } + } + + /// Updates `SkipAggregationProbe` state: + /// - increments the number of input rows + /// - replaces the number of groups with the new value + /// - on `probe_rows_threshold` exceeded calculates + /// aggregation ratio and sets `should_skip` flag + /// - if `should_skip` is set, locks further state updates + pub(super) fn update_state(&mut self, input_rows: usize, num_groups: usize) { + if self.is_locked { + return; + } + self.input_rows += input_rows; + self.num_groups = num_groups; + if self.input_rows >= self.probe_rows_threshold { + self.should_skip = self.num_groups as f64 / self.input_rows as f64 + > self.probe_ratio_threshold; + // Set is_locked to true only if we have decided to skip, otherwise we can try to skip + // during processing the next record_batch. + self.is_locked = self.should_skip; + } + } + + pub(super) fn should_skip(&self) -> bool { + self.should_skip + } + + /// Record the number of rows that were output directly without aggregation + pub(super) fn record_skipped(&mut self, batch: &RecordBatch) { + self.skipped_aggregation_rows.add(batch.num_rows()); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::aggregates::row_hash::GroupedHashAggregateStream; + use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; + use crate::execution_plan::ExecutionPlan; + use crate::test::TestMemoryExec; + + use std::sync::Arc; + + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::Result; + use datafusion_execution::TaskContext; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_functions_aggregate::count::count_udaf; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; + use datafusion_physical_expr::expressions::col; + use futures::StreamExt; + + // Migrated to PartialHashAggregateStream coverage in hash_aggregate.rs; + // kept here for the legacy GroupedHashAggregateStream implementation. + #[tokio::test] + async fn test_skip_aggregation_probe_not_locked_until_skip() -> Result<()> { + // Test that the probe is not locked until we actually decide to skip. + // This allows us to continue evaluating the skip condition across multiple batches. + // + // Scenario: + // - Batch 1: Hits rows threshold but NOT ratio threshold (low cardinality) -> don't skip + // - Batch 2: Now hits ratio threshold (high cardinality) -> skip + // + // Without the fix, the probe would be locked after batch 1, preventing the skip + // decision from being made on batch 2. + + let schema = Arc::new(Schema::new(vec![ + Field::new("group_col", DataType::Int32, false), + Field::new("value_col", DataType::Int32, false), + ])); + + // Configure thresholds: + // - probe_rows_threshold: 100 rows + // - probe_ratio_threshold: 0.8 (80%) + let probe_rows_threshold = 100; + let probe_ratio_threshold = 0.8; + + // Batch 1: 100 rows with only 10 unique groups + // Ratio: 10/100 = 0.1 (10%) < 0.8 -> should NOT skip + // This will hit the rows threshold but not the ratio threshold + let batch1_rows = 100; + let batch1_groups = 10; + let mut group_ids_batch1 = Vec::new(); + for i in 0..batch1_rows { + group_ids_batch1.push((i % batch1_groups) as i32); + } + let values_batch1: Vec = vec![1; batch1_rows]; + + let batch1 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids_batch1)), + Arc::new(Int32Array::from(values_batch1)), + ], + )?; + + // Batch 2: 360 rows with 360 unique NEW groups (starting from group 10) + // After batch 2, total: 460 rows, 370 groups + // Ratio: 370/460 is about 0.804 (80.4%) > 0.8 -> SHOULD decide to skip + let batch2_rows = 360; + let batch2_groups = 360; + let group_ids_batch2: Vec = (batch1_groups..(batch1_groups + batch2_groups)) + .map(|x| x as i32) + .collect(); + let values_batch2: Vec = vec![1; batch2_rows]; + + let batch2 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids_batch2)), + Arc::new(Int32Array::from(values_batch2)), + ], + )?; + + // Batch 3: This batch should be skipped since we decided to skip after batch 2 + // 100 rows with 100 unique groups (continuing from where batch 2 left off) + let batch3_rows = 100; + let batch3_groups = 100; + let batch3_start_group = batch1_groups + batch2_groups; + let group_ids_batch3: Vec = (batch3_start_group + ..(batch3_start_group + batch3_groups)) + .map(|x| x as i32) + .collect(); + let values_batch3: Vec = vec![1; batch3_rows]; + + let batch3 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(group_ids_batch3)), + Arc::new(Int32Array::from(values_batch3)), + ], + )?; + + let input_partitions = vec![vec![batch1, batch2, batch3]]; + + let runtime = RuntimeEnvBuilder::default().build_arc()?; + let mut task_ctx = TaskContext::default().with_runtime(runtime); + + // Configure skip aggregation settings + let mut session_config = task_ctx.session_config().clone(); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &datafusion_common::ScalarValue::UInt64(Some(probe_rows_threshold)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &datafusion_common::ScalarValue::Float64(Some(probe_ratio_threshold)), + ); + task_ctx = task_ctx.with_session_config(session_config); + let task_ctx = Arc::new(task_ctx); + + // Create aggregate: COUNT(*) GROUP BY group_col + let group_expr = vec![(col("group_col", &schema)?, "group_col".to_string())]; + let aggr_expr = vec![Arc::new( + AggregateExprBuilder::new(count_udaf(), vec![col("value_col", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("count_value") + .build()?, + )]; + + let exec = TestMemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; + let exec = Arc::new(TestMemoryExec::update_cache(&Arc::new(exec))); + + // Use Partial mode + let aggregate_exec = AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new_single(group_expr), + aggr_expr, + vec![None], + exec, + Arc::clone(&schema), + )?; + + // Execute and collect results + let mut stream = + GroupedHashAggregateStream::new(&aggregate_exec, &Arc::clone(&task_ctx), 0)?; + let mut results = Vec::new(); + + while let Some(result) = stream.next().await { + let batch = result?; + results.push(batch); + } + + // Check that skip aggregation actually happened. + // The key metric is skipped_aggregation_rows. + let metrics = aggregate_exec.metrics().unwrap(); + let skipped_rows = metrics + .sum_by_name("skipped_aggregation_rows") + .map(|m| m.as_usize()) + .unwrap_or(0); + + // We expect batch 3's rows to be skipped (100 rows) + assert_eq!( + skipped_rows, batch3_rows, + "Expected batch 3's rows ({batch3_rows}) to be skipped", + ); + + Ok(()) + } + + #[test] + fn test_skip_aggregation_probe_equality_does_not_skip() { + // When num_groups / input_rows == probe_ratio_threshold, the `>` boundary + // means we must NOT skip: equality is not sufficient to trigger skip. + let threshold_ratio = 0.5_f64; + let threshold_rows = 10_usize; + let mut probe = SkipAggregationProbe::new( + threshold_rows, + threshold_ratio, + metrics::Count::new(), + ); + + // 10 rows, 5 groups: ratio = 5/10 = 0.5 exactly equals threshold + probe.update_state(10, 5); + + assert!( + !probe.should_skip(), + "ratio == threshold should not trigger skip (boundary is exclusive)" + ); + } +} diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 3774a300209d0..aa624e050c9d2 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -52,7 +52,6 @@ use crate::projection::{ try_pushdown_through_join, }; use crate::repartition::REPARTITION_RANDOM_STATE; -use crate::spill::get_record_batch_memory_size; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, Statistics, @@ -72,7 +71,7 @@ use arrow::record_batch::RecordBatch; use arrow::util::bit_util; use arrow_schema::{DataType, Schema}; use datafusion_common::config::ConfigOptions; -use datafusion_common::utils::memory::estimate_memory_size; +use datafusion_common::utils::memory::{RecordBatchMemoryCounter, estimate_memory_size}; use datafusion_common::{ JoinSide, JoinType, NullEquality, Result, assert_or_internal_err, internal_err, plan_err, project_schema, @@ -845,6 +844,14 @@ impl HashJoinExec { return false; } + // Bounds and membership filters derived from the build side do not + // account for null-equal matching: a probe-side NULL key evaluates + // such predicates to NULL and would be pruned, even though it can + // match a build-side NULL when nulls compare equal. + if self.null_equality == NullEquality::NullEqualsNull { + return false; + } + // `preserve_file_partitions` can report Hash partitioning for Hive-style // file groups, but those partitions are not actually hash-distributed. // Partitioned dynamic filters rely on hash routing, so disable them in @@ -1141,6 +1148,8 @@ impl DisplayAs for HashJoinExec { let display_fetch = self .fetch .map_or_else(String::new, |f| format!(", fetch={f}")); + let display_null_aware = + if self.null_aware { ", null_aware" } else { "" }; let on = self .on .iter() @@ -1149,7 +1158,7 @@ impl DisplayAs for HashJoinExec { .join(", "); write!( f, - "HashJoinExec: mode={:?}, join_type={:?}, on=[{}]{}{}{}{}", + "HashJoinExec: mode={:?}, join_type={:?}, on=[{}]{}{}{}{}{}", self.mode, self.join_type, on, @@ -1157,6 +1166,7 @@ impl DisplayAs for HashJoinExec { display_projections, display_null_equality, display_fetch, + display_null_aware, ) } DisplayFormatType::TreeRender => { @@ -1179,6 +1189,10 @@ impl DisplayAs for HashJoinExec { writeln!(f, "NullsEqual: true")?; } + if self.null_aware { + writeln!(f, "null_aware")?; + } + if let Some(filter) = self.filter.as_ref() { writeln!(f, "filter={filter}")?; } @@ -1817,6 +1831,10 @@ struct BuildSideState { metrics: BuildProbeJoinMetrics, reservation: MemoryReservation, bounds_accumulators: Option>, + /// Counts the memory of `batches` for `reservation`. Batches can share + /// underlying buffers (e.g. when the input emits zero-copy slices of one + /// larger batch), so each buffer must be reserved only once. + memory_counter: RecordBatchMemoryCounter, } impl BuildSideState { @@ -1833,6 +1851,7 @@ impl BuildSideState { num_rows: 0, metrics, reservation, + memory_counter: RecordBatchMemoryCounter::new(), bounds_accumulators: should_compute_dynamic_filters .then(|| { on_left @@ -1923,7 +1942,7 @@ async fn collect_left_input( } // Decide if we spill or not - let batch_size = get_record_batch_memory_size(&batch); + let batch_size = state.memory_counter.count_batch(&batch); // Reserve memory for incoming batch state.reservation.try_grow(batch_size)?; // Update metrics @@ -1945,6 +1964,7 @@ async fn collect_left_input( metrics, mut reservation, bounds_accumulators, + memory_counter: _, } = state; // Compute bounds @@ -5369,6 +5389,61 @@ mod tests { Ok(()) } + #[tokio::test] + async fn build_side_sliced_batches_memory_accounting() -> Result<()> { + // The build side emits zero-copy slices of one large batch, as e.g. an + // aggregate emitting its output in batch_size chunks does. The buffers + // shared by the slices must be reserved once in total, not once per + // slice: per-slice accounting reserves number_of_slices x parent size + // and aborts queries that fit in memory with room to spare. + let n = 4096; + let v: Vec = (0..n).collect(); + let parent = build_table_i32(("a1", &v), ("b1", &v), ("c1", &v)); + let slices: Vec = + (0..16).map(|i| parent.slice(i * 256, 256)).collect(); + let left = + TestMemoryExec::try_new_exec(&[slices], parent.schema(), None).unwrap(); + + let right_batch = build_table_i32( + ("a2", &vec![10, 11]), + ("b2", &vec![0, 1]), + ("c2", &vec![14, 15]), + ); + let right = TestMemoryExec::try_new_exec( + &[vec![right_batch.clone()]], + right_batch.schema(), + None, + ) + .unwrap(); + let on = vec![( + Arc::new(Column::new_with_schema("b1", &parent.schema())?) as _, + Arc::new(Column::new_with_schema("b2", &right_batch.schema())?) as _, + )]; + + // Enough for the parent batch (~48KB) plus the join hash table, but far + // below the ~768KB that per-slice accounting would reserve + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(400_000, 1.0) + .build_arc()?; + let task_ctx = TaskContext::default().with_runtime(runtime); + let task_ctx = Arc::new(task_ctx); + + let join = join( + left, + right, + on, + &JoinType::Inner, + NullEquality::NullEqualsNothing, + )?; + + let stream = join.execute(0, task_ctx)?; + let batches = common::collect(stream).await?; + let num_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); + assert_eq!(num_rows, 2); + + Ok(()) + } + #[tokio::test] async fn partitioned_join_overallocation() -> Result<()> { // Prepare partitioned inputs for HashJoinExec @@ -6350,6 +6425,35 @@ mod tests { Ok(()) } + #[test] + fn test_dynamic_filter_pushdown_rejects_null_equal_join() -> Result<()> { + let (_, _, on) = build_schema_and_on()?; + let left = build_table(("a1", &vec![1]), ("b1", &vec![1]), ("c1", &vec![1])); + let right = build_table(("a2", &vec![1]), ("b1", &vec![1]), ("c2", &vec![1])); + + let mut session_config = SessionConfig::default(); + session_config + .options_mut() + .optimizer + .enable_join_dynamic_filter_pushdown = true; + + let join = HashJoinExec::try_new( + left, + right, + on, + None, + &JoinType::RightSemi, + None, + PartitionMode::CollectLeft, + NullEquality::NullEqualsNull, + false, + )?; + + assert!(!join.allow_join_dynamic_filter_pushdown(session_config.options())); + + Ok(()) + } + #[test] fn test_with_dynamic_filter_rejects_invalid_columns() -> Result<()> { let (_, _, on) = build_schema_and_on()?; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 0bd053a9db12c..a4cea2c0ccc44 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -874,6 +874,16 @@ enum NLJState { FetchingRight, ProbeRight, EmitRightUnmatched, + /// Entered exactly once per left chunk, when the probe (right) side is + /// exhausted and probing for the current chunk is finished. This state + /// owns the single [`JoinLeftData::report_probe_completed`] call that + /// decrements the shared probe-threads counter, and records in + /// `is_unmatched_left_emitter` whether this stream is the one responsible + /// for emitting unmatched-left rows. Splitting this decision out of + /// `EmitLeftUnmatched` makes "decrement exactly once" a structural + /// property of the state graph, so the (re-enterable) emit state no longer + /// has to guard against decrementing twice. + ProbeEnd, EmitLeftUnmatched, /// Emit unmatched right rows using the global bitmap accumulated across /// all left chunks. Only used in memory-limited mode for join types that @@ -1065,16 +1075,17 @@ pub(crate) struct NestedLoopJoinStream { /// Memory-limited spill fallback state. See [`SpillState`] for details. spill_state: SpillState, - /// Whether this stream has already reported probe completion for the current - /// left chunk via [`JoinLeftData::report_probe_completed`]. The shared - /// probe-threads counter must be decremented exactly once per probe stream; - /// without this guard a stream that yields a ready batch while finishing the - /// `EmitLeftUnmatched` state (and is then re-polled with `left_emit_idx` - /// still 0) would decrement the counter twice, driving it to zero - /// prematurely and causing a sibling partition to emit unmatched-left rows - /// before all partitions finished probing (spurious NULL-padded rows). - /// Reset to `false` when starting a new left chunk in memory-limited mode. - probe_completed_reported: bool, + /// Whether this stream is the one responsible for emitting unmatched-left + /// rows for the current left chunk. Set in the [`NLJState::ProbeEnd`] state, + /// which is entered exactly once per chunk and owns the single + /// [`JoinLeftData::report_probe_completed`] call: the stream that drives the + /// shared probe-threads counter to zero (the last to finish probing) becomes + /// the emitter. Because the decrement happens once in `ProbeEnd` rather than + /// in the re-enterable `EmitLeftUnmatched` state, the counter can never be + /// decremented twice, so it cannot reach zero before all partitions finish + /// probing (which would otherwise let a partition emit spurious NULL-padded + /// unmatched-left rows early). + is_unmatched_left_emitter: bool, } pub(crate) struct NestedLoopJoinMetrics { @@ -1118,7 +1129,7 @@ impl Stream for NestedLoopJoinStream { /// BufferingLeft → FetchingRight /// /// FetchingRight → ProbeRight (if right batch available) - /// FetchingRight → EmitLeftUnmatched (if right exhausted) + /// FetchingRight → ProbeEnd (if right exhausted) /// /// ProbeRight → ProbeRight (next left row or after yielding output) /// ProbeRight → EmitRightUnmatched (for special join types like right join) @@ -1126,6 +1137,9 @@ impl Stream for NestedLoopJoinStream { /// /// EmitRightUnmatched → FetchingRight /// + /// ProbeEnd → EmitLeftUnmatched (records whether this stream is the + /// unmatched-left emitter, then always continues to EmitLeftUnmatched) + /// /// EmitLeftUnmatched → EmitLeftUnmatched (only process 1 chunk for each /// iteration) /// EmitLeftUnmatched → Done (if finished) @@ -1161,8 +1175,8 @@ impl Stream for NestedLoopJoinStream { // 1. --> ProbeRight // Start processing the join for the newly fetched right // batch. - // 2. --> EmitLeftUnmatched: When the right side input is exhausted, (maybe) emit - // unmatched left side rows. + // 2. --> ProbeEnd: When the right side input is exhausted, + // probing for the current left chunk is finished. // // After fetching a new batch from the right side, it will // process all rows from the buffered left data: @@ -1176,9 +1190,10 @@ impl Stream for NestedLoopJoinStream { // at once in memory. // // So after the right side input is exhausted, the join phase - // for the current buffered left data is finished. We can go to - // the next `EmitLeftUnmatched` phase to check if there is any - // special handling (e.g., in cases like left join). + // for the current buffered left data is finished. We go to the + // `ProbeEnd` state, which records probe completion before the + // `EmitLeftUnmatched` phase checks if there is any special + // handling (e.g., in cases like left join). NLJState::FetchingRight => { debug!("[NLJState] Entering: {:?}", self.state); // stop on drop @@ -1241,6 +1256,28 @@ impl Stream for NestedLoopJoinStream { } } + // NLJState transitions: + // 1. --> EmitLeftUnmatched + // Probing for the current left chunk is finished. Report + // probe completion exactly once (decrementing the shared + // probe-threads counter) and record whether this stream is + // the unmatched-left emitter, then always advance to + // `EmitLeftUnmatched`. + NLJState::ProbeEnd => { + debug!("[NLJState] Entering: {:?}", self.state); + + // stop on drop + let join_metric = self.metrics.join_metrics.join_time.clone(); + let _join_timer = join_metric.timer(); + + match self.handle_probe_end() { + ControlFlow::Continue(()) => continue, + ControlFlow::Break(poll) => { + return self.metrics.join_metrics.baseline.record_poll(poll); + } + } + } + // NLJState transitions: // 1. --> EmitLeftUnmatched(1) // If we have already buffered enough output to yield, it @@ -1348,7 +1385,7 @@ impl NestedLoopJoinStream { handled_empty_output: false, should_track_unmatched_right: need_produce_right_in_final(join_type), spill_state, - probe_completed_reported: false, + is_unmatched_left_emitter: false, } } @@ -1724,7 +1761,10 @@ impl NestedLoopJoinStream { } Some(Err(e)) => ControlFlow::Break(Poll::Ready(Some(Err(e)))), None => { - self.state = NLJState::EmitLeftUnmatched; + // Right side exhausted: probing for the current left chunk + // is finished. `ProbeEnd` reports probe completion before + // emitting unmatched-left rows. + self.state = NLJState::ProbeEnd; ControlFlow::Continue(()) } }, @@ -1837,6 +1877,34 @@ impl NestedLoopJoinStream { } } + /// Handle ProbeEnd state - record probe completion for the current chunk. + /// + /// Entered exactly once per left chunk, when the right side is exhausted. + /// This is the single place that decrements the shared probe-threads counter + /// via [`JoinLeftData::report_probe_completed`]: the stream that drives the + /// counter to zero (the last to finish probing) is the one responsible for + /// emitting unmatched-left rows, recorded in `is_unmatched_left_emitter`. + /// + /// Owning the decrement here — rather than in the re-enterable + /// `EmitLeftUnmatched` state — makes "decrement exactly once per stream" a + /// structural property of the state graph, so the counter cannot reach zero + /// before all partitions finish probing (which would let a partition emit + /// spurious NULL-padded unmatched-left rows early). + /// + /// Always transitions to `EmitLeftUnmatched`. + fn handle_probe_end(&mut self) -> ControlFlow>>> { + // Decrement the shared counter exactly once for this stream/chunk. The + // last stream to finish probing (the one that drives the counter to + // zero) becomes the unmatched-left emitter. + let is_emitter = match self.get_left_data() { + Ok(left_data) => left_data.report_probe_completed(), + Err(e) => return ControlFlow::Break(Poll::Ready(Some(Err(e)))), + }; + self.is_unmatched_left_emitter = is_emitter; + self.state = NLJState::EmitLeftUnmatched; + ControlFlow::Continue(()) + } + /// Handle EmitLeftUnmatched state - emit unmatched left rows. /// /// In memory-limited mode, after processing all unmatched rows for the @@ -1876,9 +1944,9 @@ impl NestedLoopJoinStream { self.left_probe_idx = 0; self.left_emit_idx = 0; // Each memory-limited chunk gets a fresh per-chunk - // `JoinLeftData`/counter, so allow this stream to report - // completion again for the next chunk. - self.probe_completed_reported = false; + // `JoinLeftData`/counter; `is_unmatched_left_emitter` is + // recomputed when `ProbeEnd` is re-entered for the next + // chunk, so it does not need to be reset here. self.state = NLJState::BufferingLeft; } else if self.is_memory_limited() && self.should_track_unmatched_right @@ -2357,9 +2425,7 @@ impl NestedLoopJoinStream { /// true -> continue in the same EmitLeftUnmatched state /// false -> next state (Done) fn process_left_unmatched(&mut self) -> Result { - // Clone the shared `Arc` so the immutable borrow of `self` - // ends here and we can update `self.probe_completed_reported` below. - let left_data = Arc::clone(self.get_left_data()?); + let left_data = self.get_left_data()?; let left_batch = left_data.batch(); // ======== @@ -2368,29 +2434,14 @@ impl NestedLoopJoinStream { // Early return if join type can't have unmatched rows let join_type_no_produce_left = !need_produce_result_in_final(self.join_type); - // Early return if another thread is already processing unmatched rows. - // - // The shared probe-threads counter must be decremented exactly once per - // probe stream. This function can be re-entered with `left_emit_idx` - // still 0 (e.g. when a ready batch was flushed via an early return in - // `handle_emit_left_unmatched` before the state advanced), so guard the - // decrement with `probe_completed_reported` instead of relying solely on - // `left_emit_idx == 0`. Decrementing twice would drive the counter to - // zero prematurely and let a partition emit unmatched-left rows before - // all partitions finished probing, producing spurious NULL-padded rows. - let handled_by_other_partition = if self.probe_completed_reported { - // Already counted this stream's completion; if we're the designated - // emitter we have `left_emit_idx > 0` (or are mid-emit) and continue, - // otherwise another partition is handling emission. - self.left_emit_idx == 0 - } else { - self.probe_completed_reported = true; - self.left_emit_idx == 0 && !left_data.report_probe_completed() - }; // Stop processing unmatched rows, the caller will go to the next state let finished = self.left_emit_idx >= left_batch.num_rows(); - if join_type_no_produce_left || handled_by_other_partition || finished { + // `ProbeEnd` already recorded whether this stream emits unmatched-left + // rows. Every probe partition passes through this state, but only the + // one that finished probing last is the emitter, so this flag is false + // for the others. + if join_type_no_produce_left || !self.is_unmatched_left_emitter || finished { return Ok(false); } @@ -2402,7 +2453,7 @@ impl NestedLoopJoinStream { let end_idx = std::cmp::min(start_idx + self.batch_size, left_batch.num_rows()); if let Some(batch) = - self.process_left_unmatched_range(&left_data, start_idx, end_idx)? + self.process_left_unmatched_range(left_data, start_idx, end_idx)? { self.output_buffer.push_batch(batch)?; } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/bitwise_stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/bitwise_stream.rs index ad7312426bd18..99aef6ed82a36 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/bitwise_stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/bitwise_stream.rs @@ -119,8 +119,6 @@ //! factor than the pair-materialization approach. use std::cmp::Ordering; -use std::fs::File; -use std::io::BufReader; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -134,7 +132,6 @@ use crate::{EmptyRecordBatchStream, RecordBatchStream}; use arrow::array::{Array, ArrayRef, BooleanArray, BooleanBufferBuilder, RecordBatch}; use arrow::compute::{BatchCoalescer, SortOptions, filter_record_batch, not}; use arrow::datatypes::SchemaRef; -use arrow::ipc::reader::StreamReader; use arrow::util::bit_chunk_iterator::UnalignedBitChunk; use arrow::util::bit_util::apply_bitwise_binary_op; use datafusion_common::{ @@ -259,6 +256,13 @@ pub(crate) struct BitwiseSortMergeJoinStream { inner_key_buffer: Vec, inner_key_spill: Option, + // Track the active spill_stream + spill_stream: Option, + // Whether the active spill stream has produced any batches yet. + spill_stream_has_data: bool, + // Prevents wiping out the buffer if we yield while evaluating the filter + inner_group_buffered: bool, + // True when buffer_inner_key_group returned Pending after partially // filling inner_key_buffer. On re-entry, buffer_inner_key_group // must skip clear() and resume from poll_next_inner_batch (the @@ -371,6 +375,9 @@ impl BitwiseSortMergeJoinStream { matched: BooleanBufferBuilder::new(0), inner_key_buffer: vec![], inner_key_spill: None, + spill_stream: None, + spill_stream_has_data: false, + inner_group_buffered: false, buffering_inner_pending: false, pending_boundary: None, on_outer, @@ -468,6 +475,9 @@ impl BitwiseSortMergeJoinStream { fn clear_inner_key_group(&mut self) { self.inner_key_buffer.clear(); self.inner_key_spill = None; + self.spill_stream = None; + self.spill_stream_has_data = false; + self.inner_group_buffered = false; self.inner_buffer_size = 0; } @@ -749,7 +759,10 @@ impl BitwiseSortMergeJoinStream { /// Process a key match with a filter. For each inner row in the buffered /// key group, evaluates the filter against the outer key group and ORs /// the results into the matched bitset using u64-chunked bitwise ops. - fn process_key_match_with_filter(&mut self) -> Result<()> { + fn process_key_match_with_filter( + &mut self, + cx: &mut Context<'_>, + ) -> Poll> { self.get_outer_self_cmp()?; let filter = self.filter.as_ref().unwrap(); let outer_batch = self.outer_batch.as_ref().unwrap(); @@ -785,24 +798,47 @@ impl BitwiseSortMergeJoinStream { ) .count_ones(); - // Process spilled inner batches first (read back from disk). - if let Some(spill_file) = &self.inner_key_spill { - let file = BufReader::new(File::open(spill_file.path())?); - let reader = StreamReader::try_new(file, None)?; - for batch_result in reader { - let inner_slice = batch_result?; - matched_count = eval_filter_for_inner_slice( - self.outer_is_left, - filter, - &outer_slice, - &inner_slice, - &mut self.matched, - self.outer_offset, - outer_group_len, - matched_count, - )?; - if matched_count == outer_group_len { - break; + // Process spilled inner batches first asynchronously. + if matched_count < outer_group_len + && (self.inner_key_spill.is_some() || self.spill_stream.is_some()) + { + if self.spill_stream.is_none() + && let Some(spill_file) = &self.inner_key_spill + { + let stream = self + .spill_manager + .read_spill_as_stream(spill_file.clone(), None)?; + self.spill_stream = Some(stream); + } + + while matched_count < outer_group_len { + let stream = self.spill_stream.as_mut().unwrap(); + match ready!(stream.poll_next_unpin(cx)) { + Some(Ok(inner_slice)) => { + self.spill_stream_has_data = true; + matched_count = eval_filter_for_inner_slice( + self.outer_is_left, + filter, + &outer_slice, + &inner_slice, + &mut self.matched, + self.outer_offset, + outer_group_len, + matched_count, + )?; + } + Some(Err(e)) => { + self.spill_stream = None; + self.spill_stream_has_data = false; + return Poll::Ready(Err(e)); + } + None => { + self.spill_stream = None; + if !self.spill_stream_has_data { + return Poll::Ready(internal_err!("Spill file was empty")); + } + break; + } } } } @@ -830,13 +866,17 @@ impl BitwiseSortMergeJoinStream { } self.outer_offset = outer_group_end; - Ok(()) + + self.spill_stream = None; + self.spill_stream_has_data = false; + + Poll::Ready(Ok(())) } /// Continue processing an outer key group that spans multiple outer /// batches. Returns `true` if this outer batch was fully consumed /// by the key group and the caller should load another. - fn resume_boundary(&mut self) -> Result { + fn resume_boundary(&mut self, cx: &mut Context<'_>) -> Poll> { debug_assert!( self.outer_batch.is_some(), "caller must load outer_batch first" @@ -858,7 +898,7 @@ impl BitwiseSortMergeJoinStream { }); self.emit_outer_batch()?; self.outer_batch = None; - return Ok(true); + return Poll::Ready(Ok(true)); } } } @@ -874,7 +914,15 @@ impl BitwiseSortMergeJoinStream { self.null_equality, )?; if same_key { - self.process_key_match_with_filter()?; + match self.process_key_match_with_filter(cx) { + Poll::Ready(Ok(())) => (), + Poll::Ready(Err(e)) => return Poll::Ready(Err(e)), + Poll::Pending => { + self.pending_boundary = + Some(PendingBoundary::Filtered { saved_keys }); + return Poll::Pending; + } + } let num_outer = self.outer_batch.as_ref().unwrap().num_rows(); if self.outer_offset >= num_outer { self.pending_boundary = Some(PendingBoundary::Filtered { @@ -882,14 +930,63 @@ impl BitwiseSortMergeJoinStream { }); self.emit_outer_batch()?; self.outer_batch = None; - return Ok(true); + return Poll::Ready(Ok(true)); } } self.clear_inner_key_group(); } None => {} } - Ok(false) + Poll::Ready(Ok(false)) + } + + /// Helper to process an Equal match across potential outer batch boundaries. + fn process_filtered_match_loop(&mut self, cx: &mut Context<'_>) -> Poll> { + loop { + ready!(self.process_key_match_with_filter(cx))?; + + let outer_batch = self.outer_batch.as_ref().unwrap(); + if self.outer_offset >= outer_batch.num_rows() { + let saved_keys = + slice_keys(&self.outer_key_arrays, outer_batch.num_rows() - 1); + + self.emit_outer_batch()?; + self.pending_boundary = Some(PendingBoundary::Filtered { saved_keys }); + + // Clear stale batch before polling + self.outer_batch = None; + + match ready!(self.poll_next_outer_batch(cx)) { + Err(e) => return Poll::Ready(Err(e)), + Ok(false) => { + self.pending_boundary = None; + break; + } + Ok(true) => { + let Some(PendingBoundary::Filtered { saved_keys }) = + self.pending_boundary.take() + else { + unreachable!() + }; + let same = keys_match( + &saved_keys, + &self.outer_key_arrays, + &self.sort_options, + self.null_equality, + )?; + if same { + continue; + } + break; + } + } + } else { + break; + } + } + + self.clear_inner_key_group(); // This resets inner_group_buffered to false + Poll::Ready(Ok(())) } /// Main loop: drive the merge-scan to produce output batches. @@ -911,14 +1008,21 @@ impl BitwiseSortMergeJoinStream { } return Poll::Ready(Ok(None)); } - Ok(true) => { - if self.resume_boundary()? { - continue; - } - } + Ok(true) => {} // Loaded batch, move on to checks } } + // Handles pausing while fetching a NEW outer batch. + if self.pending_boundary.is_some() && ready!(self.resume_boundary(cx))? { + continue; + } + + // Handles pausing while reading the disk stream mid-batch. + if self.inner_group_buffered { + ready!(self.process_filtered_match_loop(cx))?; + continue; + } + // 2. Ensure we have an inner batch (unless inner is exhausted). // Skip this when resuming a pending boundary — inner was already // advanced past the key group before the boundary loop started. @@ -1043,65 +1147,17 @@ impl BitwiseSortMergeJoinStream { } Ordering::Equal => { if self.filter.is_some() { + debug_assert!(!self.inner_group_buffered); // Buffer inner key group (may span batches) match ready!(self.buffer_inner_key_group(cx)) { Err(e) => return Poll::Ready(Err(e)), - Ok(_inner_exhausted) => {} + Ok(_inner_exhausted) => { + self.inner_group_buffered = true; + } } - // Process outer rows against buffered inner group // (may need to handle outer batch boundary) - loop { - self.process_key_match_with_filter()?; - - let outer_batch = self.outer_batch.as_ref().unwrap(); - if self.outer_offset >= outer_batch.num_rows() { - let saved_keys = slice_keys( - &self.outer_key_arrays, - outer_batch.num_rows() - 1, - ); - - self.emit_outer_batch()?; - debug_assert!( - !self.inner_key_buffer.is_empty() - || self.inner_key_spill.is_some(), - "Filtered pending boundary requires inner key data in buffer or spill" - ); - self.pending_boundary = - Some(PendingBoundary::Filtered { saved_keys }); - - match ready!(self.poll_next_outer_batch(cx)) { - Err(e) => return Poll::Ready(Err(e)), - Ok(false) => { - self.pending_boundary = None; - self.outer_batch = None; - break; - } - Ok(true) => { - let Some(PendingBoundary::Filtered { - saved_keys, - }) = self.pending_boundary.take() - else { - unreachable!() - }; - let same = keys_match( - &saved_keys, - &self.outer_key_arrays, - &self.sort_options, - self.null_equality, - )?; - if same { - continue; - } - break; - } - } - } else { - break; - } - } - - self.clear_inner_key_group(); + ready!(self.process_filtered_match_loop(cx))?; } else { // No filter: advance inner past key group, then // mark all outer rows with this key as matched. diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/materializing_stream.rs b/datafusion/physical-plan/src/joins/sort_merge_join/materializing_stream.rs index 9bcc749c23dce..f1a18aac762f5 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/materializing_stream.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/materializing_stream.rs @@ -23,8 +23,7 @@ use std::cmp::Ordering; use std::collections::{HashMap, VecDeque}; -use std::fs::File; -use std::io::BufReader; +use std::fmt::Debug; use std::mem::size_of; use std::ops::Range; use std::pin::Pin; @@ -47,10 +46,9 @@ use crate::{PhysicalExpr, RecordBatchStream, SendableRecordBatchStream}; use arrow::array::{types::UInt64Type, *}; use arrow::compute::{ self, BatchCoalescer, SortOptions, concat_batches, filter_record_batch, interleave, - take, take_arrays, + take_arrays, }; use arrow::datatypes::SchemaRef; -use arrow::ipc::reader::StreamReader; use datafusion_common::cast::as_uint64_array; use datafusion_common::{JoinType, NullEquality, Result, exec_err, internal_err}; use datafusion_execution::disk_manager::RefCountedTempFile; @@ -58,7 +56,7 @@ use datafusion_execution::memory_pool::MemoryReservation; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_physical_expr_common::physical_expr::PhysicalExprRef; -use futures::{Stream, StreamExt}; +use futures::{Stream, StreamExt, ready}; /// State of SMJ stream #[derive(Debug, PartialEq, Eq)] @@ -382,6 +380,11 @@ pub(super) struct MaterializingSortMergeJoinStream { /// Manages the process of spilling and reading back intermediate data pub spill_manager: SpillManager, + /// Tracks the active stream when loading spilled buffered batches back in memory + pub spill_stream: Option, + /// Tracks the number of batches currently spilled + pub spilled_batch_count: usize, + // ======================================================================== // CACHED COMPARATORS: // Pre-built comparators to avoid per-row type dispatch in hot loops. @@ -599,6 +602,16 @@ impl Stream for MaterializingSortMergeJoinStream { .filter_mask .len(); if accumulated >= self.batch_size { + // Ensure required spilled batches are restored to memory + // before processing, as this path invokes freeze_all(). + let needed = self.get_required_batch_indices( + self.buffered_data.batches.len(), + ); + if let Err(e) = ready!( + self.poll_spilled_batches(cx, &needed) + ) { + return Poll::Ready(Some(Err(e))); + } match self.process_filtered_batches()? { Poll::Ready(Some(batch)) => { return Poll::Ready(Some(Ok(batch))); @@ -684,14 +697,13 @@ impl Stream for MaterializingSortMergeJoinStream { self.state = SortMergeJoinState::Init; } SortMergeJoinState::JoinOutput => { - self.join_partial()?; + // If the batch size limit is reached, restore required spilled batches to memory and freeze. + // Guarding at the top of the loop safely handles re-entry from Poll::Pending. + if self.num_unfrozen_pairs() >= self.batch_size { + let needed = self + .get_required_batch_indices(self.buffered_data.batches.len()); + ready!(self.poll_spilled_batches(cx, &needed))?; - if self.num_unfrozen_pairs() < self.batch_size { - if self.buffered_data.scanning_finished() { - self.buffered_data.scanning_reset(); - self.state = SortMergeJoinState::EmitReadyThenInit; - } - } else { self.freeze_all()?; // Verify metadata alignment before checking if we have batches to output @@ -705,7 +717,6 @@ impl Stream for MaterializingSortMergeJoinStream { } // For non-filtered joins, only output if we have a completed batch - // (opportunistic output when target batch size is reached) if self .joined_record_batches .joined_batches @@ -720,10 +731,26 @@ impl Stream for MaterializingSortMergeJoinStream { .record_output(&self.join_metrics.baseline_metrics()); return Poll::Ready(Some(Ok(record_batch))); } + // Otherwise keep buffering (don't output yet) + continue; + } + + self.join_partial()?; + + if self.num_unfrozen_pairs() < self.batch_size + && self.buffered_data.scanning_finished() + { + self.buffered_data.scanning_reset(); + self.state = SortMergeJoinState::EmitReadyThenInit; } + // Note: If join_partial() reached the batch size, the loop repeats to freeze the data. } SortMergeJoinState::Exhausted => { + let needed = + self.get_required_batch_indices(self.buffered_data.batches.len()); + ready!(self.poll_spilled_batches(cx, &needed))?; + self.freeze_all()?; // Verify metadata alignment before final output @@ -843,6 +870,8 @@ impl MaterializingSortMergeJoinStream { reservation, runtime_env, spill_manager, + spill_stream: None, + spilled_batch_count: 0, streamed_buffered_cmp: None, buffered_equality_cmp: None, streamed_batch_counter: AtomicUsize::new(0), @@ -917,6 +946,84 @@ impl MaterializingSortMergeJoinStream { Poll::Pending } + /// Identifies which buffered batches are needed for the upcoming freeze operation + fn get_required_batch_indices(&self, buffered_freeze_count: usize) -> Vec { + let mut needed = vec![]; + // Avoid scanning if no spilled batches exist + if self.spilled_batch_count == 0 { + return needed; + } + // We need all batches that matched with streamed rows + for chunk in &self.streamed_batch.output_indices { + if let Some(idx) = chunk.buffered_batch_idx { + needed.push(idx); + } + } + + // Full Joins need to emit null-joined rows, so we need batches up to freeze_count + if self.join_type == JoinType::Full { + needed.extend(0..buffered_freeze_count); + } + + needed.sort_unstable(); + needed.dedup(); + needed + } + + /// Asynchronously reads spilled batches back into memory. + /// Only processes the required indices to avoid OOMs. + fn poll_spilled_batches( + &mut self, + cx: &mut Context<'_>, + required_indices: &[usize], + ) -> Poll> { + for &idx in required_indices { + // Guard against indices that might be out of bounds if the queue was cleared + if idx >= self.buffered_data.batches.len() { + continue; + } + + let bb = &mut self.buffered_data.batches[idx]; + + if let BufferedBatchState::Spilled(spill_file) = &bb.batch { + if self.spill_stream.is_none() { + let stream = self + .spill_manager + .read_spill_as_stream(spill_file.clone(), None)?; + self.spill_stream = Some(stream); + } + + match ready!(self.spill_stream.as_mut().unwrap().poll_next_unpin(cx)) { + Some(Ok(batch)) => { + // Transition the batch back to InMemory + bb.batch = BufferedBatchState::InMemory(batch); + self.spilled_batch_count -= 1; + // The batch is back in memory, so we must account for its size. + let newly_allocated = + bb.size_estimation.saturating_sub(bb.reserved_amount); + self.reservation.grow(newly_allocated); + bb.reserved_amount = bb.size_estimation; + + self.join_metrics + .peak_mem_used() + .set_max(self.reservation.size()); + + self.spill_stream = None; + } + Some(Err(e)) => { + self.spill_stream = None; + return Poll::Ready(Err(e)); + } + None => { + self.spill_stream = None; + return Poll::Ready(internal_err!("Spill file was empty")); + } + } + } + } + Poll::Ready(Ok(())) + } + /// Poll next streamed row fn poll_streamed_row(&mut self, cx: &mut Context) -> Poll>> { loop { @@ -931,33 +1038,41 @@ impl MaterializingSortMergeJoinStream { self.streamed_state = StreamedState::Polling; } } - StreamedState::Polling => match self.streamed.poll_next_unpin(cx)? { - Poll::Pending => { - return Poll::Pending; - } - Poll::Ready(None) => { - // Release the streamed input pipeline's resources. - let streamed_schema = self.streamed.schema(); - self.streamed = - Box::pin(EmptyRecordBatchStream::new(streamed_schema)); - self.streamed_state = StreamedState::Exhausted; + StreamedState::Polling => { + let needed = + self.get_required_batch_indices(self.buffered_data.batches.len()); + if let Err(e) = ready!(self.poll_spilled_batches(cx, &needed)) { + return Poll::Ready(Some(Err(e))); } - Poll::Ready(Some(batch)) => { - if batch.num_rows() > 0 { - self.freeze_streamed()?; - self.join_metrics.input_batches().add(1); - self.join_metrics.input_rows().add(batch.num_rows()); - self.streamed_batch = - StreamedBatch::new(batch, &self.on_streamed); - self.rebuild_streamed_buffered_cmp()?; - // Every incoming streaming batch should have its unique id - // Check `JoinedRecordBatches.self.streamed_batch_counter` documentation - self.streamed_batch_counter - .fetch_add(1, std::sync::atomic::Ordering::SeqCst); - self.streamed_state = StreamedState::Ready; + + match self.streamed.poll_next_unpin(cx)? { + Poll::Pending => { + return Poll::Pending; + } + Poll::Ready(None) => { + // Release the streamed input pipeline's resources. + let streamed_schema = self.streamed.schema(); + self.streamed = + Box::pin(EmptyRecordBatchStream::new(streamed_schema)); + self.streamed_state = StreamedState::Exhausted; + } + Poll::Ready(Some(batch)) => { + if batch.num_rows() > 0 { + self.freeze_streamed()?; + self.join_metrics.input_batches().add(1); + self.join_metrics.input_rows().add(batch.num_rows()); + self.streamed_batch = + StreamedBatch::new(batch, &self.on_streamed); + self.rebuild_streamed_buffered_cmp()?; + // Every incoming streaming batch should have its unique id + // Check `JoinedRecordBatches.self.streamed_batch_counter` documentation + self.streamed_batch_counter + .fetch_add(1, std::sync::atomic::Ordering::SeqCst); + self.streamed_state = StreamedState::Ready; + } } } - }, + } StreamedState::Ready => { return Poll::Ready(Some(Ok(()))); } @@ -997,6 +1112,7 @@ impl MaterializingSortMergeJoinStream { .unwrap(); // Operation only return None if no batches are spilled, here we ensure that at least one batch is spilled buffered_batch.batch = BufferedBatchState::Spilled(spill_file); + self.spilled_batch_count += 1; // Join key arrays remain in memory after the batch is // spilled — the comparator needs them for key boundary @@ -1036,12 +1152,25 @@ impl MaterializingSortMergeJoinStream { let head_batch = self.buffered_data.head_batch(); // If the head batch is fully processed, dequeue it and produce output of it. if head_batch.range.end == head_batch.num_rows { + // load the spilled head batch before dequeuing + let needed = self.get_required_batch_indices(1); + if let Err(e) = ready!(self.poll_spilled_batches(cx, &needed)) + { + return Poll::Ready(Some(Err(e))); + } + self.freeze_dequeuing_buffered()?; if let Some(mut buffered_batch) = self.buffered_data.batches.pop_front() { self.produce_buffered_not_matched(&mut buffered_batch)?; self.free_reservation(&buffered_batch); + if matches!( + buffered_batch.batch, + BufferedBatchState::Spilled(_) + ) { + self.spilled_batch_count -= 1; + } head_changed = true; } } else { @@ -1556,18 +1685,6 @@ impl MaterializingSortMergeJoinStream { as_uint64_array(&compute::concat(&refs)?)?.clone() }; - let spill_reservation = self.reservation.new_empty(); - if matches!( - &self.buffered_data.batches[first_batch_idx].batch, - BufferedBatchState::Spilled(_) - ) { - spill_reservation - .grow(self.buffered_data.batches[first_batch_idx].size_estimation); - self.join_metrics - .peak_mem_used() - .set_max(self.reservation.size() + spill_reservation.size()); - } - return fetch_right_columns_by_idxs( &self.buffered_data, first_batch_idx, @@ -1603,29 +1720,20 @@ impl MaterializingSortMergeJoinStream { let num_right_cols = self.buffered_schema.fields().len(); // Read each source batch once (spilled batches require disk I/O). - // Track memory for each spilled batch at the point of deserialization - // so the pool reflects actual usage as it grows. - let spill_reservation = self.reservation.new_empty(); - let mut source_data: Vec> = - Vec::with_capacity(source_batches.len()); - for &idx in &source_batches { - let bb = &self.buffered_data.batches[idx]; - match &bb.batch { - BufferedBatchState::InMemory(batch) => { - source_data.push(Some(batch.clone())); - } - BufferedBatchState::Spilled(spill_file) => { - spill_reservation.grow(bb.size_estimation); - self.join_metrics - .peak_mem_used() - .set_max(self.reservation.size() + spill_reservation.size()); - - let file = BufReader::new(File::open(spill_file.path())?); - let reader = StreamReader::try_new(file, None)?; - source_data.push(reader.into_iter().next().transpose()?); + let source_data_result: Result> = source_batches + .iter() + .map(|&idx| { + let bb = &self.buffered_data.batches[idx]; + match &bb.batch { + BufferedBatchState::InMemory(batch) => Ok(batch.clone()), + BufferedBatchState::Spilled(_) => { + internal_err!("Buffered batch should have been unspilled before fetching columns") + } } - } - } + }) + .collect(); + + let source_data = source_data_result?; let mut right_columns = Vec::with_capacity(num_right_cols); for col_idx in 0..num_right_cols { @@ -1637,14 +1745,7 @@ impl MaterializingSortMergeJoinStream { source_arrays.push(null_array.as_ref()); for data in &source_data { - match data { - Some(batch) => source_arrays.push(batch.column(col_idx).as_ref()), - None => { - return internal_err!( - "Failed to read spilled buffered batch during interleave" - ); - } - } + source_arrays.push(data.column(col_idx).as_ref()); } right_columns.push(interleave(&source_arrays, &interleave_indices)?); } @@ -1838,32 +1939,17 @@ fn fetch_right_columns_from_batch_by_idxs( buffered_indices: &UInt64Array, ) -> Result> { match &buffered_batch.batch { - // In memory batch - // In memory batch BufferedBatchState::InMemory(batch) => { - // When indices form a contiguous range (common in SMJ since the - // buffered side is scanned sequentially), use zero-copy slice. if let Some(range) = is_contiguous_range(buffered_indices) { Ok(batch.slice(range.start, range.len()).columns().to_vec()) } else { Ok(take_arrays(batch.columns(), buffered_indices, None)?) } } - // If the batch was spilled to disk, less likely - BufferedBatchState::Spilled(spill_file) => { - let mut buffered_cols: Vec = - Vec::with_capacity(buffered_indices.len()); - - let file = BufReader::new(File::open(spill_file.path())?); - let reader = StreamReader::try_new(file, None)?; - - for batch in reader { - batch?.columns().iter().for_each(|column| { - buffered_cols.extend(take(column, &buffered_indices, None)) - }); - } - - Ok(buffered_cols) + BufferedBatchState::Spilled(_) => { + internal_err!( + "Buffered batch should have been unspilled before fetching columns" + ) } } } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs index b1fdf3ddabb5a..0347299dd0094 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join/tests.rs @@ -2459,6 +2459,24 @@ async fn overallocation_multi_batch_spill() -> Result<()> { assert!(join.metrics().unwrap().spilled_bytes().unwrap() > 0); assert!(join.metrics().unwrap().spilled_rows().unwrap() > 0); + // For Full joins, get_required_batch_indices extends 0..batches.len(), so + // poll_spilled_batches can restore all spilled batches at once via infallible + // grow(). Verify accounting tracked the transient spike and cleaned up. + let peak_mem = join + .metrics() + .and_then(|m| m.sum_by_name("peak_mem_used")) + .map(|m| m.as_usize()) + .unwrap_or(0); + assert!( + peak_mem > 0, + "peak_mem_used should be > 0 for {join_type:?} batch_size={batch_size}" + ); + assert_eq!( + runtime.memory_pool.reserved(), + 0, + "memory should be fully released after {join_type:?} completes + (batch_size={batch_size}): infallible grow during restore must be balanced" + ); // Run the test with no spill configuration as let task_ctx_no_spill = TaskContext::default().with_session_config(session_config.clone()); diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 9da606dc90db2..8a8bfd204ecb6 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -255,7 +255,9 @@ impl TopK { let array = filtered.into_array(num_rows)?; let mut filter = array.as_boolean().clone(); if !filter.has_true() { - // nothing to filter, so no need to update + // The heap is unchanged, but a fully rejected batch can still prove + // that the shared sort prefix has passed the heap boundary. + self.attempt_early_completion(&batch)?; return Ok(()); } // only update the keys / rows if the filter does not match all rows @@ -1099,20 +1101,15 @@ mod tests { assert_eq!(record_batch_store.batches_size, 0); } - /// This test validates that the `try_finish` method marks the TopK operator as finished - /// when the prefix (on column "a") of the last row in the current batch is strictly greater - /// than the max top‑k row. - /// The full sort expression is defined on both columns ("a", "b"), but the input ordering is only on "a". - #[tokio::test] - async fn test_try_finish_marks_finished_with_prefix() -> Result<()> { - // Create a schema with two columns. + /// Builds an `(a Int32, b Float64)` schema and a `TopK` with full sort + /// `(a ASC, b ASC)`, input prefix `[a]`, `k = 3`, `batch_size = 2`. Used by + /// the prefix-completion tests below to keep their per-scenario logic in focus. + fn build_ab_prefix_topk() -> Result<(Arc, TopK)> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, false), Field::new("b", DataType::Float64, false), ])); - // Create sort expressions. - // Full sort: first by "a", then by "b". let sort_expr_a = PhysicalSortExpr { expr: col("a", schema.as_ref())?, options: SortOptions::default(), @@ -1122,28 +1119,33 @@ mod tests { options: SortOptions::default(), }; - // Input ordering uses only column "a" (a prefix of the full sort). + // Input ordering uses only column "a" (a prefix of the full sort on (a, b)). let prefix = vec![sort_expr_a.clone()]; let full_expr = LexOrdering::from([sort_expr_a, sort_expr_b]); - // Create a dummy runtime environment and metrics. - let runtime = Arc::new(RuntimeEnv::default()); - let metrics = ExecutionPlanMetricsSet::new(); - - // Create a TopK instance with k = 3 and batch_size = 2. - let mut topk = TopK::try_new( + let topk = TopK::try_new( 0, Arc::clone(&schema), prefix, full_expr, 3, 2, - runtime, - &metrics, + Arc::new(RuntimeEnv::default()), + &ExecutionPlanMetricsSet::new(), Arc::new(RwLock::new(TopKDynamicFilters::new(Arc::new( DynamicFilterPhysicalExpr::new(vec![], lit(true)), )))), )?; + Ok((schema, topk)) + } + + /// This test validates that the `try_finish` method marks the TopK operator as finished + /// when the prefix (on column "a") of the last row in the current batch is strictly greater + /// than the max top‑k row. + /// The full sort expression is defined on both columns ("a", "b"), but the input ordering is only on "a". + #[tokio::test] + async fn test_try_finish_marks_finished_with_prefix() -> Result<()> { + let (schema, mut topk) = build_ab_prefix_topk()?; // Create the first batch with two columns: // Column "a": [1, 1, 2], Column "b": [20.0, 15.0, 30.0]. @@ -1196,6 +1198,67 @@ mod tests { Ok(()) } + /// Regression test for #22849: a batch whose rows are entirely rejected by the + /// heap's dynamic filter must still trigger `attempt_early_completion` when its + /// last row's prefix is worse than the heap's worst. + /// + /// Before the fix, the `!filter.has_true()` short-circuit returned without calling + /// `attempt_early_completion`. Because the heap's filter is itself derived from the + /// heap's worst row, a batch from a strictly-worse prefix is exactly the case the + /// filter rejects entirely — i.e. the very signal the early-exit was designed to + /// detect was being silently dropped. + #[tokio::test] + async fn test_try_finish_fires_when_filter_rejects_entire_batch() -> Result<()> { + let (schema, mut topk) = build_ab_prefix_topk()?; + + // Batch 1 fills the heap with (1, 20.0), (1, 15.0), (2, 30.0). + // heap.max becomes (a=2, b=30.0); update_filter tightens the heap filter to + // a < 2 OR (a = 2 AND b < 30.0). + let array_a1: ArrayRef = + Arc::new(Int32Array::from(vec![Some(1), Some(1), Some(2)])); + let array_b1: ArrayRef = Arc::new(Float64Array::from(vec![20.0, 15.0, 30.0])); + let batch1 = RecordBatch::try_new(Arc::clone(&schema), vec![array_a1, array_b1])?; + topk.insert_batch(batch1)?; + assert!( + !topk.finished, + "Expected 'finished' to be false after batch 1 \ + (last row prefix a=2 equals heap.max prefix a=2, not strictly greater)." + ); + + // Batch 2: every row has a=3, so the heap's filter (a < 2 OR (a = 2 AND b < 30)) + // rejects every row. Before the fix, `insert_batch` would short-circuit on + // `!filter.has_true()` and return without checking the prefix; `finished` + // would stay false even though no future batch could improve the heap. + let array_a2: ArrayRef = Arc::new(Int32Array::from(vec![Some(3), Some(3)])); + let array_b2: ArrayRef = Arc::new(Float64Array::from(vec![10.0, 20.0])); + let batch2 = RecordBatch::try_new(Arc::clone(&schema), vec![array_a2, array_b2])?; + topk.insert_batch(batch2)?; + assert!( + topk.finished, + "Expected 'finished' to be true after batch 2 \ + (filter rejected every row, but the batch's last row prefix a=3 \ + is strictly greater than heap.max prefix a=2)." + ); + + // The emitted top-k is unchanged from after batch 1 since none of batch 2's + // rows could improve the heap. + let results: Vec<_> = topk.emit()?.try_collect().await?; + assert_batches_eq!( + &[ + "+---+------+", + "| a | b |", + "+---+------+", + "| 1 | 15.0 |", + "| 1 | 20.0 |", + "| 2 | 30.0 |", + "+---+------+", + ], + &results + ); + + Ok(()) + } + /// This test verifies that the dynamic filter is marked as complete after TopK processing finishes. #[tokio::test] async fn test_topk_marks_filter_complete() -> Result<()> { diff --git a/datafusion/proto-common/gen/Cargo.toml b/datafusion/proto-common/gen/Cargo.toml index f0e60819d42a8..0cbba311b2c3f 100644 --- a/datafusion/proto-common/gen/Cargo.toml +++ b/datafusion/proto-common/gen/Cargo.toml @@ -38,4 +38,4 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.9.0" -prost-build = "=0.14.3" +prost-build = "=0.14.4" diff --git a/datafusion/proto-models/.gitignore b/datafusion/proto-models/.gitignore new file mode 100644 index 0000000000000..662b95f238c24 --- /dev/null +++ b/datafusion/proto-models/.gitignore @@ -0,0 +1,5 @@ +# Files generated by regen.sh +proto/proto_descriptor.bin +src/datafusion.rs +src/datafusion.serde.rs +src/datafusion_common.rs diff --git a/datafusion/proto-models/gen/Cargo.toml b/datafusion/proto-models/gen/Cargo.toml index 8b48dfe70e6c7..9724b63cccf3c 100644 --- a/datafusion/proto-models/gen/Cargo.toml +++ b/datafusion/proto-models/gen/Cargo.toml @@ -38,4 +38,4 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic pbjson-build = "=0.9.0" -prost-build = "=0.14.3" +prost-build = "=0.14.4" diff --git a/datafusion/proto-models/proto/datafusion.proto b/datafusion/proto-models/proto/datafusion.proto index 322395ab3728c..f13494cf43834 100644 --- a/datafusion/proto-models/proto/datafusion.proto +++ b/datafusion/proto-models/proto/datafusion.proto @@ -311,13 +311,63 @@ message DmlNode{ INSERT_OVERWRITE = 4; INSERT_REPLACE = 5; TRUNCATE = 6; + MERGE_INTO = 7; } Type dml_type = 1; LogicalPlanNode input = 2; TableReference table_name = 3; LogicalPlanNode target = 5; + // Populated only when dml_type == MERGE_INTO. + MergeIntoOpNode merge_into = 6; } +// Carries the ON condition and WHEN clauses of a MERGE INTO operation. +message MergeIntoOpNode { + LogicalExprNode on = 1; + repeated MergeIntoClauseNode clauses = 2; +} + +// A single WHEN clause within a MERGE INTO statement. +message MergeIntoClauseNode { + enum Kind { + MATCHED = 0; + NOT_MATCHED = 1; + NOT_MATCHED_BY_TARGET = 2; + NOT_MATCHED_BY_SOURCE = 3; + } + Kind kind = 1; + // Optional `AND ` predicate. Absent when the clause has no predicate. + LogicalExprNode predicate = 2; + MergeIntoActionNode action = 3; +} + +// The action for a single WHEN clause. +message MergeIntoActionNode { + oneof action { + MergeUpdateAction update = 1; + MergeInsertAction insert = 2; + MergeDeleteAction delete = 3; + } +} + +message MergeUpdateAction { + repeated MergeAssignment assignments = 1; +} + +message MergeAssignment { + string column = 1; + LogicalExprNode value = 2; +} + +message MergeInsertAction { + // May be empty (meaning all columns). + repeated string columns = 1; + // One expression per inserted column. + repeated LogicalExprNode values = 2; +} + +message MergeDeleteAction {} + message UnnestNode { LogicalPlanNode input = 1; repeated datafusion_common.Column exec_columns = 2; diff --git a/datafusion/proto-models/src/generated/pbjson.rs b/datafusion/proto-models/src/generated/pbjson.rs index 1eb9de00fb362..2ba5e25054259 100644 --- a/datafusion/proto-models/src/generated/pbjson.rs +++ b/datafusion/proto-models/src/generated/pbjson.rs @@ -5487,6 +5487,9 @@ impl serde::Serialize for DmlNode { if self.target.is_some() { len += 1; } + if self.merge_into.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.DmlNode", len)?; if self.dml_type != 0 { let v = dml_node::Type::try_from(self.dml_type) @@ -5502,6 +5505,9 @@ impl serde::Serialize for DmlNode { if let Some(v) = self.target.as_ref() { struct_ser.serialize_field("target", v)?; } + if let Some(v) = self.merge_into.as_ref() { + struct_ser.serialize_field("mergeInto", v)?; + } struct_ser.end() } } @@ -5518,6 +5524,8 @@ impl<'de> serde::Deserialize<'de> for DmlNode { "table_name", "tableName", "target", + "merge_into", + "mergeInto", ]; #[allow(clippy::enum_variant_names)] @@ -5526,6 +5534,7 @@ impl<'de> serde::Deserialize<'de> for DmlNode { Input, TableName, Target, + MergeInto, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -5551,6 +5560,7 @@ impl<'de> serde::Deserialize<'de> for DmlNode { "input" => Ok(GeneratedField::Input), "tableName" | "table_name" => Ok(GeneratedField::TableName), "target" => Ok(GeneratedField::Target), + "mergeInto" | "merge_into" => Ok(GeneratedField::MergeInto), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -5574,6 +5584,7 @@ impl<'de> serde::Deserialize<'de> for DmlNode { let mut input__ = None; let mut table_name__ = None; let mut target__ = None; + let mut merge_into__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::DmlType => { @@ -5600,6 +5611,12 @@ impl<'de> serde::Deserialize<'de> for DmlNode { } target__ = map_.next_value()?; } + GeneratedField::MergeInto => { + if merge_into__.is_some() { + return Err(serde::de::Error::duplicate_field("mergeInto")); + } + merge_into__ = map_.next_value()?; + } } } Ok(DmlNode { @@ -5607,6 +5624,7 @@ impl<'de> serde::Deserialize<'de> for DmlNode { input: input__, table_name: table_name__, target: target__, + merge_into: merge_into__, }) } } @@ -5627,6 +5645,7 @@ impl serde::Serialize for dml_node::Type { Self::InsertOverwrite => "INSERT_OVERWRITE", Self::InsertReplace => "INSERT_REPLACE", Self::Truncate => "TRUNCATE", + Self::MergeInto => "MERGE_INTO", }; serializer.serialize_str(variant) } @@ -5645,6 +5664,7 @@ impl<'de> serde::Deserialize<'de> for dml_node::Type { "INSERT_OVERWRITE", "INSERT_REPLACE", "TRUNCATE", + "MERGE_INTO", ]; struct GeneratedVisitor; @@ -5692,6 +5712,7 @@ impl<'de> serde::Deserialize<'de> for dml_node::Type { "INSERT_OVERWRITE" => Ok(dml_node::Type::InsertOverwrite), "INSERT_REPLACE" => Ok(dml_node::Type::InsertReplace), "TRUNCATE" => Ok(dml_node::Type::Truncate), + "MERGE_INTO" => Ok(dml_node::Type::MergeInto), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } @@ -14033,6 +14054,818 @@ impl<'de> serde::Deserialize<'de> for MemoryScanExecNode { deserializer.deserialize_struct("datafusion.MemoryScanExecNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for MergeAssignment { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.column.is_empty() { + len += 1; + } + if self.value.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MergeAssignment", len)?; + if !self.column.is_empty() { + struct_ser.serialize_field("column", &self.column)?; + } + if let Some(v) = self.value.as_ref() { + struct_ser.serialize_field("value", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeAssignment { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "column", + "value", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Column, + Value, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "column" => Ok(GeneratedField::Column), + "value" => Ok(GeneratedField::Value), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeAssignment; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeAssignment") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut column__ = None; + let mut value__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Column => { + if column__.is_some() { + return Err(serde::de::Error::duplicate_field("column")); + } + column__ = Some(map_.next_value()?); + } + GeneratedField::Value => { + if value__.is_some() { + return Err(serde::de::Error::duplicate_field("value")); + } + value__ = map_.next_value()?; + } + } + } + Ok(MergeAssignment { + column: column__.unwrap_or_default(), + value: value__, + }) + } + } + deserializer.deserialize_struct("datafusion.MergeAssignment", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MergeDeleteAction { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("datafusion.MergeDeleteAction", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeDeleteAction { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeDeleteAction; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeDeleteAction") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; + } + Ok(MergeDeleteAction { + }) + } + } + deserializer.deserialize_struct("datafusion.MergeDeleteAction", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MergeInsertAction { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.columns.is_empty() { + len += 1; + } + if !self.values.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MergeInsertAction", len)?; + if !self.columns.is_empty() { + struct_ser.serialize_field("columns", &self.columns)?; + } + if !self.values.is_empty() { + struct_ser.serialize_field("values", &self.values)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeInsertAction { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "columns", + "values", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Columns, + Values, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "columns" => Ok(GeneratedField::Columns), + "values" => Ok(GeneratedField::Values), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeInsertAction; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeInsertAction") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut columns__ = None; + let mut values__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Columns => { + if columns__.is_some() { + return Err(serde::de::Error::duplicate_field("columns")); + } + columns__ = Some(map_.next_value()?); + } + GeneratedField::Values => { + if values__.is_some() { + return Err(serde::de::Error::duplicate_field("values")); + } + values__ = Some(map_.next_value()?); + } + } + } + Ok(MergeInsertAction { + columns: columns__.unwrap_or_default(), + values: values__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.MergeInsertAction", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MergeIntoActionNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.action.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MergeIntoActionNode", len)?; + if let Some(v) = self.action.as_ref() { + match v { + merge_into_action_node::Action::Update(v) => { + struct_ser.serialize_field("update", v)?; + } + merge_into_action_node::Action::Insert(v) => { + struct_ser.serialize_field("insert", v)?; + } + merge_into_action_node::Action::Delete(v) => { + struct_ser.serialize_field("delete", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeIntoActionNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "update", + "insert", + "delete", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Update, + Insert, + Delete, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "update" => Ok(GeneratedField::Update), + "insert" => Ok(GeneratedField::Insert), + "delete" => Ok(GeneratedField::Delete), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeIntoActionNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeIntoActionNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut action__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Update => { + if action__.is_some() { + return Err(serde::de::Error::duplicate_field("update")); + } + action__ = map_.next_value::<::std::option::Option<_>>()?.map(merge_into_action_node::Action::Update) +; + } + GeneratedField::Insert => { + if action__.is_some() { + return Err(serde::de::Error::duplicate_field("insert")); + } + action__ = map_.next_value::<::std::option::Option<_>>()?.map(merge_into_action_node::Action::Insert) +; + } + GeneratedField::Delete => { + if action__.is_some() { + return Err(serde::de::Error::duplicate_field("delete")); + } + action__ = map_.next_value::<::std::option::Option<_>>()?.map(merge_into_action_node::Action::Delete) +; + } + } + } + Ok(MergeIntoActionNode { + action: action__, + }) + } + } + deserializer.deserialize_struct("datafusion.MergeIntoActionNode", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MergeIntoClauseNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.kind != 0 { + len += 1; + } + if self.predicate.is_some() { + len += 1; + } + if self.action.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MergeIntoClauseNode", len)?; + if self.kind != 0 { + let v = merge_into_clause_node::Kind::try_from(self.kind) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.kind)))?; + struct_ser.serialize_field("kind", &v)?; + } + if let Some(v) = self.predicate.as_ref() { + struct_ser.serialize_field("predicate", v)?; + } + if let Some(v) = self.action.as_ref() { + struct_ser.serialize_field("action", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeIntoClauseNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "kind", + "predicate", + "action", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Kind, + Predicate, + Action, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "kind" => Ok(GeneratedField::Kind), + "predicate" => Ok(GeneratedField::Predicate), + "action" => Ok(GeneratedField::Action), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeIntoClauseNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeIntoClauseNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut kind__ = None; + let mut predicate__ = None; + let mut action__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Kind => { + if kind__.is_some() { + return Err(serde::de::Error::duplicate_field("kind")); + } + kind__ = Some(map_.next_value::()? as i32); + } + GeneratedField::Predicate => { + if predicate__.is_some() { + return Err(serde::de::Error::duplicate_field("predicate")); + } + predicate__ = map_.next_value()?; + } + GeneratedField::Action => { + if action__.is_some() { + return Err(serde::de::Error::duplicate_field("action")); + } + action__ = map_.next_value()?; + } + } + } + Ok(MergeIntoClauseNode { + kind: kind__.unwrap_or_default(), + predicate: predicate__, + action: action__, + }) + } + } + deserializer.deserialize_struct("datafusion.MergeIntoClauseNode", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for merge_into_clause_node::Kind { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Matched => "MATCHED", + Self::NotMatched => "NOT_MATCHED", + Self::NotMatchedByTarget => "NOT_MATCHED_BY_TARGET", + Self::NotMatchedBySource => "NOT_MATCHED_BY_SOURCE", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for merge_into_clause_node::Kind { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "MATCHED", + "NOT_MATCHED", + "NOT_MATCHED_BY_TARGET", + "NOT_MATCHED_BY_SOURCE", + ]; + + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = merge_into_clause_node::Kind; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "MATCHED" => Ok(merge_into_clause_node::Kind::Matched), + "NOT_MATCHED" => Ok(merge_into_clause_node::Kind::NotMatched), + "NOT_MATCHED_BY_TARGET" => Ok(merge_into_clause_node::Kind::NotMatchedByTarget), + "NOT_MATCHED_BY_SOURCE" => Ok(merge_into_clause_node::Kind::NotMatchedBySource), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} +impl serde::Serialize for MergeIntoOpNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.on.is_some() { + len += 1; + } + if !self.clauses.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MergeIntoOpNode", len)?; + if let Some(v) = self.on.as_ref() { + struct_ser.serialize_field("on", v)?; + } + if !self.clauses.is_empty() { + struct_ser.serialize_field("clauses", &self.clauses)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeIntoOpNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "on", + "clauses", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + On, + Clauses, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "on" => Ok(GeneratedField::On), + "clauses" => Ok(GeneratedField::Clauses), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeIntoOpNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeIntoOpNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut on__ = None; + let mut clauses__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::On => { + if on__.is_some() { + return Err(serde::de::Error::duplicate_field("on")); + } + on__ = map_.next_value()?; + } + GeneratedField::Clauses => { + if clauses__.is_some() { + return Err(serde::de::Error::duplicate_field("clauses")); + } + clauses__ = Some(map_.next_value()?); + } + } + } + Ok(MergeIntoOpNode { + on: on__, + clauses: clauses__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.MergeIntoOpNode", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for MergeUpdateAction { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.assignments.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MergeUpdateAction", len)?; + if !self.assignments.is_empty() { + struct_ser.serialize_field("assignments", &self.assignments)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MergeUpdateAction { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "assignments", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Assignments, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl serde::de::Visitor<'_> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "assignments" => Ok(GeneratedField::Assignments), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MergeUpdateAction; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MergeUpdateAction") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut assignments__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Assignments => { + if assignments__.is_some() { + return Err(serde::de::Error::duplicate_field("assignments")); + } + assignments__ = Some(map_.next_value()?); + } + } + } + Ok(MergeUpdateAction { + assignments: assignments__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.MergeUpdateAction", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for NamedStructField { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto-models/src/generated/prost.rs b/datafusion/proto-models/src/generated/prost.rs index 3ac04a6164db8..e8fa4599e1f9a 100644 --- a/datafusion/proto-models/src/generated/prost.rs +++ b/datafusion/proto-models/src/generated/prost.rs @@ -453,6 +453,9 @@ pub struct DmlNode { pub table_name: ::core::option::Option, #[prost(message, optional, boxed, tag = "5")] pub target: ::core::option::Option<::prost::alloc::boxed::Box>, + /// Populated only when dml_type == MERGE_INTO. + #[prost(message, optional, boxed, tag = "6")] + pub merge_into: ::core::option::Option<::prost::alloc::boxed::Box>, } /// Nested message and enum types in `DmlNode`. pub mod dml_node { @@ -476,6 +479,7 @@ pub mod dml_node { InsertOverwrite = 4, InsertReplace = 5, Truncate = 6, + MergeInto = 7, } impl Type { /// String value of the enum field names used in the ProtoBuf definition. @@ -491,6 +495,7 @@ pub mod dml_node { Self::InsertOverwrite => "INSERT_OVERWRITE", Self::InsertReplace => "INSERT_REPLACE", Self::Truncate => "TRUNCATE", + Self::MergeInto => "MERGE_INTO", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -503,11 +508,117 @@ pub mod dml_node { "INSERT_OVERWRITE" => Some(Self::InsertOverwrite), "INSERT_REPLACE" => Some(Self::InsertReplace), "TRUNCATE" => Some(Self::Truncate), + "MERGE_INTO" => Some(Self::MergeInto), _ => None, } } } } +/// Carries the ON condition and WHEN clauses of a MERGE INTO operation. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergeIntoOpNode { + #[prost(message, optional, boxed, tag = "1")] + pub on: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, repeated, tag = "2")] + pub clauses: ::prost::alloc::vec::Vec, +} +/// A single WHEN clause within a MERGE INTO statement. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergeIntoClauseNode { + #[prost(enumeration = "merge_into_clause_node::Kind", tag = "1")] + pub kind: i32, + /// Optional `AND ` predicate. Absent when the clause has no predicate. + #[prost(message, optional, tag = "2")] + pub predicate: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub action: ::core::option::Option, +} +/// Nested message and enum types in `MergeIntoClauseNode`. +pub mod merge_into_clause_node { + #[derive( + Clone, + Copy, + Debug, + PartialEq, + Eq, + Hash, + PartialOrd, + Ord, + ::prost::Enumeration + )] + #[repr(i32)] + pub enum Kind { + Matched = 0, + NotMatched = 1, + NotMatchedByTarget = 2, + NotMatchedBySource = 3, + } + impl Kind { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + Self::Matched => "MATCHED", + Self::NotMatched => "NOT_MATCHED", + Self::NotMatchedByTarget => "NOT_MATCHED_BY_TARGET", + Self::NotMatchedBySource => "NOT_MATCHED_BY_SOURCE", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "MATCHED" => Some(Self::Matched), + "NOT_MATCHED" => Some(Self::NotMatched), + "NOT_MATCHED_BY_TARGET" => Some(Self::NotMatchedByTarget), + "NOT_MATCHED_BY_SOURCE" => Some(Self::NotMatchedBySource), + _ => None, + } + } + } +} +/// The action for a single WHEN clause. +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergeIntoActionNode { + #[prost(oneof = "merge_into_action_node::Action", tags = "1, 2, 3")] + pub action: ::core::option::Option, +} +/// Nested message and enum types in `MergeIntoActionNode`. +pub mod merge_into_action_node { + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Action { + #[prost(message, tag = "1")] + Update(super::MergeUpdateAction), + #[prost(message, tag = "2")] + Insert(super::MergeInsertAction), + #[prost(message, tag = "3")] + Delete(super::MergeDeleteAction), + } +} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergeUpdateAction { + #[prost(message, repeated, tag = "1")] + pub assignments: ::prost::alloc::vec::Vec, +} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergeAssignment { + #[prost(string, tag = "1")] + pub column: ::prost::alloc::string::String, + #[prost(message, optional, tag = "2")] + pub value: ::core::option::Option, +} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct MergeInsertAction { + /// May be empty (meaning all columns). + #[prost(string, repeated, tag = "1")] + pub columns: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + /// One expression per inserted column. + #[prost(message, repeated, tag = "2")] + pub values: ::prost::alloc::vec::Vec, +} +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct MergeDeleteAction {} #[derive(Clone, PartialEq, ::prost::Message)] pub struct UnnestNode { #[prost(message, optional, boxed, tag = "1")] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index c68b83964f4cf..b79b21b3599c7 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -25,7 +25,9 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::registry::FunctionRegistry; -use datafusion_expr::dml::InsertOp; +use datafusion_expr::dml::{ + InsertOp, MergeIntoAction, MergeIntoClause, MergeIntoClauseKind, MergeIntoOp, +}; use datafusion_expr::expr::{Alias, NullTreatment, Placeholder, Sort}; use datafusion_expr::expr::{Unnest, WildcardOptions}; use datafusion_expr::logical_plan::Subquery; @@ -239,22 +241,133 @@ impl FromProto for NullEquality { } } -impl FromProto for WriteOp { - fn from_proto(t: protobuf::dml_node::Type) -> Self { - match t { - protobuf::dml_node::Type::Update => WriteOp::Update, - protobuf::dml_node::Type::Delete => WriteOp::Delete, - protobuf::dml_node::Type::InsertAppend => WriteOp::Insert(InsertOp::Append), - protobuf::dml_node::Type::InsertOverwrite => { - WriteOp::Insert(InsertOp::Overwrite) +impl FromProto for MergeIntoClauseKind { + fn from_proto(k: protobuf::merge_into_clause_node::Kind) -> Self { + match k { + protobuf::merge_into_clause_node::Kind::Matched => { + MergeIntoClauseKind::Matched + } + protobuf::merge_into_clause_node::Kind::NotMatched => { + MergeIntoClauseKind::NotMatched + } + protobuf::merge_into_clause_node::Kind::NotMatchedByTarget => { + MergeIntoClauseKind::NotMatchedByTarget + } + protobuf::merge_into_clause_node::Kind::NotMatchedBySource => { + MergeIntoClauseKind::NotMatchedBySource } - protobuf::dml_node::Type::InsertReplace => WriteOp::Insert(InsertOp::Replace), - protobuf::dml_node::Type::Ctas => WriteOp::Ctas, - protobuf::dml_node::Type::Truncate => WriteOp::Truncate, } } } +/// Reconstruct a [`WriteOp`] from a [`protobuf::DmlNode`], reading the +/// `merge_into` payload when the type tag is `MergeInto`. +pub fn parse_write_op( + node: &protobuf::DmlNode, + ctx: &TaskContext, + codec: &dyn LogicalExtensionCodec, +) -> Result { + let typ = node.dml_type(); + Ok(match typ { + protobuf::dml_node::Type::Update => WriteOp::Update, + protobuf::dml_node::Type::Delete => WriteOp::Delete, + protobuf::dml_node::Type::InsertAppend => WriteOp::Insert(InsertOp::Append), + protobuf::dml_node::Type::InsertOverwrite => WriteOp::Insert(InsertOp::Overwrite), + protobuf::dml_node::Type::InsertReplace => WriteOp::Insert(InsertOp::Replace), + protobuf::dml_node::Type::Ctas => WriteOp::Ctas, + protobuf::dml_node::Type::Truncate => WriteOp::Truncate, + protobuf::dml_node::Type::MergeInto => { + let merge_into = node.merge_into.as_deref().ok_or_else(|| { + Error::General( + "DmlNode with MERGE_INTO type is missing the merge_into payload" + .to_string(), + ) + })?; + WriteOp::MergeInto(Box::new(parse_merge_into_op(merge_into, ctx, codec)?)) + } + }) +} + +fn parse_merge_into_op( + op: &protobuf::MergeIntoOpNode, + ctx: &TaskContext, + codec: &dyn LogicalExtensionCodec, +) -> Result { + let on = op.on.as_ref().ok_or_else(|| { + Error::General("MergeIntoOpNode is missing required `on` expression".to_string()) + })?; + let on = parse_expr(on, ctx, codec)?; + let clauses = op + .clauses + .iter() + .map(|c| parse_merge_into_clause(c, ctx, codec)) + .collect::, Error>>()?; + Ok(MergeIntoOp { on, clauses }) +} + +fn parse_merge_into_clause( + clause: &protobuf::MergeIntoClauseNode, + ctx: &TaskContext, + codec: &dyn LogicalExtensionCodec, +) -> Result { + let kind = protobuf::merge_into_clause_node::Kind::try_from(clause.kind) + .map_err(|_| { + Error::General(format!( + "MergeIntoClauseNode has unknown kind tag {}", + clause.kind + )) + }) + .map(MergeIntoClauseKind::from_proto)?; + let predicate = clause + .predicate + .as_ref() + .map(|e| parse_expr(e, ctx, codec)) + .transpose()?; + let action = clause.action.as_ref().ok_or_else(|| { + Error::General("MergeIntoClauseNode is missing required `action`".to_string()) + })?; + let action = parse_merge_into_action(action, ctx, codec)?; + Ok(MergeIntoClause { + kind, + predicate, + action, + }) +} + +fn parse_merge_into_action( + action: &protobuf::MergeIntoActionNode, + ctx: &TaskContext, + codec: &dyn LogicalExtensionCodec, +) -> Result { + use protobuf::merge_into_action_node::Action; + let action = action.action.as_ref().ok_or_else(|| { + Error::General("MergeIntoActionNode is missing the `action` oneof".to_string()) + })?; + Ok(match action { + Action::Update(update) => { + let assignments = update + .assignments + .iter() + .map(|a| { + let value = a.value.as_ref().ok_or_else(|| { + Error::General(format!( + "MergeAssignment for column `{}` is missing its value", + a.column + )) + })?; + Ok((a.column.clone(), parse_expr(value, ctx, codec)?)) + }) + .collect::, Error>>()?; + MergeIntoAction::Update(assignments) + } + Action::Insert(insert) => MergeIntoAction::Insert { + columns: insert.columns.clone(), + values: parse_exprs(&insert.values, ctx, codec)?, + }, + Action::Delete(_) => MergeIntoAction::Delete, + }) +} + impl FromProto for NullTreatment { fn from_proto(t: protobuf::NullTreatment) -> Self { match t { diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 35c2e76d880b9..0e73898c67d82 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -58,6 +58,7 @@ use datafusion_datasource_json::file_format::{ }; #[cfg(feature = "parquet")] use datafusion_datasource_parquet::file_format::{ParquetFormat, ParquetFormatFactory}; +use datafusion_expr::dml::InsertOp; use datafusion_expr::{ AggregateUDF, DmlStatement, FetchType, HigherOrderUDF, RecursiveQuery, SkipType, TableSource, Unnest, WriteOp, @@ -1248,10 +1249,12 @@ impl AsLogicalPlan for LogicalPlanNode { .build() } LogicalPlanType::Dml(dml_node) => { + let write_op = + from_proto::parse_write_op(dml_node, ctx, extension_codec)?; Ok(LogicalPlan::Dml(datafusion_expr::DmlStatement::new( from_table_reference(dml_node.table_name.as_ref(), "DML ")?, to_table_source(&dml_node.target, ctx, extension_codec)?, - WriteOp::from_proto(dml_node.dml_type()), + write_op, Arc::new(into_logical_plan!(dml_node.input, ctx, extension_codec)?), ))) } @@ -2085,7 +2088,33 @@ impl AsLogicalPlan for LogicalPlanNode { }) => { let input = LogicalPlanNode::try_from_logical_plan(input, extension_codec)?; - let dml_type = dml_node::Type::from_proto(op); + let (dml_type, merge_into) = match op { + WriteOp::Insert(InsertOp::Append) => { + (dml_node::Type::InsertAppend, None) + } + WriteOp::Insert(InsertOp::Overwrite) => { + (dml_node::Type::InsertOverwrite, None) + } + WriteOp::Insert(InsertOp::Replace) => { + (dml_node::Type::InsertReplace, None) + } + WriteOp::Delete => (dml_node::Type::Delete, None), + WriteOp::Update => (dml_node::Type::Update, None), + WriteOp::Ctas => (dml_node::Type::Ctas, None), + WriteOp::Truncate => (dml_node::Type::Truncate, None), + WriteOp::MergeInto(merge_op) => ( + dml_node::Type::MergeInto, + Some(Box::new(to_proto::serialize_merge_into_op( + merge_op, + extension_codec, + )?)), + ), + other => { + return Err(proto_error(format!( + "WriteOp variant has no DmlNode encoding: {other}" + ))); + } + }; Ok(LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Dml(Box::new(DmlNode { input: Some(Box::new(input)), @@ -2098,6 +2127,7 @@ impl AsLogicalPlan for LogicalPlanNode { table_name.clone(), )), dml_type: dml_type.into(), + merge_into, }))), }) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 71a6bd824a369..516aca4094451 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -22,8 +22,9 @@ use std::collections::HashMap; use datafusion_common::{NullEquality, TableReference, UnnestOptions}; -use datafusion_expr::WriteOp; -use datafusion_expr::dml::InsertOp; +use datafusion_expr::dml::{ + MergeIntoAction, MergeIntoClause, MergeIntoClauseKind, MergeIntoOp, +}; use datafusion_expr::expr::{ self, AggregateFunctionParams, Alias, Between, BinaryExpr, Cast, GroupingSet, InList, Like, NullTreatment, Placeholder, ScalarFunction, Unnest, @@ -753,22 +754,92 @@ impl FromProto for protobuf::NullEquality { } } -impl FromProto<&WriteOp> for protobuf::dml_node::Type { - fn from_proto(t: &WriteOp) -> Self { - match t { - WriteOp::Insert(InsertOp::Append) => protobuf::dml_node::Type::InsertAppend, - WriteOp::Insert(InsertOp::Overwrite) => { - protobuf::dml_node::Type::InsertOverwrite +impl FromProto for protobuf::merge_into_clause_node::Kind { + fn from_proto(k: MergeIntoClauseKind) -> Self { + match k { + MergeIntoClauseKind::Matched => { + protobuf::merge_into_clause_node::Kind::Matched + } + MergeIntoClauseKind::NotMatched => { + protobuf::merge_into_clause_node::Kind::NotMatched + } + MergeIntoClauseKind::NotMatchedByTarget => { + protobuf::merge_into_clause_node::Kind::NotMatchedByTarget + } + MergeIntoClauseKind::NotMatchedBySource => { + protobuf::merge_into_clause_node::Kind::NotMatchedBySource } - WriteOp::Insert(InsertOp::Replace) => protobuf::dml_node::Type::InsertReplace, - WriteOp::Delete => protobuf::dml_node::Type::Delete, - WriteOp::Update => protobuf::dml_node::Type::Update, - WriteOp::Ctas => protobuf::dml_node::Type::Ctas, - WriteOp::Truncate => protobuf::dml_node::Type::Truncate, } } } +pub fn serialize_merge_into_op( + op: &MergeIntoOp, + codec: &dyn LogicalExtensionCodec, +) -> Result { + Ok(protobuf::MergeIntoOpNode { + on: Some(Box::new(serialize_expr(&op.on, codec)?)), + clauses: op + .clauses + .iter() + .map(|c| serialize_merge_into_clause(c, codec)) + .collect::, Error>>()?, + }) +} + +fn serialize_merge_into_clause( + clause: &MergeIntoClause, + codec: &dyn LogicalExtensionCodec, +) -> Result { + let kind = protobuf::merge_into_clause_node::Kind::from_proto(clause.kind); + let predicate = clause + .predicate + .as_ref() + .map(|e| serialize_expr(e, codec)) + .transpose()?; + Ok(protobuf::MergeIntoClauseNode { + kind: kind.into(), + predicate, + action: Some(serialize_merge_into_action(&clause.action, codec)?), + }) +} + +fn serialize_merge_into_action( + action: &MergeIntoAction, + codec: &dyn LogicalExtensionCodec, +) -> Result { + let action = match action { + MergeIntoAction::Update(assignments) => { + let assignments = assignments + .iter() + .map(|(column, value)| { + Ok(protobuf::MergeAssignment { + column: column.clone(), + value: Some(serialize_expr(value, codec)?), + }) + }) + .collect::, Error>>()?; + protobuf::merge_into_action_node::Action::Update( + protobuf::MergeUpdateAction { assignments }, + ) + } + MergeIntoAction::Insert { columns, values } => { + protobuf::merge_into_action_node::Action::Insert( + protobuf::MergeInsertAction { + columns: columns.clone(), + values: serialize_exprs(values, codec)?, + }, + ) + } + MergeIntoAction::Delete => protobuf::merge_into_action_node::Action::Delete( + protobuf::MergeDeleteAction {}, + ), + }; + Ok(protobuf::MergeIntoActionNode { + action: Some(action), + }) +} + impl FromProto for protobuf::NullTreatment { fn from_proto(t: NullTreatment) -> Self { match t { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 7f1d0a666fdce..9d8e5c2b1ef48 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -78,6 +78,9 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_expr::dml::CopyTo; +use datafusion_expr::dml::{ + MergeIntoAction, MergeIntoClause, MergeIntoClauseKind, MergeIntoOp, +}; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Cast, GroupingSet, InList, Like, NullTreatment, ScalarFunction, Unnest, WildcardOptions, @@ -86,10 +89,11 @@ use datafusion_expr::logical_plan::{ ExplainOption, Extension, UserDefinedLogicalNodeCore, }; use datafusion_expr::{ - Accumulator, AggregateUDF, ColumnarValue, ExprFunctionExt, ExprSchemable, - LimitEffect, Literal, LogicalPlan, LogicalPlanBuilder, Operator, PartitionEvaluator, - ScalarUDF, Signature, TryCast, Volatility, WindowFrame, WindowFrameBound, - WindowFrameUnits, WindowFunctionDefinition, WindowUDF, WindowUDFImpl, + Accumulator, AggregateUDF, ColumnarValue, DmlStatement, ExprFunctionExt, + ExprSchemable, LimitEffect, Literal, LogicalPlan, LogicalPlanBuilder, Operator, + PartitionEvaluator, ScalarUDF, Signature, TryCast, Volatility, WindowFrame, + WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, WindowUDF, + WindowUDFImpl, WriteOp, }; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::expr_fn::{ @@ -524,6 +528,181 @@ async fn roundtrip_logical_plan_dml() -> Result<()> { Ok(()) } +#[tokio::test] +async fn roundtrip_logical_plan_dml_merge_into() -> Result<()> { + let ctx = SessionContext::new(); + let schema = Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Decimal128(15, 2), true), + ]); + ctx.register_csv( + "t1", + "tests/testdata/test.csv", + CsvReadOptions::default().schema(&schema), + ) + .await?; + + let scan = ctx.table("t1").await?.into_optimized_plan()?; + let target = match &scan { + LogicalPlan::TableScan(t) => Arc::clone(&t.source), + other => panic!("expected TableScan, got {other:?}"), + }; + + let merge = WriteOp::MergeInto(Box::new(MergeIntoOp { + on: col("a").eq(lit(1_i64)), + clauses: vec![ + MergeIntoClause { + kind: MergeIntoClauseKind::Matched, + predicate: Some(col("b").gt(lit(ScalarValue::Decimal128( + Some(0), + 15, + 2, + )))), + action: MergeIntoAction::Update(vec![("b".to_string(), col("b"))]), + }, + MergeIntoClause { + kind: MergeIntoClauseKind::NotMatched, + predicate: None, + action: MergeIntoAction::Insert { + columns: vec!["a".to_string(), "b".to_string()], + values: vec![col("a"), col("b")], + }, + }, + MergeIntoClause { + kind: MergeIntoClauseKind::NotMatchedByTarget, + predicate: None, + action: MergeIntoAction::Insert { + columns: vec![], + values: vec![col("a"), col("b")], + }, + }, + MergeIntoClause { + kind: MergeIntoClauseKind::NotMatchedBySource, + predicate: Some(col("a").eq(lit(2_i64))), + action: MergeIntoAction::Delete, + }, + ], + })); + + let plan = LogicalPlan::Dml(DmlStatement::new( + "t1".into(), + target, + merge, + Arc::new(scan), + )); + + let bytes = logical_plan_to_bytes(&plan)?; + let round_trip = logical_plan_from_bytes(&bytes, &ctx.task_ctx())?; + assert_eq!(format!("{plan}"), format!("{round_trip}")); + Ok(()) +} + +#[test] +fn parse_write_op_merge_into_without_payload_errors() { + let ctx = SessionContext::new(); + let codec = DefaultLogicalExtensionCodec {}; + let node = protobuf::DmlNode { + dml_type: protobuf::dml_node::Type::MergeInto.into(), + ..Default::default() + }; + let err = from_proto::parse_write_op(&node, ctx.task_ctx().as_ref(), &codec) + .expect_err("MergeInto tag without payload must fail"); + assert!( + err.to_string().contains("merge_into"), + "unexpected error: {err}" + ); +} + +/// Build a `DmlNode` whose `merge_into` payload is exactly the supplied +/// `MergeIntoOpNode`. Used by the error-path tests below. +fn dml_node_with_merge_payload(payload: protobuf::MergeIntoOpNode) -> protobuf::DmlNode { + protobuf::DmlNode { + dml_type: protobuf::dml_node::Type::MergeInto.into(), + merge_into: Some(Box::new(payload)), + ..Default::default() + } +} + +#[test] +fn parse_merge_into_op_missing_on_errors() { + let ctx = SessionContext::new(); + let codec = DefaultLogicalExtensionCodec {}; + let node = dml_node_with_merge_payload(protobuf::MergeIntoOpNode { + on: None, + clauses: vec![], + }); + let err = from_proto::parse_write_op(&node, ctx.task_ctx().as_ref(), &codec) + .expect_err("missing `on` must fail"); + assert!(err.to_string().contains("`on`"), "unexpected error: {err}"); +} + +#[test] +fn parse_merge_into_clause_unknown_kind_errors() { + let ctx = SessionContext::new(); + let codec = DefaultLogicalExtensionCodec {}; + let on = serialize_expr(&lit(true), &codec).unwrap(); + let node = dml_node_with_merge_payload(protobuf::MergeIntoOpNode { + on: Some(Box::new(on)), + clauses: vec![protobuf::MergeIntoClauseNode { + kind: 999, // unknown enum tag + predicate: None, + action: Some(protobuf::MergeIntoActionNode { + action: Some(protobuf::merge_into_action_node::Action::Delete( + protobuf::MergeDeleteAction {}, + )), + }), + }], + }); + let err = from_proto::parse_write_op(&node, ctx.task_ctx().as_ref(), &codec) + .expect_err("unknown clause kind tag must fail"); + assert!( + err.to_string().contains("unknown kind tag"), + "unexpected error: {err}" + ); +} + +#[test] +fn parse_merge_into_clause_missing_action_errors() { + let ctx = SessionContext::new(); + let codec = DefaultLogicalExtensionCodec {}; + let on = serialize_expr(&lit(true), &codec).unwrap(); + let node = dml_node_with_merge_payload(protobuf::MergeIntoOpNode { + on: Some(Box::new(on)), + clauses: vec![protobuf::MergeIntoClauseNode { + kind: protobuf::merge_into_clause_node::Kind::Matched.into(), + predicate: None, + action: None, + }], + }); + let err = from_proto::parse_write_op(&node, ctx.task_ctx().as_ref(), &codec) + .expect_err("missing clause `action` must fail"); + assert!( + err.to_string().contains("missing required `action`"), + "unexpected error: {err}" + ); +} + +#[test] +fn parse_merge_into_action_missing_oneof_errors() { + let ctx = SessionContext::new(); + let codec = DefaultLogicalExtensionCodec {}; + let on = serialize_expr(&lit(true), &codec).unwrap(); + let node = dml_node_with_merge_payload(protobuf::MergeIntoOpNode { + on: Some(Box::new(on)), + clauses: vec![protobuf::MergeIntoClauseNode { + kind: protobuf::merge_into_clause_node::Kind::Matched.into(), + predicate: None, + action: Some(protobuf::MergeIntoActionNode { action: None }), + }], + }); + let err = from_proto::parse_write_op(&node, ctx.task_ctx().as_ref(), &codec) + .expect_err("missing action oneof must fail"); + assert!( + err.to_string().contains("missing the `action` oneof"), + "unexpected error: {err}" + ); +} + #[tokio::test] async fn roundtrip_logical_plan_copy_to_sql_options() -> Result<()> { let ctx = SessionContext::new(); diff --git a/datafusion/spark/src/function/aggregate/avg.rs b/datafusion/spark/src/function/aggregate/avg.rs index 5f4d2c253a2dc..6ca3c59309e70 100644 --- a/datafusion/spark/src/function/aggregate/avg.rs +++ b/datafusion/spark/src/function/aggregate/avg.rs @@ -289,7 +289,6 @@ where &mut self, values: &[ArrayRef], group_indices: &[usize], - _opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { assert_eq!(values.len(), 2, "two arguments to merge_batch"); @@ -464,7 +463,6 @@ mod tests { acc.merge_batch( &state, &[0, 0, 0], - None, 1, // single group ) .unwrap(); @@ -486,7 +484,7 @@ mod tests { Some(3.0), ]))]; let state = acc.convert_to_state(&input, None).unwrap(); - acc.merge_batch(&state, &[0, 0, 0], None, 1).unwrap(); + acc.merge_batch(&state, &[0, 0, 0], 1).unwrap(); let result = acc.evaluate(EmitTo::All).unwrap(); let result = result.as_primitive::(); diff --git a/datafusion/spark/src/function/datetime/next_day.rs b/datafusion/spark/src/function/datetime/next_day.rs index 2241043d44cd7..2ef222526f387 100644 --- a/datafusion/spark/src/function/datetime/next_day.rs +++ b/datafusion/spark/src/function/datetime/next_day.rs @@ -210,7 +210,7 @@ where fn spark_next_day(days: i32, day_of_week: &str) -> Option { let date = Date32Type::to_naive_date_opt(days)?; - let day_of_week = day_of_week.trim().to_uppercase(); + let day_of_week = day_of_week.to_uppercase(); let day_of_week = match day_of_week.as_str() { "MO" | "MON" | "MONDAY" => Some("MONDAY"), "TU" | "TUE" | "TUESDAY" => Some("TUESDAY"), @@ -279,4 +279,10 @@ mod tests { assert_eq!(field.data_type(), &DataType::Date32); assert!(field.is_nullable()); } + + #[test] + fn next_day_rejects_whitespace_padded_day_names() { + let monday = 19723; // 2024-01-01 + assert_eq!(spark_next_day(monday, " MO "), None); + } } diff --git a/datafusion/spark/src/function/string/concat_ws.rs b/datafusion/spark/src/function/string/concat_ws.rs new file mode 100644 index 0000000000000..c9ed1369a51a7 --- /dev/null +++ b/datafusion/spark/src/function/string/concat_ws.rs @@ -0,0 +1,297 @@ +// 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. + +//! Spark-compatible `concat_ws`: joins strings (and array elements) with a separator. +//! +//! Null scalar args and null array elements are skipped; a null separator yields a +//! null row. Non-string args are coerced to STRING; list args (`List`, `LargeList`, +//! `ListView`, `LargeListView`, `FixedSizeList`) expand their elements. +//! +//! Differences with DataFusion core `concat_ws`: +//! - Accepts list arguments and expands their elements +//! - Always returns Utf8 (Spark's `STRING` type) +//! - Coerces non-string scalars (numbers, booleans, dates, ...) to Utf8 + +use std::fmt::Write as _; +use std::sync::Arc; + +use arrow::array::{ + Array, ArrayRef, AsArray, GenericListArray, LargeStringArray, OffsetSizeTrait, + StringArray, StringBuilder, StringViewArray, +}; +use arrow::datatypes::{DataType, Field}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::{ + ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, +}; + +use crate::function::error_utils::{ + invalid_arg_count_exec_err, unsupported_data_type_exec_err, +}; + +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct SparkConcatWs { + signature: Signature, +} + +impl Default for SparkConcatWs { + fn default() -> Self { + Self::new() + } +} + +impl SparkConcatWs { + pub fn new() -> Self { + Self { + signature: Signature::user_defined(Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for SparkConcatWs { + fn name(&self) -> &str { + "concat_ws" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Utf8) + } + + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + if arg_types.is_empty() { + return Err(invalid_arg_count_exec_err("concat_ws", (1, i32::MAX), 0)); + } + Ok(arg_types + .iter() + .enumerate() + .map(|(i, dt)| match dt { + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => dt.clone(), + // Non-separator list args expand their elements at runtime. + // Normalize the list variant so the kernel only sees + // List/LargeList, AND force the element type to Utf8 so the + // planner inserts a cast for non-string children (Spark + // coerces them to STRING the same way it does for scalars). + DataType::List(f) + | DataType::ListView(f) + | DataType::FixedSizeList(f, _) + if i > 0 => + { + DataType::List(Arc::new(Field::new( + f.name(), + DataType::Utf8, + f.is_nullable(), + ))) + } + DataType::LargeList(f) | DataType::LargeListView(f) if i > 0 => { + DataType::LargeList(Arc::new(Field::new( + f.name(), + DataType::Utf8, + f.is_nullable(), + ))) + } + // Spark casts everything else (numbers, booleans, dates, + // binary, null...) to STRING. + _ => DataType::Utf8, + }) + .collect()) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + // Only separator provided → empty string (or NULL if separator is null). + // Arg-count validation happens in coerce_types at planning time. + if args.args.len() == 1 { + return only_separator(&args.args[0]); + } + + spark_concat_ws(&args.args, args.number_rows) + } +} + +fn only_separator(sep: &ColumnarValue) -> Result { + match sep { + ColumnarValue::Scalar(s) if s.is_null() => { + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(None))) + } + ColumnarValue::Scalar(_) => Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some( + String::new(), + )))), + ColumnarValue::Array(arr) => { + let mut builder = StringBuilder::with_capacity(arr.len(), 0); + for row_idx in 0..arr.len() { + if arr.is_null(row_idx) { + builder.append_null(); + } else { + builder.append_value(""); + } + } + Ok(ColumnarValue::Array(Arc::new(builder.finish()) as ArrayRef)) + } + } +} + +fn spark_concat_ws(args: &[ColumnarValue], num_rows: usize) -> Result { + let arrays = ColumnarValue::values_to_arrays(args)?; + let sep_view = StringView::try_new(&arrays[0])?; + let arg_views: Vec = arrays[1..] + .iter() + .map(ArgView::try_new) + .collect::>()?; + + let mut builder = StringBuilder::with_capacity(num_rows, num_rows * 16); + + for row_idx in 0..num_rows { + if sep_view.is_null(row_idx) { + builder.append_null(); + continue; + } + + // Write parts directly into the builder via its `fmt::Write` impl; + // `append_value("")` then finalises the row (offset + validity) with + // no extra copy from an intermediate `String`. + let separator = sep_view.value(row_idx); + let mut first = true; + for view in &arg_views { + view.write_row(row_idx, separator, &mut builder, &mut first)?; + } + builder.append_value(""); + } + + Ok(ColumnarValue::Array(Arc::new(builder.finish()) as ArrayRef)) +} + +/// Typed view over a string array that downcasts once and exposes +/// per-row access without further dispatch. +enum StringView<'a> { + Utf8(&'a StringArray), + LargeUtf8(&'a LargeStringArray), + Utf8View(&'a StringViewArray), +} + +impl<'a> StringView<'a> { + fn try_new(arr: &'a ArrayRef) -> Result { + match arr.data_type() { + DataType::Utf8 => Ok(Self::Utf8(arr.as_string::())), + DataType::LargeUtf8 => Ok(Self::LargeUtf8(arr.as_string::())), + DataType::Utf8View => Ok(Self::Utf8View(arr.as_string_view())), + other => Err(unsupported_data_type_exec_err("concat_ws", "STRING", other)), + } + } + + fn value(&self, idx: usize) -> &str { + match self { + Self::Utf8(a) => a.value(idx), + Self::LargeUtf8(a) => a.value(idx), + Self::Utf8View(a) => a.value(idx), + } + } + + fn is_null(&self, idx: usize) -> bool { + match self { + Self::Utf8(a) => a.is_null(idx), + Self::LargeUtf8(a) => a.is_null(idx), + Self::Utf8View(a) => a.is_null(idx), + } + } +} + +/// Per-argument view: a string array or a list of strings. The downcast +/// happens once at construction time. `DataType::Null` cannot appear here — +/// `coerce_types` rewrites it to `Utf8` before invocation. +enum ArgView<'a> { + Str(StringView<'a>), + List(&'a GenericListArray), + LargeList(&'a GenericListArray), +} + +impl<'a> ArgView<'a> { + fn try_new(arr: &'a ArrayRef) -> Result { + match arr.data_type() { + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { + Ok(Self::Str(StringView::try_new(arr)?)) + } + DataType::List(_) => Ok(Self::List(arr.as_list::())), + DataType::LargeList(_) => Ok(Self::LargeList(arr.as_list::())), + other => Err(unsupported_data_type_exec_err( + "concat_ws", + "STRING or ARRAY", + other, + )), + } + } + + fn write_row( + &self, + row_idx: usize, + sep: &str, + builder: &mut StringBuilder, + first: &mut bool, + ) -> Result<()> { + match self { + Self::Str(view) => { + if !view.is_null(row_idx) { + push_part(builder, view.value(row_idx), sep, first); + } + } + Self::List(list) => write_list_row(*list, row_idx, sep, builder, first)?, + Self::LargeList(list) => write_list_row(*list, row_idx, sep, builder, first)?, + } + Ok(()) + } +} + +fn write_list_row( + list: &GenericListArray, + row_idx: usize, + sep: &str, + builder: &mut StringBuilder, + first: &mut bool, +) -> Result<()> { + if list.is_null(row_idx) { + return Ok(()); + } + let values = list.value(row_idx); + // An empty array (e.g. `array()`) contributes nothing — Spark renders it + // as the empty string, not an error. + if values.is_empty() { + return Ok(()); + } + let view = StringView::try_new(&values)?; + for i in 0..values.len() { + if !view.is_null(i) { + push_part(builder, view.value(i), sep, first); + } + } + Ok(()) +} + +// `StringBuilder::write_str` only does `extend_from_slice` and never errors; +// the `.expect(..)` is a documentation hint, not a real failure path. +fn push_part(builder: &mut StringBuilder, part: &str, sep: &str, first: &mut bool) { + if !*first { + builder + .write_str(sep) + .expect("StringBuilder::write_str is infallible"); + } + *first = false; + builder + .write_str(part) + .expect("StringBuilder::write_str is infallible"); +} diff --git a/datafusion/spark/src/function/string/mod.rs b/datafusion/spark/src/function/string/mod.rs index 9c90ded5f7e1b..bc94c27732c91 100644 --- a/datafusion/spark/src/function/string/mod.rs +++ b/datafusion/spark/src/function/string/mod.rs @@ -19,6 +19,7 @@ pub mod ascii; pub mod base64; pub mod char; pub mod concat; +pub mod concat_ws; pub mod elt; pub mod format_string; pub mod ilike; @@ -40,6 +41,7 @@ make_udf_function!(ascii::SparkAscii, ascii); make_udf_function!(base64::SparkBase64, base64); make_udf_function!(char::CharFunc, char); make_udf_function!(concat::SparkConcat, concat); +make_udf_function!(concat_ws::SparkConcatWs, concat_ws); make_udf_function!(ilike::SparkILike, ilike); make_udf_function!(length::SparkLengthFunc, length); make_udf_function!(elt::SparkElt, elt); @@ -77,6 +79,11 @@ pub mod expr_fn { "Concatenates multiple input strings into a single string. Returns NULL if any input is NULL.", args )); + export_functions!(( + concat_ws, + "Concatenates strings with separator. Supports arrays. Null values are skipped.", + sep args + )); export_functions!(( elt, "Returns the n-th input (1-indexed), e.g. returns 2nd input when n is 2. The function returns NULL if the index is 0 or exceeds the length of the array.", @@ -142,6 +149,7 @@ pub fn functions() -> Vec> { base64(), char(), concat(), + concat_ws(), elt(), ilike(), length(), diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 401313f9d396c..06efcde407933 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -39,7 +39,9 @@ use datafusion_common::{ internal_err, not_impl_err, plan_datafusion_err, plan_err, schema_err, unqualified_field_not_found, }; -use datafusion_expr::dml::{CopyTo, InsertOp}; +use datafusion_expr::dml::{ + CopyTo, InsertOp, MergeIntoAction, MergeIntoClause, MergeIntoClauseKind, MergeIntoOp, +}; use datafusion_expr::expr_rewriter::normalize_col_with_schemas_and_ambiguity_check; use datafusion_expr::logical_plan::DdlStatement; use datafusion_expr::logical_plan::builder::project; @@ -1217,6 +1219,8 @@ impl SqlToRel<'_, S> { self.delete_to_plan(&table_name, selection, limit) } + Statement::Merge(merge) => self.merge_to_plan(merge), + Statement::StartTransaction { modes, begin: false, @@ -2407,6 +2411,189 @@ impl SqlToRel<'_, S> { Ok(plan) } + fn merge_to_plan(&self, merge: ast::Merge) -> Result { + let ast::Merge { + table, + source, + on, + clauses, + into: _, + merge_token: _, + optimizer_hints, + output, + } = merge; + + if !optimizer_hints.is_empty() { + plan_err!("Optimizer hints not supported")?; + } + + if output.is_some() { + return not_impl_err!("MERGE OUTPUT clause is not supported"); + } + + // 1. Resolve target table + let (target_table_name, target_alias) = match &table { + TableFactor::Table { name, alias, .. } => (name.clone(), alias.clone()), + _ => plan_err!("Cannot MERGE INTO non-table relation!")?, + }; + let target_table_ref = self.object_name_to_table_reference(target_table_name)?; + let target_table_source = self + .context_provider + .get_table_source(target_table_ref.clone())?; + let target_schema = Arc::new(DFSchema::try_from_qualified_schema( + target_table_ref.clone(), + &target_table_source.schema(), + )?); + + // 2. Plan the source (USING clause) as a LogicalPlan + let mut planner_context = PlannerContext::new(); + let source_table_with_joins = TableWithJoins { + relation: source, + joins: vec![], + }; + let source_plan = + self.plan_from_tables(vec![source_table_with_joins], &mut planner_context)?; + + // 3. Build a combined schema for resolving expressions in ON and WHEN clauses + let combined_schema = + Arc::new(target_schema.as_ref().join(source_plan.schema())?); + + // 4. Convert the ON condition from sqlparser Expr to datafusion Expr + let on_expr = self.sql_to_expr(*on, &combined_schema, &mut planner_context)?; + + // 5. Convert each WHEN clause + let df_clauses = clauses + .into_iter() + .map(|clause| { + self.merge_clause_to_plan( + clause, + &combined_schema, + &target_schema, + &target_alias, + &mut planner_context, + ) + }) + .collect::>>()?; + + // 6. Build the DmlStatement + let plan = LogicalPlan::Dml(DmlStatement::new( + target_table_ref, + target_table_source, + WriteOp::MergeInto(Box::new(MergeIntoOp { + on: on_expr, + clauses: df_clauses, + })), + Arc::new(source_plan), + )); + + Ok(plan) + } + + fn merge_clause_to_plan( + &self, + clause: ast::MergeClause, + combined_schema: &DFSchema, + target_schema: &DFSchema, + _target_alias: &Option, + planner_context: &mut PlannerContext, + ) -> Result { + let kind = match clause.clause_kind { + ast::MergeClauseKind::Matched => MergeIntoClauseKind::Matched, + ast::MergeClauseKind::NotMatched => MergeIntoClauseKind::NotMatched, + ast::MergeClauseKind::NotMatchedByTarget => { + MergeIntoClauseKind::NotMatchedByTarget + } + ast::MergeClauseKind::NotMatchedBySource => { + MergeIntoClauseKind::NotMatchedBySource + } + }; + + let predicate = clause + .predicate + .map(|p| self.sql_to_expr(p, combined_schema, planner_context)) + .transpose()?; + + let action = match clause.action { + ast::MergeAction::Update(update_expr) => { + let assignments = update_expr + .assignments + .into_iter() + .map(|assign| { + let col_name = match &assign.target { + AssignmentTarget::ColumnName(cols) => cols + .0 + .iter() + .last() + .ok_or_else(|| plan_datafusion_err!("Empty column id"))? + .as_ident() + .unwrap() + .value + .clone(), + _ => plan_err!("Tuples are not supported")?, + }; + // Validate column exists in target + target_schema.field_with_unqualified_name(&col_name)?; + let value = self.sql_to_expr( + assign.value, + combined_schema, + planner_context, + )?; + Ok((col_name, value)) + }) + .collect::>>()?; + MergeIntoAction::Update(assignments) + } + ast::MergeAction::Insert(insert_expr) => { + let columns: Vec = insert_expr + .columns + .iter() + .map(|c| { + c.0.iter() + .last() + .map(|p| p.as_ident().unwrap().value.clone()) + .ok_or_else(|| { + plan_datafusion_err!("Empty column name in MERGE INSERT") + }) + }) + .collect::>>()?; + + let values = match insert_expr.kind { + ast::MergeInsertKind::Values(values) => { + // VALUES clause has rows; for MERGE INSERT we expect exactly one row + if values.rows.len() != 1 { + return plan_err!( + "MERGE INSERT must have exactly one row of values" + ); + } + values + .rows + .into_iter() + .next() + .unwrap() + .content + .into_iter() + .map(|v| { + self.sql_to_expr(v, combined_schema, planner_context) + }) + .collect::>>()? + } + ast::MergeInsertKind::Row => { + return not_impl_err!("MERGE INSERT ROW is not supported"); + } + }; + + MergeIntoAction::Insert { columns, values } + } + ast::MergeAction::Delete { .. } => MergeIntoAction::Delete, + }; + + Ok(MergeIntoClause { + kind, + predicate, + action, + }) + } + fn insert_to_plan( &self, table_name: ObjectName, diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index a642fbe22a6e3..a0c18c90867c7 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -65,7 +65,7 @@ tempfile = { workspace = true } testcontainers-modules = { workspace = true, features = ["postgres"], optional = true } thiserror = "2.0.18" tokio = { workspace = true } -tokio-postgres = { version = "0.7.17", optional = true } +tokio-postgres = { version = "0.7.18", optional = true } [features] avro = ["datafusion/avro"] diff --git a/datafusion/sqllogictest/test_files/date_bin_errors.slt b/datafusion/sqllogictest/test_files/date_bin_errors.slt index 20408c84ef79a..53cba506defd6 100644 --- a/datafusion/sqllogictest/test_files/date_bin_errors.slt +++ b/datafusion/sqllogictest/test_files/date_bin_errors.slt @@ -23,10 +23,24 @@ select date_bin(interval '1637426858 months', to_timestamp_millis(1040292460), t ---- NULL -# Negative timestamp with month interval - should return NULL instead of panicking +# Issue #22528: negative sub-second source with month interval. query P select date_bin(interval '1 month', to_timestamp_millis(-1040292460), timestamp '1984-01-07 00:00:00'); ---- +1969-12-07T00:00:00 + +# Array path should match the scalar path above. +query P +select date_bin(interval '1 month', c, timestamp '1984-01-07 00:00:00') +from values (to_timestamp_millis(-1040292460)) t(c); +---- +1969-12-07T00:00:00 + +# Array path should return NULL for per-row overflow. +query P +select date_bin(interval '1637426858 months', c, timestamp '1984-01-07 00:00:00') +from values (to_timestamp_millis(1040292460)) t(c); +---- NULL # Large stride causing overflow - should return NULL @@ -79,16 +93,18 @@ select date_bin( ---- NULL -# Source timestamp scaling to nanoseconds overflows: should return an error, not panic -query error DataFusion error: Execution error: DATE_BIN source timestamp 9223372036854775807 cannot be represented in nanoseconds +# Source timestamp scaling to nanoseconds overflows: should return NULL, not panic +query P select date_bin( interval '1 nanosecond', arrow_cast(9223372036854775807, 'Timestamp(Second, None)'), timestamp '1970-01-01 00:00:00' ); +---- +NULL -# Source timestamp scaling to nanoseconds overflows in array path: should return an error, not panic -query error DataFusion error: Execution error: DATE_BIN source timestamp 9223372036854775807 cannot be represented in nanoseconds +# Source timestamp scaling to nanoseconds overflows in array path: should return NULL, not panic +query P select date_bin( interval '1 nanosecond', ts, @@ -97,3 +113,5 @@ select date_bin( from ( values (arrow_cast(9223372036854775807, 'Timestamp(Second, None)')) ) as t(ts); +---- +NULL diff --git a/datafusion/sqllogictest/test_files/datetime/timestamps.slt b/datafusion/sqllogictest/test_files/datetime/timestamps.slt index 89c6f0a12139e..06740fa0f5439 100644 --- a/datafusion/sqllogictest/test_files/datetime/timestamps.slt +++ b/datafusion/sqllogictest/test_files/datetime/timestamps.slt @@ -5379,6 +5379,25 @@ SELECT to_timestamp(arrow_cast(-9223372036, 'Int64')); query error converted value exceeds the representable i64 range SELECT to_timestamp(arrow_cast(9223372037, 'Int64')); +# TRY_CAST returns NULL for timestamp/date casts that overflow +query P +SELECT TRY_CAST(arrow_cast(9223372037, 'Timestamp(s)') AS TIMESTAMP(9)); +---- +NULL + +query P +SELECT TRY_CAST(DATE '3000-01-01' AS TIMESTAMP(9)); +---- +NULL + +query P +SELECT TRY_CAST(ts AS TIMESTAMP(9)) AS ts +FROM ( + VALUES (arrow_cast(9223372037, 'Timestamp(s)')) +) t(ts); +---- +NULL + # Float truncation behavior query P SELECT to_timestamp_seconds(arrow_cast(-1.9, 'Float64')); diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index e779ce2cbffb0..e436ca795208d 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -374,14 +374,14 @@ FROM left_parquet l WHERE l.id NOT IN (SELECT r.id FROM right_parquet r); ---- logical_plan -01)LeftAnti Join: l.id = __correlated_sq_1.id +01)LeftAnti Join: l.id = __correlated_sq_1.id null_aware 02)--SubqueryAlias: l 03)----TableScan: left_parquet projection=[id, data] 04)--SubqueryAlias: __correlated_sq_1 05)----SubqueryAlias: r 06)------TableScan: right_parquet projection=[id] physical_plan -01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] +01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)], null_aware 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet 03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] @@ -469,7 +469,7 @@ ORDER BY l.id LIMIT 2; ---- logical_plan 01)Sort: l.id ASC NULLS LAST, fetch=2 -02)--LeftAnti Join: l.id = __correlated_sq_1.id +02)--LeftAnti Join: l.id = __correlated_sq_1.id null_aware 03)----SubqueryAlias: l 04)------TableScan: left_parquet projection=[id, data] 05)----SubqueryAlias: __correlated_sq_1 @@ -477,7 +477,7 @@ logical_plan 07)--------TableScan: right_parquet projection=[id] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] +02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)], null_aware 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] 04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index d8e90e294f8a3..8588c0e7ba2ae 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1100,6 +1100,31 @@ physical_plan 24)-----------------------------│ format: csv │ 25)-----------------------------└───────────────────────────┘ +# Query with null-aware anti join (NOT IN subquery). +query TT +explain select int_col from table1 where int_col not in (select int_col from table2); +---- +physical_plan +01)┌───────────────────────────┐ +02)│ HashJoinExec │ +03)│ -------------------- │ +04)│ join_type: LeftAnti │ +05)│ │ +06)│ null_aware ├──────────────┐ +07)│ │ │ +08)│ on: │ │ +09)│ (int_col = int_col) │ │ +10)└─────────────┬─────────────┘ │ +11)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +12)│ DataSourceExec ││ DataSourceExec │ +13)│ -------------------- ││ -------------------- │ +14)│ files: 1 ││ files: 1 │ +15)│ format: csv ││ format: parquet │ +16)│ ││ │ +17)│ ││ predicate: │ +18)│ ││ DynamicFilter [ empty ] │ +19)└───────────────────────────┘└───────────────────────────┘ + # Query with nested loop join. query TT explain select int_col from table1 where exists (select count(*) from table2); diff --git a/datafusion/sqllogictest/test_files/file_row_index.slt b/datafusion/sqllogictest/test_files/file_row_index.slt new file mode 100644 index 0000000000000..38822bebfdfd3 --- /dev/null +++ b/datafusion/sqllogictest/test_files/file_row_index.slt @@ -0,0 +1,171 @@ +# 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. + +statement ok +COPY (VALUES (10), (20), (30), (40), (50)) +TO 'test_files/scratch/file_row_index/parquet_table/data.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE parquet_table(column1 int) +STORED AS PARQUET +LOCATION 'test_files/scratch/file_row_index/parquet_table/'; + +query TT +EXPLAIN SELECT file_row_index(), column1 FROM parquet_table +---- +logical_plan +01)Projection: file_row_index(), parquet_table.column1 +02)--TableScan: parquet_table projection=[column1] +physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/file_row_index/parquet_table/data.parquet]]}, projection=[CAST(__datafusion_file_row_index@1 AS Int64) as file_row_index(), column1], file_type=parquet + + +query II +SELECT file_row_index(), column1 FROM parquet_table ORDER BY column1 +---- +0 10 +1 20 +2 30 +3 40 +4 50 + +query III +SELECT file_row_index(), file_row_index() + 1, column1 +FROM parquet_table +ORDER BY column1 +---- +0 1 10 +1 2 20 +2 3 30 +3 4 40 +4 5 50 + + +query II +SELECT file_row_index(), column1 +FROM parquet_table +WHERE file_row_index() > 2 +ORDER BY column1 +---- +3 40 +4 50 + +# Filter on file_row_index without having it in projection + +query TT +EXPLAIN SELECT column1 FROM parquet_table WHERE file_row_index() > 2 ORDER BY column1 +---- +logical_plan +01)Sort: parquet_table.column1 ASC NULLS LAST +02)--Projection: parquet_table.column1 +03)----Filter: __datafusion_extracted_1 > Int64(2) +04)------Projection: file_row_index() AS __datafusion_extracted_1, parquet_table.column1 +05)--------TableScan: parquet_table projection=[column1] +physical_plan +01)SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--FilterExec: __datafusion_extracted_1@0 > 2, projection=[column1@1] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/file_row_index/parquet_table/data.parquet]]}, projection=[CAST(__datafusion_file_row_index@1 AS Int64) as __datafusion_extracted_1, column1], file_type=parquet + +query I +SELECT column1 FROM parquet_table WHERE file_row_index() > 2 ORDER BY column1 +---- +40 +50 + +# Filter on file_row_index without projecting it, while enabling filter pushdown + +statement ok +SET datafusion.execution.parquet.pushdown_filters = true; + +query TT +EXPLAIN SELECT column1 FROM parquet_table WHERE file_row_index() > 2 ORDER BY column1 +---- +logical_plan +01)Sort: parquet_table.column1 ASC NULLS LAST +02)--Projection: parquet_table.column1 +03)----Filter: __datafusion_extracted_1 > Int64(2) +04)------Projection: file_row_index() AS __datafusion_extracted_1, parquet_table.column1 +05)--------TableScan: parquet_table projection=[column1] +physical_plan +01)SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[false] +02)--FilterExec: __datafusion_extracted_1@0 > 2, projection=[column1@1] +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/file_row_index/parquet_table/data.parquet]]}, projection=[CAST(__datafusion_file_row_index@1 AS Int64) as __datafusion_extracted_1, column1], file_type=parquet + +query I +SELECT column1 FROM parquet_table WHERE file_row_index() > 2 ORDER BY column1 +---- +40 +50 + +statement ok +RESET datafusion.execution.parquet.pushdown_filters; + +# Without the rewrite in ParquetSource, `file_row_index()` errors because it +# depends on file-source context + +query error file_row_index\(\) is source dependent and cannot be evaluated directly +SELECT file_row_index() + +# Testing pushdown over a source that doesn't support `file_row_index()`. + +statement ok +COPY (VALUES (10), (20), (30), (40), (50)) +TO 'test_files/scratch/file_row_index/csv_table/data.csv' +STORED AS CSV; + +statement ok +CREATE EXTERNAL TABLE csv_table(column1 int) +STORED AS CSV +LOCATION 'test_files/scratch/file_row_index/csv_table/data.csv'; + +query error file_row_index\(\) is source dependent and cannot be evaluated directly +SELECT *, file_row_index() FROM csv_table; + +# Testing a table with two files. + +statement ok +COPY (VALUES (10), (20)) +TO 'test_files/scratch/file_row_index/parquet_two_files/part-1.parquet' +STORED AS PARQUET; + +statement ok +COPY (VALUES (30), (40)) +TO 'test_files/scratch/file_row_index/parquet_two_files/part-2.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE parquet_two_files(column1 int) +STORED AS PARQUET +LOCATION 'test_files/scratch/file_row_index/parquet_two_files/'; + +query II +SELECT file_row_index(), column1 +FROM parquet_two_files +WHERE file_row_index() = 1 +ORDER BY column1 +---- +1 20 +1 40 + +statement ok +DROP TABLE parquet_two_files; + +statement ok +DROP TABLE parquet_table; + +statement ok +DROP TABLE csv_table; diff --git a/datafusion/sqllogictest/test_files/grouping.slt b/datafusion/sqllogictest/test_files/grouping.slt index eac901b2a300f..2c05dd851e61a 100644 --- a/datafusion/sqllogictest/test_files/grouping.slt +++ b/datafusion/sqllogictest/test_files/grouping.slt @@ -232,6 +232,12 @@ SELECT SUM(v1) FROM generate_series(10) AS t1(v1) WHERE false GROUP BY GROUPING ---- NULL +# grouping_sets_empty_input_avg: AVG returns NULL for the empty group +query R +SELECT AVG(v1) FROM generate_series(10) AS t1(v1) WHERE false GROUP BY GROUPING SETS(()) +---- +NULL + # grouping_sets_empty_input_count: COUNT returns 0 for the empty group, not a missing row query I SELECT COUNT(*) FROM generate_series(10) AS t1(v1) WHERE false GROUP BY GROUPING SETS(()) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ea1b0aefe9a2b..04ee70b963ceb 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -218,7 +218,7 @@ datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics true datafusion.execution.enable_ansi_mode false -datafusion.execution.enable_migration_aggregate false +datafusion.execution.enable_migration_aggregate true datafusion.execution.enable_recursive_ctes true datafusion.execution.enforce_batch_size_in_joins false datafusion.execution.hash_join_buffering_capacity 0 @@ -375,7 +375,7 @@ datafusion.execution.batch_size 8192 Default batch size while creating new batch datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics true Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. datafusion.execution.enable_ansi_mode false Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. -datafusion.execution.enable_migration_aggregate false Temporary switch for aggregate stream implementations that are being migrated from `GroupedHashAggregateStream`. When set to true, DataFusion tries the migrated implementations when their preconditions are satisfied. When set to false, grouped aggregation falls back to `GroupedHashAggregateStream`. This option will be removed after the migration is finished. See for details. +datafusion.execution.enable_migration_aggregate true Temporary switch for aggregate stream implementations that are being migrated from `GroupedHashAggregateStream`. When set to true, DataFusion tries the migrated implementations when their preconditions are satisfied. When set to false, grouped aggregation falls back to `GroupedHashAggregateStream`. This option will be removed after the migration is finished. See for details. datafusion.execution.enable_recursive_ctes true Should DataFusion support recursive CTEs datafusion.execution.enforce_batch_size_in_joins false Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. datafusion.execution.hash_join_buffering_capacity 0 How many bytes to buffer in the probe side of hash joins while the build side is concurrently being built. Without this, hash joins will wait until the full materialization of the build side before polling the probe side. This is useful in scenarios where the query is not completely CPU bounded, allowing to do some early work concurrently and reducing the latency of the query. Note that when hash join buffering is enabled, the probe side will start eagerly polling data, not giving time for the producer side of dynamic filters to produce any meaningful predicate. Queries with dynamic filters might see performance degradation. Disabled by default, set to a number greater than 0 for enabling it. diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 9be1d39d63605..082b10167274c 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1989,7 +1989,7 @@ where join_t1.t1_id + 12 not in (select join_t2.t2_id + 1 from join_t2 where join_t1.t1_int > 0) ---- logical_plan -01)LeftAnti Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int > UInt32(0) +01)LeftAnti Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int > UInt32(0) null_aware 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--SubqueryAlias: __correlated_sq_1 04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) diff --git a/datafusion/sqllogictest/test_files/math.slt b/datafusion/sqllogictest/test_files/math.slt index 1748c9b3e5d36..583d6f6777865 100644 --- a/datafusion/sqllogictest/test_files/math.slt +++ b/datafusion/sqllogictest/test_files/math.slt @@ -686,6 +686,38 @@ select gcd(-9223372036854775808, 0); query error DataFusion error: Arrow error: Compute error: Signed integer overflow in GCD\(0, \-9223372036854775808\) select gcd(0, -9223372036854775808); +# gcd decimal +query RT +select gcd(2::decimal(38, 0), 3::decimal(38, 0)), arrow_typeof(gcd(2::decimal(38, 0), 3::decimal(38, 0))); +---- +1 Decimal128(38, 0) + +query RT +select gcd(0::decimal(38, 0), 3::decimal(38, 0)), arrow_typeof(gcd(0::decimal(38, 0), 3::decimal(38, 0))); +---- +3 Decimal128(38, 0) + +query RT +select gcd(2, 3::decimal(38, 0)), arrow_typeof(gcd(2, 3::decimal(38, 0))); +---- +1 Decimal128(38, 0) + +query RR +select gcd(-15::decimal(38, 0), -3::decimal(38, 0)), gcd(-15::decimal(38, 0), 3::decimal(38, 0)); +---- +3 3 + +# non-whole number case +query RT +select gcd(15.3::decimal(38, 1), 2.9::decimal(38, 1)), arrow_typeof(gcd(15.3::decimal(38, 1), 2.9::decimal(38, 1))); +---- +0.1 Decimal128(38, 1) + +# both decimal arguments are coerced to widest - decimal(38, 5), return type is that as well +query RT +select gcd(15::decimal(30, 2), 3::decimal(38, 5)), arrow_typeof(gcd(15::decimal(30, 2), 3::decimal(38, 5))); +---- +3 Decimal128(38, 5) ## lcm @@ -727,6 +759,28 @@ select lcm(1, -9223372036854775808); query error DataFusion error: Arrow error: Compute error: Signed integer overflow in LCM\(2, 9223372036854775803\) select lcm(2, 9223372036854775803); +# lcm decimal +query R +select lcm(2::decimal(38, 0), 3::decimal(38, 0)); +---- +6 + +query RT +select lcm(0::decimal(38, 0), 3::decimal(38, 0)), arrow_typeof(lcm(0::decimal(38, 0), 3::decimal(38, 0))); +---- +0 Decimal128(38, 0) + +query RT +select lcm(2, 3::decimal(38, 0)), arrow_typeof(lcm(2, 3::decimal(38, 0))); +---- +6 Decimal128(38, 0) + +# both decimal arguments are coerced to widest - decimal(38, 5), return type is that as well +query RT +select lcm(2::decimal(30, 2), 3::decimal(38, 5)), arrow_typeof(lcm(2::decimal(30, 2), 3::decimal(38, 5))); +---- +6 Decimal128(38, 5) + ## pow/power @@ -899,6 +953,28 @@ SELECT lcm(6, column1) FROM (VALUES (4), (9), (0)); 18 0 +query I +SELECT lcm(column1, column2) FROM (VALUES (0, 5), (3, 5), (25, 5), (-16, 5)); +---- +0 +15 +25 +80 + +query R +SELECT lcm(6, arrow_cast(column1, 'Decimal128(38,0)')) FROM (VALUES (4), (9), (0)); +---- +12 +18 +0 + +query R +SELECT lcm(arrow_cast(column1, 'Decimal128(38,0)'), arrow_cast(column2, 'Decimal128(38,0)')) FROM (VALUES (6, 4), (6, 9), (6, 0)); +---- +12 +18 +0 + # lcm array and scalar with nulls in the array query I SELECT lcm(column1, 5) FROM (VALUES (0), (NULL), (25)); @@ -942,6 +1018,29 @@ SELECT gcd(15, column1) FROM (VALUES (10), (25), (0)); 5 15 +query I +SELECT gcd(column1, column2) FROM (VALUES (8, 12), (18, 12), (0, 12), (-36, 12)); +---- +4 +6 +12 +12 + +query R +SELECT gcd(15, arrow_cast(column1, 'Decimal128(38,0)')) FROM (VALUES (10), (25), (0)); +---- +5 +5 +15 + +query R +SELECT gcd(arrow_cast(column1, 'Decimal128(38,0)'), arrow_cast(column2, 'Decimal128(38,0)')) FROM (VALUES (15, 10), (15, 25), (15, 0)); +---- +5 +5 +15 + + # gcd array and scalar with nulls in the array query I SELECT gcd(column1, 12) FROM (VALUES (8), (NULL), (0), (-36)); diff --git a/datafusion/sqllogictest/test_files/null_aware_anti_join.slt b/datafusion/sqllogictest/test_files/null_aware_anti_join.slt index 5907a85a9b923..b18f3b3ae7a99 100644 --- a/datafusion/sqllogictest/test_files/null_aware_anti_join.slt +++ b/datafusion/sqllogictest/test_files/null_aware_anti_join.slt @@ -53,12 +53,12 @@ query TT EXPLAIN SELECT * FROM outer_table WHERE id NOT IN (SELECT id FROM inner_table_no_null); ---- logical_plan -01)LeftAnti Join: outer_table.id = __correlated_sq_1.id +01)LeftAnti Join: outer_table.id = __correlated_sq_1.id null_aware 02)--TableScan: outer_table projection=[id, value] 03)--SubqueryAlias: __correlated_sq_1 04)----TableScan: inner_table_no_null projection=[id] physical_plan -01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] +01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)], null_aware 02)--DataSourceExec: partitions=1, partition_sizes=[1] 03)--DataSourceExec: partitions=1, partition_sizes=[1] @@ -193,12 +193,12 @@ query TT EXPLAIN SELECT * FROM outer_table WHERE id NOT IN (SELECT id FROM inner_table_with_null); ---- logical_plan -01)LeftAnti Join: outer_table.id = __correlated_sq_1.id +01)LeftAnti Join: outer_table.id = __correlated_sq_1.id null_aware 02)--TableScan: outer_table projection=[id, value] 03)--SubqueryAlias: __correlated_sq_1 04)----TableScan: inner_table_with_null projection=[id] physical_plan -01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] +01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)], null_aware 02)--DataSourceExec: partitions=1, partition_sizes=[1] 03)--DataSourceExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index 40bfe79dcc633..b5a06bc7cb313 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -1023,6 +1023,48 @@ statement ok drop table int_probe; +######## +# Dynamic filters must not be created for null-equal joins (IS NOT DISTINCT +# FROM, INTERSECT): min/max bounds and membership filters derived from the +# build side evaluate to NULL for probe-side NULL keys and would prune rows +# that can null-match a build-side NULL. +######## + +statement ok +COPY (SELECT * FROM (VALUES (11), (22), (NULL)) v(id)) TO 'test_files/scratch/push_down_filter_parquet/nej_probe.parquet' STORED AS PARQUET; + +statement ok +COPY (SELECT * FROM (VALUES (11), (NULL)) v(id)) TO 'test_files/scratch/push_down_filter_parquet/nej_build.parquet' STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE nej_probe STORED AS PARQUET LOCATION 'test_files/scratch/push_down_filter_parquet/nej_probe.parquet'; + +statement ok +CREATE EXTERNAL TABLE nej_build STORED AS PARQUET LOCATION 'test_files/scratch/push_down_filter_parquet/nej_build.parquet'; + +# The probe-side NULL key must survive to match the build-side NULL +query II rowsort +SELECT nej_build.id, nej_probe.id FROM nej_build JOIN nej_probe ON nej_build.id IS NOT DISTINCT FROM nej_probe.id +---- +11 11 +NULL NULL + +# No DynamicFilter predicate may appear on the probe side of a null-equal join +query TT +EXPLAIN SELECT nej_build.id, nej_probe.id FROM nej_build JOIN nej_probe ON nej_build.id IS NOT DISTINCT FROM nej_probe.id +---- +physical_plan +01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], NullsEqual: true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nej_build.parquet]]}, projection=[id], file_type=parquet +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nej_probe.parquet]]}, projection=[id], file_type=parquet + +statement ok +drop table nej_build; + +statement ok +drop table nej_probe; + + # Config reset statement ok RESET datafusion.explain.physical_plan_only; diff --git a/datafusion/sqllogictest/test_files/spark/datetime/next_day.slt b/datafusion/sqllogictest/test_files/spark/datetime/next_day.slt index 872d1f2b58eb6..b0ffd7d0e412f 100644 --- a/datafusion/sqllogictest/test_files/spark/datetime/next_day.slt +++ b/datafusion/sqllogictest/test_files/spark/datetime/next_day.slt @@ -36,6 +36,12 @@ SELECT next_day('2015-07-27'::DATE, 'Sat'::string); ---- 2015-08-01 +# Whitespace-padded day names should be rejected (return NULL) per Spark behavior +query D +SELECT next_day('2015-01-14'::DATE, ' MO '::string); +---- +NULL + query error Failed to coerce arguments to satisfy a call to 'next_day' function SELECT next_day('2015-07-27'::DATE); diff --git a/datafusion/sqllogictest/test_files/spark/string/concat_ws.slt b/datafusion/sqllogictest/test_files/spark/string/concat_ws.slt index 62df636bba9ce..f6404cab8f3cd 100644 --- a/datafusion/sqllogictest/test_files/spark/string/concat_ws.slt +++ b/datafusion/sqllogictest/test_files/spark/string/concat_ws.slt @@ -21,22 +21,367 @@ # For more information, please see: # https://github.com/apache/datafusion/issues/15914 -## Original Query: SELECT concat_ws(' ', 'Spark', 'SQL'); -## PySpark 3.5.5 Result: {'concat_ws( , Spark, SQL)': 'Spark SQL', 'typeof(concat_ws( , Spark, SQL))': 'string', 'typeof( )': 'string', 'typeof(Spark)': 'string', 'typeof(SQL)': 'string'} -#query -#SELECT concat_ws(' '::string, 'Spark'::string, 'SQL'::string); - -## Original Query: SELECT concat_ws('/', 'foo', null, 'bar'); -## PySpark 3.5.5 Result: {'concat_ws(/, foo, NULL, bar)': 'foo/bar', 'typeof(concat_ws(/, foo, NULL, bar))': 'string', 'typeof(/)': 'string', 'typeof(foo)': 'string', 'typeof(NULL)': 'void', 'typeof(bar)': 'string'} -#query -#SELECT concat_ws('/'::string, 'foo'::string, NULL::void, 'bar'::string); - -## Original Query: SELECT concat_ws('s'); -## PySpark 3.5.5 Result: {'concat_ws(s)': '', 'typeof(concat_ws(s))': 'string', 'typeof(s)': 'string'} -#query -#SELECT concat_ws('s'::string); - -## Original Query: SELECT concat_ws(null, 'Spark', 'SQL'); -## PySpark 3.5.5 Result: {'concat_ws(NULL, Spark, SQL)': None, 'typeof(concat_ws(NULL, Spark, SQL))': 'string', 'typeof(NULL)': 'void', 'typeof(Spark)': 'string', 'typeof(SQL)': 'string'} -#query -#SELECT concat_ws(NULL::void, 'Spark'::string, 'SQL'::string); +## ── Basic scalar usage ────────────────────────────────────── + +## Multiple string arguments +query T +SELECT concat_ws(',', 'a', 'b', 'c'); +---- +a,b,c + +## Space separator +query T +SELECT concat_ws(' ', 'Spark', 'SQL'); +---- +Spark SQL + +## Slash separator with null skipped +query T +SELECT concat_ws('/', 'foo', NULL, 'bar'); +---- +foo/bar + +## Single argument after separator +query T +SELECT concat_ws(',', 'a'); +---- +a + +## No arguments after separator → empty string +query T +SELECT concat_ws(','); +---- +(empty) + +## Null separator returns null +query T +SELECT concat_ws(NULL, 'a', 'b', 'c'); +---- +NULL + +## All null arguments → empty string +query T +SELECT concat_ws(',', CAST(NULL AS STRING), CAST(NULL AS STRING)); +---- +(empty) + +## ── Array arguments ───────────────────────────────────────── + +## Array argument +query T +SELECT concat_ws(',', array('a', 'b', 'c')); +---- +a,b,c + +## Array with nulls skipped +query T +SELECT concat_ws(',', array('a', NULL, 'c')); +---- +a,c + +## Multiple arrays +query T +SELECT concat_ws(',', array('a', 'b'), array('c', 'd')); +---- +a,b,c,d + +## Mixed scalar and array arguments +query T +SELECT concat_ws(',', 'x', array('a', 'b'), 'y'); +---- +x,a,b,y + +## Null array is skipped +query T +SELECT concat_ws(',', 'x', CAST(NULL AS ARRAY), 'y'); +---- +x,y + +## ── Edge cases ─────────────────────────────────────────────── + +## Separator column with no value arguments +query T +SELECT concat_ws(sep) AS result FROM VALUES (','), ('-') AS t(sep); +---- +(empty) +(empty) + +## Null separator in column with no value arguments +query T +SELECT concat_ws(sep) AS result FROM VALUES (CAST(NULL AS STRING)), (',') AS t(sep); +---- +NULL +(empty) + +## ── Column expressions ────────────────────────────────────── + +## concat_ws on columns +query T +SELECT concat_ws('-', a, b) AS result FROM VALUES ('hello', 'world'), ('foo', 'bar') AS t(a, b); +---- +hello-world +foo-bar + +## concat_ws with null in columns +query T +SELECT concat_ws(',', a, b) AS result FROM VALUES ('a', 'b'), ('c', CAST(NULL AS STRING)), (CAST(NULL AS STRING), 'd') AS t(a, b); +---- +a,b +c +d + +## Scalar-only arguments over multiple rows (broadcast test) +query T +SELECT concat_ws(',', 'a', 'b') AS result FROM VALUES (1), (2), (3) AS t(x); +---- +a,b +a,b +a,b + +## ── Additional edge cases ─────────────────────────────────── + +## Empty separator — values concatenated with nothing between +query T +SELECT concat_ws('', 'a', 'b', 'c'); +---- +abc + +## Empty-string values are NOT skipped (only NULLs are) +query T +SELECT concat_ws(',', '', 'a', '', 'b'); +---- +,a,,b + +## Multi-character separator +query T +SELECT concat_ws(' - ', 'a', 'b', 'c'); +---- +a - b - c + +## Utf8View separator +query TT +SELECT concat_ws(arrow_cast(',', 'Utf8View'), 'a', 'b'), arrow_typeof(concat_ws(arrow_cast(',', 'Utf8View'), 'a', 'b')); +---- +a,b Utf8 + +## LargeUtf8 separator +query TT +SELECT concat_ws(arrow_cast(',', 'LargeUtf8'), 'a', 'b'), arrow_typeof(concat_ws(arrow_cast(',', 'LargeUtf8'), 'a', 'b')); +---- +a,b Utf8 + +## Empty array → empty string +query T +SELECT concat_ws(',', array()); +---- +(empty) + +## Scalar + array + array mix +query T +SELECT concat_ws(',', array('a', 'b'), 'c', array('d', 'e')); +---- +a,b,c,d,e + +## All-NULL row mixed with non-NULL rows +query T +SELECT concat_ws(',', a, b, c) AS result FROM VALUES + ('a', 'b', 'c'), + (CAST(NULL AS STRING), 'b', 'c'), + ('a', CAST(NULL AS STRING), CAST(NULL AS STRING)), + (CAST(NULL AS STRING), CAST(NULL AS STRING), CAST(NULL AS STRING)) + AS t(a, b, c); +---- +a,b,c +b,c +a +(empty) + +## Separator from column (per-row separator), with NULL rows +query T +SELECT concat_ws(sep, a, b) AS result FROM VALUES + (',', 'a', 'b'), + (CAST(NULL AS STRING), 'a', 'b'), + ('|', 'x', 'y') + AS t(sep, a, b); +---- +a,b +NULL +x|y + +## ── Spark cross-checked extras ────────────────────────────── + +## Zero arguments → error (Spark: WRONG_NUM_ARGS) +query error +SELECT concat_ws(); + +## Numeric separator coerced to string +query T +SELECT concat_ws(1, 'a', 'b'); +---- +a1b + +## Only numeric separator, no values → empty string +query T +SELECT concat_ws(123); +---- +(empty) + +## Numeric values coerced to string +query T +SELECT concat_ws(',', 1, 2, 3); +---- +1,2,3 + +## Float values +query T +SELECT concat_ws(',', 1.5, 2.5); +---- +1.5,2.5 + +## Boolean values +query T +SELECT concat_ws(',', true, false); +---- +true,false + +## Mixed numeric and string +query T +SELECT concat_ws(',', CAST(1 AS BIGINT), 'a'); +---- +1,a + +## Date values +query T +SELECT concat_ws(',', DATE '2024-01-01', 'x'); +---- +2024-01-01,x + +## Multi-byte UTF-8 separator +query T +SELECT concat_ws('é', 'a', 'b'); +---- +aéb + +## Nested concat_ws +query T +SELECT concat_ws('|', concat_ws(',', 'a', 'b'), concat_ws(',', 'c', 'd')); +---- +a,b|c,d + +## All-NULL elements in array → empty string +query T +SELECT concat_ws(',', array(CAST(NULL AS STRING), CAST(NULL AS STRING), CAST(NULL AS STRING))); +---- +(empty) + +## Empty-string elements in arrays are NOT skipped +query T +SELECT concat_ws(',', array(''), array('')); +---- +, + +## Multiple arrays interleaved with scalars and NULLs +query T +SELECT concat_ws(',', array('a', 'b'), 'c', CAST(NULL AS STRING), array('d'), '', 'e'); +---- +a,b,c,d,,e + +## Long argument list (variadic) +query T +SELECT concat_ws('-', 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j'); +---- +a-b-c-d-e-f-g-h-i-j + +## Long array +query T +SELECT concat_ws(',', array('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j')); +---- +a,b,c,d,e,f,g,h,i,j + +## All-empty arguments +query T +SELECT concat_ws('', '', '', ''); +---- +(empty) + +## Empty separator with all NULLs +query T +SELECT concat_ws('', CAST(NULL AS STRING), CAST(NULL AS STRING)); +---- +(empty) + +## Long string preserved (length sanity) +query I +SELECT length(concat_ws(',', repeat('x', 1000), repeat('y', 1000))); +---- +2001 + +## ── List variants (FixedSizeList / ListView) ──────────────── + +## FixedSizeList argument is expanded element-by-element +query T +SELECT concat_ws(',', arrow_cast(make_array('a', 'b', 'c'), 'FixedSizeList(3, Utf8)')); +---- +a,b,c + +## ListView argument is expanded element-by-element +query T +SELECT concat_ws(',', arrow_cast(make_array('a', 'b'), 'ListView(Utf8)')); +---- +a,b + +## LargeListView argument is expanded element-by-element +query T +SELECT concat_ws(',', arrow_cast(make_array('a', 'b'), 'LargeListView(Utf8)')); +---- +a,b + +## ── Binary coercion (Spark casts binary to its UTF-8 view) ── + +## Binary argument is coerced to its string representation +query T +SELECT concat_ws(',', X'4869'); +---- +Hi + +## ── Null-separator-over-rows shape ────────────────────────── + +## Null separator on a multi-row column yields NULL on every row +query T +SELECT concat_ws(NULL, v) AS result FROM VALUES ('a'), ('b'), ('c') AS t(v) ORDER BY v; +---- +NULL +NULL +NULL + +## ── Non-string list elements (planner-inserted element cast) ─ + +## Array of integers — elements must be cast to STRING +query T +SELECT concat_ws(',', array(1, 2, 3)); +---- +1,2,3 + +## Array of doubles +query T +SELECT concat_ws('-', array(1.5, 2.5, 3.5)); +---- +1.5-2.5-3.5 + +## Array of booleans +query T +SELECT concat_ws(',', array(true, false, true)); +---- +true,false,true + +## Mixed: string scalar + int array + string scalar +query T +SELECT concat_ws(',', 'x', array(1, 2), 'y'); +---- +x,1,2,y + +## ── Struct rejection ──────────────────────────────────────── + +## Struct argument is rejected (not coerced to string) +query error +SELECT concat_ws(',', named_struct('a', 1)); diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index 982e2c6f4acce..a0e47a8691f34 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -1714,3 +1714,22 @@ RESET datafusion.sql_parser.dialect; statement ok drop table t_agg_window; + +# extract_leaf_expressions regression +statement ok +create table leaf_base as select named_struct('status', 'active') as s, 1 as id; + +statement ok +create view leaf_view as select s, id, id + 1 as synth from leaf_base; + +query T?I +select s['status'], s, id from leaf_view where s['status'] is not null; +---- +active {status: active} 1 + +statement ok +drop view leaf_view; + +statement ok +drop table leaf_base; + diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 8d8eb0ed11828..ab830714b1dde 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -54,7 +54,7 @@ logical_plan 02)--Projection: part.p_brand, part.p_type, part.p_size, count(alias1) AS supplier_cnt 03)----Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[count(alias1)]] 04)------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] -05)--------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey +05)--------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey null_aware 06)----------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size 07)------------Inner Join: partsupp.ps_partkey = part.p_partkey 08)--------------TableScan: partsupp projection=[ps_partkey, ps_suppkey] @@ -74,7 +74,7 @@ physical_plan 07)------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] 08)--------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] -10)------------------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] +10)------------------HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)], null_aware 11)--------------------CoalescePartitionsExec 12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] 13)------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 1b51950a70e1b..090f44f5628f7 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -5981,6 +5981,41 @@ FROM table_distinct_sum_nulls; 5 5 +# SUM(DISTINCT) over sliding (bounded) window frames is only implemented +# for Int64. Other SUM-supported input types must fail with a clear +# capability error instead of an accumulator-internal one. +statement ok +CREATE TABLE table_distinct_sum_types(ts INT, f DOUBLE, d DECIMAL(10, 2)) AS VALUES + (1, 1.5, 1.50), (2, 2.5, 2.50), (3, 1.5, 1.50); + +query error DataFusion error: This feature is not implemented: SUM\(DISTINCT\) over sliding window frames is only supported for Int64, got Float64 +SELECT SUM(DISTINCT f) OVER ( + ORDER BY ts ROWS BETWEEN 1 PRECEDING AND CURRENT ROW +) FROM table_distinct_sum_types; + +query error DataFusion error: This feature is not implemented: SUM\(DISTINCT\) over sliding window frames is only supported for Int64, got Decimal128\(10, 2\) +SELECT SUM(DISTINCT d) OVER ( + ORDER BY ts ROWS BETWEEN 1 PRECEDING AND CURRENT ROW +) FROM table_distinct_sum_types; + +query error DataFusion error: This feature is not implemented: SUM\(DISTINCT\) over sliding window frames is only supported for Int64, got UInt64 +SELECT SUM(DISTINCT arrow_cast(ts, 'UInt64')) OVER ( + ORDER BY ts ROWS BETWEEN 1 PRECEDING AND CURRENT ROW +) FROM table_distinct_sum_types; + +# Unbounded frames take the regular distinct-sum path and keep +# supporting all SUM input types. +query R +SELECT SUM(DISTINCT f) OVER (ORDER BY ts) FROM table_distinct_sum_types; +---- +1.5 +4 +4 + +statement ok +DROP TABLE table_distinct_sum_types; + + # FILTER clause with window functions # Verify FILTER clause with non-aggregate window functions fails with a clear message @@ -6626,6 +6661,142 @@ ORDER BY i; 3 1 4 NULL +# Covariance/correlation sliding-window regression test. Verifies correct +# results across row removals and a NULL-gap empty-frame transition. +query IRRR +SELECT + column1, + covar_pop(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ), + covar_samp(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ), + corr(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ) +FROM ( + VALUES + (1, 10.0, 5.0), + (2, NULL, NULL), + (3, NULL, NULL), + (4, 30.0, 10.0), + (5, 40.0, 20.0), + (6, 50.0, 10.0) +); +---- +1 0 NULL NULL +2 0 NULL NULL +3 NULL NULL NULL +4 0 NULL NULL +5 25 50 1 +6 -25 -50 -1 + +# Multi-row covariance/correlation sliding-window regression test. Verifies +# correct accumulation when valid rows enter the frame after a reset. +query IRRR +SELECT + column1, + covar_pop(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 2 PRECEDING AND CURRENT ROW + ), + covar_samp(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 2 PRECEDING AND CURRENT ROW + ), + corr(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 2 PRECEDING AND CURRENT ROW + ) +FROM ( + VALUES + (1, 10.0, 5.0), + (2, NULL, NULL), + (3, NULL, NULL), + (4, 30.0, 10.0), + (5, 40.0, 20.0), + (6, 50.0, 10.0) +); +---- +1 0 NULL NULL +2 0 NULL NULL +3 0 NULL NULL +4 0 NULL NULL +5 25 50 1 +6 0 0 0 + +# Covariance/correlation sliding-window regression test. Rows with NULL in +# either input column must not contribute to the aggregate state. +query IRRR +SELECT + column1, + covar_pop(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 3 PRECEDING AND CURRENT ROW + ), + covar_samp(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 3 PRECEDING AND CURRENT ROW + ), + corr(column2, column3) OVER ( + ORDER BY column1 + ROWS BETWEEN 3 PRECEDING AND CURRENT ROW + ) +FROM ( + VALUES + (1, 10.0, 5.0), + (2, 20.0, NULL), + (3, NULL, 15.0), + (4, 30.0, 10.0), + (5, 40.0, 20.0) +); +---- +1 0 NULL NULL +2 0 NULL NULL +3 0 NULL NULL +4 25 50 1 +5 25 50 1 + +# Variance/stddev sliding-window regression test. Verifies that retracting +# the last valid row resets the aggregate state. +query IRRRR +SELECT + column1, + var_pop(column2) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ), + var_samp(column2) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ), + stddev_pop(column2) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ), + stddev_samp(column2) OVER ( + ORDER BY column1 + ROWS BETWEEN 1 PRECEDING AND CURRENT ROW + ) +FROM ( + VALUES + (1, 10.0), + (2, NULL), + (3, NULL), + (4, 30.0), + (5, 40.0) +); +---- +1 0 NULL 0 NULL +2 0 NULL 0 NULL +3 NULL NULL NULL NULL +4 0 NULL 0 NULL +5 25 50 5 7.071067811865 + # Decimal variant — the integer-division path would otherwise panic on an # empty frame. query IR diff --git a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json index c476ea76347ab..526853d841421 100644 --- a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json +++ b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json @@ -2431,14 +2431,13 @@ } }, "node_modules/launch-editor": { - "version": "2.10.0", - "resolved": "https://registry.npmjs.org/launch-editor/-/launch-editor-2.10.0.tgz", - "integrity": "sha512-D7dBRJo/qcGX9xlvt/6wUYzQxjh5G1RvZPgPv8vi4KRU99DVQL/oW7tnVOCCTm2HGeo3C5HvGE5Yrh6UBoZ0vA==", + "version": "2.14.1", + "resolved": "https://registry.npmjs.org/launch-editor/-/launch-editor-2.14.1.tgz", + "integrity": "sha512-QWBrQsMpH7gPr965dsKD/3cKWiNoTjpATQf++Xq63N6sKRGMwlVXz41O1IZTMfZQgBctD/K5Zt06+/I6pP6+HA==", "dev": true, - "license": "MIT", "dependencies": { - "picocolors": "^1.0.0", - "shell-quote": "^1.8.1" + "picocolors": "^1.1.1", + "shell-quote": "^1.8.4" } }, "node_modules/loader-runner": { @@ -6028,13 +6027,13 @@ "dev": true }, "launch-editor": { - "version": "2.10.0", - "resolved": "https://registry.npmjs.org/launch-editor/-/launch-editor-2.10.0.tgz", - "integrity": "sha512-D7dBRJo/qcGX9xlvt/6wUYzQxjh5G1RvZPgPv8vi4KRU99DVQL/oW7tnVOCCTm2HGeo3C5HvGE5Yrh6UBoZ0vA==", + "version": "2.14.1", + "resolved": "https://registry.npmjs.org/launch-editor/-/launch-editor-2.14.1.tgz", + "integrity": "sha512-QWBrQsMpH7gPr965dsKD/3cKWiNoTjpATQf++Xq63N6sKRGMwlVXz41O1IZTMfZQgBctD/K5Zt06+/I6pP6+HA==", "dev": true, "requires": { - "picocolors": "^1.0.0", - "shell-quote": "^1.8.1" + "picocolors": "^1.1.1", + "shell-quote": "^1.8.4" } }, "loader-runner": { diff --git a/dev/depcheck/.gitignore b/dev/depcheck/.gitignore deleted file mode 100644 index 03314f77b5aa4..0000000000000 --- a/dev/depcheck/.gitignore +++ /dev/null @@ -1 +0,0 @@ -Cargo.lock diff --git a/dev/depcheck/Cargo.lock b/dev/depcheck/Cargo.lock new file mode 100644 index 0000000000000..3018c79c5a827 --- /dev/null +++ b/dev/depcheck/Cargo.lock @@ -0,0 +1,4167 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 4 + +[[package]] +name = "adler2" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" + +[[package]] +name = "aho-corasick" +version = "1.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddd31a130427c27518df266943a5308ed92d4b226cc639f5a8f1002816174301" +dependencies = [ + "memchr", +] + +[[package]] +name = "allocator-api2" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "annotate-snippets" +version = "0.12.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f211a51805bc641f3ad5b7664c77d2547af685cc33b4cd8d31964027a46f13f1" +dependencies = [ + "anstyle", + "memchr", + "unicode-width", +] + +[[package]] +name = "anstream" +version = "0.6.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43d5b281e737544384e969a5ccad3f1cdd24b48086a0fc1b2a5262a26b8f4f4a" +dependencies = [ + "anstyle", + "anstyle-parse 0.2.7", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is_terminal_polyfill", + "utf8parse", +] + +[[package]] +name = "anstream" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "824a212faf96e9acacdbd09febd34438f8f711fb84e09a8916013cd7815ca28d" +dependencies = [ + "anstyle", + "anstyle-parse 1.0.0", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is_terminal_polyfill", + "utf8parse", +] + +[[package]] +name = "anstyle" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "940b3a0ca603d1eade50a4846a2afffd5ef57a9feac2c0e2ec2e14f9ead76000" + +[[package]] +name = "anstyle-parse" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e7644824f0aa2c7b9384579234ef10eb7efb6a0deb83f9630a49594dd9c15c2" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-parse" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52ce7f38b242319f7cabaa6813055467063ecdc9d355bbb4ce0c68908cd8130e" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40c48f72fd53cd289104fc64099abca73db4166ad86ea0b4341abe65af83dadc" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "291e6a250ff86cd4a820112fb8898808a366d8f9f58ce16d1f538353ad55747d" +dependencies = [ + "anstyle", + "once_cell_polyfill", + "windows-sys 0.61.2", +] + +[[package]] +name = "anyhow" +version = "1.0.102" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f202df86484c868dbad7eaa557ef785d5c66295e41b460ef922eca0723b842c" + +[[package]] +name = "arc-swap" +version = "1.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a3a1fd6f75306b68087b831f025c712524bcb19aad54e557b1129cfa0a2b207" +dependencies = [ + "rustversion", +] + +[[package]] +name = "arrayref" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76a2e8124351fda1ef8aaaa3bbd7ebbcb486bbcd4225aca0aa0d84bb2db8fecb" + +[[package]] +name = "arrayvec" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" + +[[package]] +name = "autocfg" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2032f911046de80f0a198e0901378627c33f59ea0ac00e363d481118bd70a53" + +[[package]] +name = "base16ct" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c7f02d4ea65f2c1853089ffd8d2787bdbc63de2f0d29dedbcf8ccdfa0ccd4cf" + +[[package]] +name = "base64" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" + +[[package]] +name = "base64ct" +version = "1.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2af50177e190e07a26ab74f8b1efbfe2ef87da2116221318cb1c2e82baf7de06" + +[[package]] +name = "bitflags" +version = "2.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4388bee8683e3d04af747c73422af53102d2bd24d9eadb6cbc100baef4b43f8" + +[[package]] +name = "bitmaps" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "031043d04099746d8db04daf1fa424b2bc8bd69d92b25962dcde24da39ab64a2" +dependencies = [ + "typenum", +] + +[[package]] +name = "blake3" +version = "1.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0aa83c34e62843d924f905e0f5c866eb1dd6545fc4d719e803d9ba6030371fce" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", + "cpufeatures 0.3.0", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "block2" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cdeb9d870516001442e364c5220d3574d2da8dc765554b4a617230d33fa58ef5" +dependencies = [ + "objc2", +] + +[[package]] +name = "bstr" +version = "1.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63044e1ae8e69f3b5a92c736ca6269b8d12fa7efe39bf34ddb06d102cf0e2cab" +dependencies = [ + "memchr", + "regex-automata", + "serde", +] + +[[package]] +name = "bumpalo" +version = "3.20.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72f5acc6cb2ba439de613abc23857ec3d78374d8ed5ac84e9d11336e87da8649" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e748733b7cbc798e1434b6ac524f0c1ff2ab456fe201501e6497c8417a4fc33" + +[[package]] +name = "cargo" +version = "0.92.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89389877f508bae1d45a48b4e76cb0dac5e41a6ac5103752e44d29be0c69c394" +dependencies = [ + "annotate-snippets", + "anstream 0.6.21", + "anstyle", + "anyhow", + "base64", + "blake3", + "cargo-credential", + "cargo-credential-libsecret", + "cargo-credential-macos-keychain", + "cargo-credential-wincred", + "cargo-platform", + "cargo-util", + "cargo-util-schemas", + "clap", + "clap_complete", + "color-print", + "crates-io", + "curl", + "curl-sys", + "filetime", + "flate2", + "git2", + "git2-curl", + "gix", + "glob", + "hex", + "hmac", + "home", + "http-auth", + "ignore", + "im-rc", + "indexmap", + "itertools", + "jiff", + "jobserver", + "lazycell", + "libc", + "libgit2-sys", + "memchr", + "opener", + "os_info", + "pasetors", + "pathdiff", + "rand", + "regex", + "rusqlite", + "rustc-hash", + "rustc-stable-hash", + "rustfix", + "same-file", + "semver", + "serde", + "serde-untagged", + "serde_ignored", + "serde_json", + "sha1", + "shell-escape", + "supports-hyperlinks", + "supports-unicode", + "tar", + "tempfile", + "thiserror", + "time", + "toml", + "toml_edit", + "tracing", + "tracing-chrome", + "tracing-subscriber", + "unicase", + "unicode-width", + "unicode-xid", + "url", + "walkdir", + "windows-sys 0.60.2", + "winnow 0.7.15", +] + +[[package]] +name = "cargo-credential" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e36f089041deadf16226478a7737a833864fbda09408c7af237b9d615eeb6d69" +dependencies = [ + "anyhow", + "libc", + "serde", + "serde_json", + "thiserror", + "time", + "windows-sys 0.60.2", +] + +[[package]] +name = "cargo-credential-libsecret" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90161b8b1b98a28f0fbdfccafb6adcf2b0be948a4fad3acc31461abf5447debe" +dependencies = [ + "anyhow", + "cargo-credential", + "libloading", +] + +[[package]] +name = "cargo-credential-macos-keychain" +version = "0.4.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e95b9c2431165b30ea111f2933ed6799bfa9a66c9503046064cf8f001960ea1b" +dependencies = [ + "cargo-credential", + "security-framework", +] + +[[package]] +name = "cargo-credential-wincred" +version = "0.4.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c35397b066a83f2e036fb23fca2fb400bfa65e8e8453c21e0b1690cf8250e414" +dependencies = [ + "cargo-credential", + "windows-sys 0.60.2", +] + +[[package]] +name = "cargo-platform" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87a0c0e6148f11f01f32650a2ea02d532b2ad4e81d8bd41e6e565b5adc5e6082" +dependencies = [ + "serde", + "serde_core", +] + +[[package]] +name = "cargo-util" +version = "0.2.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f97c9ef0f8af69bfcecfe4c17a414d7bb978fe794bc1a38952e27b5c5d87492d" +dependencies = [ + "anyhow", + "core-foundation", + "filetime", + "hex", + "ignore", + "jobserver", + "libc", + "miow", + "same-file", + "sha2", + "shell-escape", + "tempfile", + "tracing", + "walkdir", + "windows-sys 0.60.2", +] + +[[package]] +name = "cargo-util-schemas" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "549c00f5bb23fdaf26135d747d7530563402a101f1887a5a1916afe2c09cf229" +dependencies = [ + "semver", + "serde", + "serde-untagged", + "serde-value", + "thiserror", + "toml", + "unicode-xid", + "url", +] + +[[package]] +name = "cc" +version = "1.2.64" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dad887fd958be91b5098c0248def011f4523ab786cd411be668777e55063501f" +dependencies = [ + "find-msvc-tools", + "jobserver", + "libc", + "shlex 2.0.1", +] + +[[package]] +name = "cfg-if" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9330f8b2ff13f34540b44e946ef35111825727b38d33286ef986142615121801" + +[[package]] +name = "cfg_aliases" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" + +[[package]] +name = "clap" +version = "4.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ddb117e43bbf7dacf0a4190fef4d345b9bad68dfc649cb349e7d17d28428e51" +dependencies = [ + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "714a53001bf66416adb0e2ef5ac857140e7dc3a0c48fb28b2f10762fc4b5069f" +dependencies = [ + "anstream 1.0.0", + "anstyle", + "clap_lex", + "strsim", + "terminal_size", +] + +[[package]] +name = "clap_complete" +version = "4.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0a7a9bfdb35811f9e59832f0f05975114d2251b415fb534108e6f34060fd772" +dependencies = [ + "clap", + "clap_lex", + "is_executable", + "shlex 1.3.0", +] + +[[package]] +name = "clap_lex" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8d4a3bb8b1e0c1050499d1815f5ab16d04f0959b233085fb31653fbfc9d98f9" + +[[package]] +name = "clru" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "197fd99cb113a8d5d9b6376f3aa817f32c1078f2343b714fff7d2ca44fdf67d5" +dependencies = [ + "hashbrown 0.16.1", +] + +[[package]] +name = "color-print" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3aa954171903797d5623e047d9ab69d91b493657917bdfb8c2c80ecaf9cdb6f4" +dependencies = [ + "color-print-proc-macro", +] + +[[package]] +name = "color-print-proc-macro" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "692186b5ebe54007e45a59aea47ece9eb4108e141326c304cdc91699a7118a22" +dependencies = [ + "nom", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "colorchoice" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d07550c9036bf2ae0c684c4297d503f838287c83c53686d05370d0e139ae570" + +[[package]] +name = "const-oid" +version = "0.9.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" + +[[package]] +name = "constant_time_eq" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d52eff69cd5e647efe296129160853a42795992097e8af39800e1060caeea9b" + +[[package]] +name = "core-foundation" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2a6cd9ae233e7f62ba4e9353e81a88df7fc8a5987b8d445b4d90c879bd156f6" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "core-foundation-sys" +version = "0.8.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" + +[[package]] +name = "cpufeatures" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "59ed5838eebb26a2bb2e58f6d5b5316989ae9d08bab10e0e6d103e656d1b0280" +dependencies = [ + "libc", +] + +[[package]] +name = "cpufeatures" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b2a41393f66f16b0823bb79094d54ac5fbd34ab292ddafb9a0456ac9f87d201" +dependencies = [ + "libc", +] + +[[package]] +name = "crates-io" +version = "0.40.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "574ce0b8170c097cf174097b84bff181956ad2ab2bbe092ab58d1c08d9f1f417" +dependencies = [ + "curl", + "percent-encoding", + "serde", + "serde_json", + "thiserror", + "url", +] + +[[package]] +name = "crc32fast" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9481c1c90cbf2ac953f07c8d4a58aa3945c425b7185c9154d67a65e4230da511" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "crossbeam-channel" +version = "0.5.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "82b8f8f868b36967f9606790d1903570de9ceaf870a7bf9fbbd3016d636a2cb2" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" + +[[package]] +name = "crypto-bigint" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0dc92fb57ca44df6db8059111ab3af99a63d5d0f8375d9972e319a379c6bab76" +dependencies = [ + "generic-array", + "rand_core 0.6.4", + "subtle", + "zeroize", +] + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "ct-codecs" +version = "1.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49fb0c6640b4507ebd99ff67677009e381ba5eee1d14df78de4a3d16eb123c39" + +[[package]] +name = "curl" +version = "0.4.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a45ee8994e5307cb4c60cfc1c20bf7263ffb771ddc135c9f768a14bcbc15b09" +dependencies = [ + "curl-sys", + "libc", + "openssl-probe", + "openssl-sys", + "schannel", + "socket2", + "windows-sys 0.61.2", +] + +[[package]] +name = "curl-sys" +version = "0.4.89+curl-8.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d680779285438f2d0927485973ab45b212ea990bddb80de8a55a1e3c1d9ba22" +dependencies = [ + "cc", + "libc", + "libnghttp2-sys", + "libz-sys", + "openssl-sys", + "pkg-config", + "vcpkg", + "windows-sys 0.61.2", +] + +[[package]] +name = "dashmap" +version = "6.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6361d5c062261c78a176addb82d4c821ae42bed6089de0e12603cd25de2059c" +dependencies = [ + "cfg-if", + "crossbeam-utils", + "hashbrown 0.14.5", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "depcheck" +version = "0.0.0" +dependencies = [ + "cargo", +] + +[[package]] +name = "der" +version = "0.7.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7c1832837b905bbfb5101e07cc24c8deddf52f93225eee6ead5f4d63d53ddcb" +dependencies = [ + "const-oid", + "pem-rfc7468", + "zeroize", +] + +[[package]] +name = "deranged" +version = "0.5.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7cd812cc2bc1d69d4764bd80df88b4317eaef9e773c75226407d9bc0876b211c" +dependencies = [ + "powerfmt", + "serde_core", +] + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "const-oid", + "crypto-common", + "subtle", +] + +[[package]] +name = "dispatch2" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e0e367e4e7da84520dedcac1901e4da967309406d1e51017ae1abfb97adbd38" +dependencies = [ + "bitflags", + "objc2", +] + +[[package]] +name = "displaydoc" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ac70aa55017e108007fbaf5aa0f54b021c98f92ff8af59d42eda9da96e3dd4f" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "dunce" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" + +[[package]] +name = "ecdsa" +version = "0.16.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee27f32b5c5292967d2d4a9d7f1e0b0aed2c15daded5a60300e4abb9d8020bca" +dependencies = [ + "der", + "digest", + "elliptic-curve", + "rfc6979", + "signature", + "spki", +] + +[[package]] +name = "ed25519-compact" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f5c0284a5d4b1a2fae017a9fe55fd7d01699711f1b572493f16593e173ea2801" +dependencies = [ + "getrandom 0.4.2", +] + +[[package]] +name = "either" +version = "1.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91622ff5e7162018101f2fea40d6ebf4a78bbe5a49736a2020649edf9693679e" + +[[package]] +name = "elliptic-curve" +version = "0.13.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5e6043086bf7973472e0c7dff2142ea0b680d30e18d9cc40f267efbf222bd47" +dependencies = [ + "base16ct", + "crypto-bigint", + "digest", + "ff", + "generic-array", + "group", + "hkdf", + "pem-rfc7468", + "pkcs8", + "rand_core 0.6.4", + "sec1", + "subtle", + "zeroize", +] + +[[package]] +name = "encoding_rs" +version = "0.8.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75030f3c4f45dafd7586dd6780965a8c7e8e285a5ecb86713e63a79c5b2766f3" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "equivalent" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" + +[[package]] +name = "erased-serde" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2add8a07dd6a8d93ff627029c51de145e12686fbc36ecb298ac22e74cf02dec" +dependencies = [ + "serde", + "serde_core", + "typeid", +] + +[[package]] +name = "errno" +version = "0.3.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" +dependencies = [ + "libc", + "windows-sys 0.61.2", +] + +[[package]] +name = "fallible-iterator" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2acce4a10f12dc2fb14a218589d4f1f62ef011b2d0cc4b3cb1bba8e94da14649" + +[[package]] +name = "fallible-streaming-iterator" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7360491ce676a36bf9bb3c56c1aa791658183a54d2744120f27285738d90465a" + +[[package]] +name = "faster-hex" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7223ae2d2f179b803433d9c830478527e92b8117eab39460edae7f1614d9fb73" +dependencies = [ + "heapless", + "serde", +] + +[[package]] +name = "fastrand" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f1f227452a390804cdb637b74a86990f2a7d7ba4b7d5693aac9b4dd6defd8d6" + +[[package]] +name = "ff" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0b50bfb653653f9ca9095b427bed08ab8d75a137839d9ad64eb11810d5b6393" +dependencies = [ + "rand_core 0.6.4", + "subtle", +] + +[[package]] +name = "fiat-crypto" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64cd1e32ddd350061ae6edb1b082d7c54915b5c672c389143b9a63403a109f24" + +[[package]] +name = "filetime" +version = "0.2.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c287a33c7f0a620c38e641e7f60827713987b3c0f26e8ddc9462cc69cf75759" +dependencies = [ + "cfg-if", + "libc", +] + +[[package]] +name = "find-msvc-tools" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5baebc0774151f905a1a2cc41989300b1e6fbb29aff0ceffa1064fdd3088d582" + +[[package]] +name = "flate2" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "843fba2746e448b37e26a819579957415c8cef339bf08564fe8b7ddbd959573c" +dependencies = [ + "miniz_oxide", + "zlib-rs", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "foldhash" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" + +[[package]] +name = "foldhash" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77ce24cb58228fbb8aa041425bb1050850ac19177686ea6e0f41a70416f56fdb" + +[[package]] +name = "form_urlencoded" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb4cb245038516f5f85277875cdaa4f7d2c9a0fa0468de06ed190163b1581fcf" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "generic-array" +version = "0.14.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4bb6743198531e02858aeaea5398fcc883e71851fcbcb5a2f773e2fb6cb1edf2" +dependencies = [ + "typenum", + "version_check", + "zeroize", +] + +[[package]] +name = "getrandom" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff2abc00be7fca6ebc474524697ae276ad847ad0a6b3faa4bcb027e9a4614ad0" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "getrandom" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" +dependencies = [ + "cfg-if", + "libc", + "r-efi 5.3.0", + "wasip2", +] + +[[package]] +name = "getrandom" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0de51e6874e94e7bf76d726fc5d13ba782deca734ff60d5bb2fb2607c7406555" +dependencies = [ + "cfg-if", + "js-sys", + "libc", + "r-efi 6.0.0", + "wasip2", + "wasip3", + "wasm-bindgen", +] + +[[package]] +name = "git2" +version = "0.20.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b88256088d75a56f8ecfa070513a775dd9107f6530ef14919dac831af9cfe2b" +dependencies = [ + "bitflags", + "libc", + "libgit2-sys", + "log", + "openssl-probe", + "openssl-sys", + "url", +] + +[[package]] +name = "git2-curl" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be8dcabbc09ece4d30a9aa983d5804203b7e2f8054a171f792deff59b56d31fa" +dependencies = [ + "curl", + "git2", + "log", + "url", +] + +[[package]] +name = "gix" +version = "0.73.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "514c29cc879bdc0286b0cbc205585a49b252809eb86c69df4ce4f855ee75f635" +dependencies = [ + "gix-actor", + "gix-attributes", + "gix-command", + "gix-commitgraph", + "gix-config", + "gix-credentials", + "gix-date", + "gix-diff", + "gix-dir", + "gix-discover", + "gix-features", + "gix-filter", + "gix-fs", + "gix-glob", + "gix-hash", + "gix-hashtable", + "gix-ignore", + "gix-index", + "gix-lock", + "gix-negotiate", + "gix-object", + "gix-odb", + "gix-pack", + "gix-path", + "gix-pathspec", + "gix-prompt", + "gix-protocol", + "gix-ref", + "gix-refspec", + "gix-revision", + "gix-revwalk", + "gix-sec", + "gix-shallow", + "gix-status", + "gix-submodule", + "gix-tempfile", + "gix-trace", + "gix-transport", + "gix-traverse", + "gix-url", + "gix-utils", + "gix-validate", + "gix-worktree", + "once_cell", + "prodash", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-actor" +version = "0.35.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "987a51a7e66db6ef4dc030418eb2a42af6b913a79edd8670766122d8af3ba59e" +dependencies = [ + "bstr", + "gix-date", + "gix-utils", + "itoa", + "thiserror", + "winnow 0.7.15", +] + +[[package]] +name = "gix-attributes" +version = "0.27.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "45442188216d08a5959af195f659cb1f244a50d7d2d0c3873633b1cd7135f638" +dependencies = [ + "bstr", + "gix-glob", + "gix-path", + "gix-quote", + "gix-trace", + "kstring", + "smallvec", + "thiserror", + "unicode-bom", +] + +[[package]] +name = "gix-bitmap" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d982fc7ef0608e669851d0d2a6141dae74c60d5a27e8daa451f2a4857bbf41e2" +dependencies = [ + "thiserror", +] + +[[package]] +name = "gix-chunk" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c356b3825677cb6ff579551bb8311a81821e184453cbd105e2fc5311b288eeb" +dependencies = [ + "thiserror", +] + +[[package]] +name = "gix-command" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46f9c425730a654835351e6da8c3c69ba1804f8b8d4e96d027254151138d5c64" +dependencies = [ + "bstr", + "gix-path", + "gix-quote", + "gix-trace", + "shell-words", +] + +[[package]] +name = "gix-commitgraph" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bb23121e952f43a5b07e3e80890336cb847297467a410475036242732980d06" +dependencies = [ + "bstr", + "gix-chunk", + "gix-hash", + "memmap2", + "thiserror", +] + +[[package]] +name = "gix-config" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dfb898c5b695fd4acfc3c0ab638525a65545d47706064dcf7b5ead6cdb136c0" +dependencies = [ + "bstr", + "gix-config-value", + "gix-features", + "gix-glob", + "gix-path", + "gix-ref", + "gix-sec", + "memchr", + "once_cell", + "smallvec", + "thiserror", + "unicode-bom", + "winnow 0.7.15", +] + +[[package]] +name = "gix-config-value" +version = "0.15.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c489abb061c74b0c3ad790e24a606ef968cebab48ec673d6a891ece7d5aef64" +dependencies = [ + "bitflags", + "bstr", + "gix-path", + "libc", + "thiserror", +] + +[[package]] +name = "gix-credentials" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0039dd3ac606dd80b16353a41b61fc237ca5cb8b612f67a9f880adfad4be4e05" +dependencies = [ + "bstr", + "gix-command", + "gix-config-value", + "gix-date", + "gix-path", + "gix-prompt", + "gix-sec", + "gix-trace", + "gix-url", + "thiserror", +] + +[[package]] +name = "gix-date" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "661245d045aa7c16ba4244daaabd823c562c3e45f1f25b816be2c57ee09f2171" +dependencies = [ + "bstr", + "itoa", + "jiff", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-diff" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de854852010d44a317f30c92d67a983e691c9478c8a3fb4117c1f48626bcdea8" +dependencies = [ + "bstr", + "gix-attributes", + "gix-command", + "gix-filter", + "gix-fs", + "gix-hash", + "gix-index", + "gix-object", + "gix-path", + "gix-pathspec", + "gix-tempfile", + "gix-trace", + "gix-traverse", + "gix-worktree", + "imara-diff", + "thiserror", +] + +[[package]] +name = "gix-dir" +version = "0.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dad34e4f373f94902df1ba1d2a1df3a1b29eacd15e316ac5972d842e31422dd7" +dependencies = [ + "bstr", + "gix-discover", + "gix-fs", + "gix-ignore", + "gix-index", + "gix-object", + "gix-path", + "gix-pathspec", + "gix-trace", + "gix-utils", + "gix-worktree", + "thiserror", +] + +[[package]] +name = "gix-discover" +version = "0.41.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffb180c91ca1a2cf53e828bb63d8d8f8fa7526f49b83b33d7f46cbeb5d79d30a" +dependencies = [ + "bstr", + "dunce", + "gix-fs", + "gix-hash", + "gix-path", + "gix-ref", + "gix-sec", + "thiserror", +] + +[[package]] +name = "gix-features" +version = "0.43.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd1543cd9b8abcbcebaa1a666a5c168ee2cda4dea50d3961ee0e6d1c42f81e5b" +dependencies = [ + "bytes", + "crc32fast", + "crossbeam-channel", + "flate2", + "gix-path", + "gix-trace", + "gix-utils", + "libc", + "once_cell", + "parking_lot", + "prodash", + "thiserror", + "walkdir", +] + +[[package]] +name = "gix-filter" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa6571a3927e7ab10f64279a088e0dae08e8da05547771796d7389bbe28ad9ff" +dependencies = [ + "bstr", + "encoding_rs", + "gix-attributes", + "gix-command", + "gix-hash", + "gix-object", + "gix-packetline-blocking", + "gix-path", + "gix-quote", + "gix-trace", + "gix-utils", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-fs" +version = "0.16.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a4d90307d064fa7230e0f87b03231be28f8ba63b913fc15346f489519d0c304" +dependencies = [ + "bstr", + "fastrand", + "gix-features", + "gix-path", + "gix-utils", + "thiserror", +] + +[[package]] +name = "gix-glob" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b947db8366823e7a750c254f6bb29e27e17f27e457bf336ba79b32423db62cd5" +dependencies = [ + "bitflags", + "bstr", + "gix-features", + "gix-path", +] + +[[package]] +name = "gix-hash" +version = "0.19.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "251fad79796a731a2a7664d9ea95ee29a9e99474de2769e152238d4fdb69d50e" +dependencies = [ + "faster-hex", + "gix-features", + "sha1-checked", + "thiserror", +] + +[[package]] +name = "gix-hashtable" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c35300b54896153e55d53f4180460931ccd69b7e8d2f6b9d6401122cdedc4f07" +dependencies = [ + "gix-hash", + "hashbrown 0.15.5", + "parking_lot", +] + +[[package]] +name = "gix-ignore" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "564d6fddf46e2c981f571b23d6ad40cb08bddcaf6fc7458b1d49727ad23c2870" +dependencies = [ + "bstr", + "gix-glob", + "gix-path", + "gix-trace", + "unicode-bom", +] + +[[package]] +name = "gix-index" +version = "0.41.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2af39fde3ce4ce11371d9ce826f2936ec347318f2d1972fe98c2e7134e267e25" +dependencies = [ + "bitflags", + "bstr", + "filetime", + "fnv", + "gix-bitmap", + "gix-features", + "gix-fs", + "gix-hash", + "gix-lock", + "gix-object", + "gix-traverse", + "gix-utils", + "gix-validate", + "hashbrown 0.15.5", + "itoa", + "libc", + "memmap2", + "rustix", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-lock" +version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9fa71da90365668a621e184eb5b979904471af1b3b09b943a84bc50e8ad42ed" +dependencies = [ + "gix-tempfile", + "gix-utils", + "thiserror", +] + +[[package]] +name = "gix-negotiate" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d58d4c9118885233be971e0d7a589f5cfb1a8bd6cb6e2ecfb0fc6b1b293c83b" +dependencies = [ + "bitflags", + "gix-commitgraph", + "gix-date", + "gix-hash", + "gix-object", + "gix-revwalk", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-object" +version = "0.50.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d69ce108ab67b65fbd4fb7e1331502429d78baeb2eee10008bdef55765397c07" +dependencies = [ + "bstr", + "gix-actor", + "gix-date", + "gix-features", + "gix-hash", + "gix-hashtable", + "gix-path", + "gix-utils", + "gix-validate", + "itoa", + "smallvec", + "thiserror", + "winnow 0.7.15", +] + +[[package]] +name = "gix-odb" +version = "0.70.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c9d7af10fda9df0bb4f7f9bd507963560b3c66cb15a5b825caf752e0eb109ac" +dependencies = [ + "arc-swap", + "gix-date", + "gix-features", + "gix-fs", + "gix-hash", + "gix-hashtable", + "gix-object", + "gix-pack", + "gix-path", + "gix-quote", + "parking_lot", + "tempfile", + "thiserror", +] + +[[package]] +name = "gix-pack" +version = "0.60.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8571df89bfca5abb49c3e3372393f7af7e6f8b8dbe2b96303593cef5b263019" +dependencies = [ + "clru", + "gix-chunk", + "gix-features", + "gix-hash", + "gix-hashtable", + "gix-object", + "gix-path", + "gix-tempfile", + "memmap2", + "parking_lot", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-packetline" +version = "0.19.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64286a8b5148e76ab80932e72762dd27ccf6169dd7a134b027c8a262a8262fcf" +dependencies = [ + "bstr", + "faster-hex", + "gix-trace", + "thiserror", +] + +[[package]] +name = "gix-packetline-blocking" +version = "0.19.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89c59c3ad41e68cb38547d849e9ef5ccfc0d00f282244ba1441ae856be54d001" +dependencies = [ + "bstr", + "faster-hex", + "gix-trace", + "thiserror", +] + +[[package]] +name = "gix-path" +version = "0.10.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7cb06c3e4f8eed6e24fd915fa93145e28a511f4ea0e768bae16673e05ed3f366" +dependencies = [ + "bstr", + "gix-trace", + "gix-validate", + "thiserror", +] + +[[package]] +name = "gix-pathspec" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "daedead611c9bd1f3640dc90a9012b45f790201788af4d659f28d94071da7fba" +dependencies = [ + "bitflags", + "bstr", + "gix-attributes", + "gix-config-value", + "gix-glob", + "gix-path", + "thiserror", +] + +[[package]] +name = "gix-prompt" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "868e6516dfa16fdcbc5f8c935167d085f2ae65ccd4c9476a4319579d12a69d8d" +dependencies = [ + "gix-command", + "gix-config-value", + "parking_lot", + "rustix", + "thiserror", +] + +[[package]] +name = "gix-protocol" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12b4b807c47ffcf7c1e5b8119585368a56449f3493da93b931e1d4239364e922" +dependencies = [ + "bstr", + "gix-credentials", + "gix-date", + "gix-features", + "gix-hash", + "gix-lock", + "gix-negotiate", + "gix-object", + "gix-ref", + "gix-refspec", + "gix-revwalk", + "gix-shallow", + "gix-trace", + "gix-transport", + "gix-utils", + "maybe-async", + "thiserror", + "winnow 0.7.15", +] + +[[package]] +name = "gix-quote" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96fc2ff2ec8cc0c92807f02eab1f00eb02619fc2810d13dc42679492fcc36757" +dependencies = [ + "bstr", + "gix-utils", + "thiserror", +] + +[[package]] +name = "gix-ref" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b966f578079a42f4a51413b17bce476544cca1cf605753466669082f94721758" +dependencies = [ + "gix-actor", + "gix-features", + "gix-fs", + "gix-hash", + "gix-lock", + "gix-object", + "gix-path", + "gix-tempfile", + "gix-utils", + "gix-validate", + "memmap2", + "thiserror", + "winnow 0.7.15", +] + +[[package]] +name = "gix-refspec" +version = "0.31.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d29cae1ae31108826e7156a5e60bffacab405f4413f5bc0375e19772cce0055" +dependencies = [ + "bstr", + "gix-hash", + "gix-revision", + "gix-validate", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-revision" +version = "0.35.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f651f2b1742f760bb8161d6743229206e962b73d9c33c41f4e4aefa6586cbd3d" +dependencies = [ + "bstr", + "gix-commitgraph", + "gix-date", + "gix-hash", + "gix-object", + "gix-revwalk", + "thiserror", +] + +[[package]] +name = "gix-revwalk" +version = "0.21.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06e74f91709729e099af6721bd0fa7d62f243f2005085152301ca5cdd86ec02c" +dependencies = [ + "gix-commitgraph", + "gix-date", + "gix-hash", + "gix-hashtable", + "gix-object", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-sec" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea9962ed6d9114f7f100efe038752f41283c225bb507a2888903ac593dffa6be" +dependencies = [ + "bitflags", + "gix-path", + "libc", + "windows-sys 0.61.2", +] + +[[package]] +name = "gix-shallow" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d936745103243ae4c510f19e0760ce73fb0f08096588fdbe0f0d7fb7ce8944b7" +dependencies = [ + "bstr", + "gix-hash", + "gix-lock", + "thiserror", +] + +[[package]] +name = "gix-status" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a4afff9b34eeececa8bdc32b42fb318434b6b1391d9f8d45fe455af08dc2d35" +dependencies = [ + "bstr", + "filetime", + "gix-diff", + "gix-dir", + "gix-features", + "gix-filter", + "gix-fs", + "gix-hash", + "gix-index", + "gix-object", + "gix-path", + "gix-pathspec", + "gix-worktree", + "portable-atomic", + "thiserror", +] + +[[package]] +name = "gix-submodule" +version = "0.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "657cc5dd43cbc7a14d9c5aaf02cfbe9c2a15d077cded3f304adb30ef78852d3e" +dependencies = [ + "bstr", + "gix-config", + "gix-path", + "gix-pathspec", + "gix-refspec", + "gix-url", + "thiserror", +] + +[[package]] +name = "gix-tempfile" +version = "18.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "666c0041bcdedf5fa05e9bef663c897debab24b7dc1741605742412d1d47da57" +dependencies = [ + "dashmap", + "gix-fs", + "libc", + "once_cell", + "parking_lot", + "tempfile", +] + +[[package]] +name = "gix-trace" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44dc45eae785c0eb14173e0f152e6e224dcf4d45b6a6999a3aed22af541ad678" + +[[package]] +name = "gix-transport" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12f7cc0179fc89d53c54e1f9ce51229494864ab4bf136132d69db1b011741ca3" +dependencies = [ + "base64", + "bstr", + "curl", + "gix-command", + "gix-credentials", + "gix-features", + "gix-packetline", + "gix-quote", + "gix-sec", + "gix-url", + "thiserror", +] + +[[package]] +name = "gix-traverse" +version = "0.47.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7cdc82509d792ba0ad815f86f6b469c7afe10f94362e96c4494525a6601bdd5" +dependencies = [ + "bitflags", + "gix-commitgraph", + "gix-date", + "gix-hash", + "gix-hashtable", + "gix-object", + "gix-revwalk", + "smallvec", + "thiserror", +] + +[[package]] +name = "gix-url" +version = "0.32.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b76a9d266254ad287ffd44467cd88e7868799b08f4d52e02d942b93e514d16f" +dependencies = [ + "bstr", + "gix-features", + "gix-path", + "percent-encoding", + "thiserror", + "url", +] + +[[package]] +name = "gix-utils" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66c50966184123caf580ffa64e28031a878597f1c7fceb8fe19566c38eb1b771" +dependencies = [ + "bstr", + "fastrand", + "unicode-normalization", +] + +[[package]] +name = "gix-validate" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b1e63a5b516e970a594f870ed4571a8fdcb8a344e7bd407a20db8bd61dbfde4" +dependencies = [ + "bstr", + "thiserror", +] + +[[package]] +name = "gix-worktree" +version = "0.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55f625ac9126c19bef06dbc6d2703cdd7987e21e35b497bb265ac37d383877b1" +dependencies = [ + "bstr", + "gix-attributes", + "gix-features", + "gix-fs", + "gix-glob", + "gix-hash", + "gix-ignore", + "gix-index", + "gix-object", + "gix-path", + "gix-validate", +] + +[[package]] +name = "glob" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" + +[[package]] +name = "globset" +version = "0.4.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52dfc19153a48bde0cbd630453615c8151bce3a5adfac7a0aebfbf0a1e1f57e3" +dependencies = [ + "aho-corasick", + "bstr", + "log", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "group" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0f9ef7462f7c099f518d754361858f86d8a07af53ba9af0fe635bbccb151a63" +dependencies = [ + "ff", + "rand_core 0.6.4", + "subtle", +] + +[[package]] +name = "hash32" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47d60b12902ba28e2730cd37e95b8c9223af2808df9e902d4df49588d1470606" +dependencies = [ + "byteorder", +] + +[[package]] +name = "hashbrown" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" + +[[package]] +name = "hashbrown" +version = "0.15.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" +dependencies = [ + "allocator-api2", + "equivalent", + "foldhash 0.1.5", +] + +[[package]] +name = "hashbrown" +version = "0.16.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "841d1cc9bed7f9236f321df977030373f4a4163ae1a7dbfe1a51a2c1a51d9100" +dependencies = [ + "allocator-api2", + "equivalent", + "foldhash 0.2.0", +] + +[[package]] +name = "hashbrown" +version = "0.17.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed5909b6e89a2db4456e54cd5f673791d7eca6732202bbf2a9cc504fe2f9b84a" + +[[package]] +name = "hashlink" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7382cf6263419f2d8df38c55d7da83da5c18aef87fc7a7fc1fb1e344edfe14c1" +dependencies = [ + "hashbrown 0.15.5", +] + +[[package]] +name = "heapless" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bfb9eb618601c89945a70e254898da93b13be0388091d42117462b265bb3fad" +dependencies = [ + "hash32", + "stable_deref_trait", +] + +[[package]] +name = "heck" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "hkdf" +version = "0.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b5f8eb2ad728638ea2c7d47a21db23b7b58a72ed6a38256b8a1849f15fbbdf7" +dependencies = [ + "hmac", +] + +[[package]] +name = "hmac" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c49c37c09c17a53d937dfbb742eb3a961d65a994e6bcdcf37e7399d0cc8ab5e" +dependencies = [ + "digest", +] + +[[package]] +name = "home" +version = "0.5.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc627f471c528ff0c4a49e1d5e60450c8f6461dd6d10ba9dcd3a61d3dff7728d" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "http-auth" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "150fa4a9462ef926824cf4519c84ed652ca8f4fbae34cb8af045b5cbcaf98822" +dependencies = [ + "memchr", +] + +[[package]] +name = "icu_collections" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2984d1cd16c883d7935b9e07e44071dca8d917fd52ecc02c04d5fa0b5a3f191c" +dependencies = [ + "displaydoc", + "potential_utf", + "utf8_iter", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locale_core" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92219b62b3e2b4d88ac5119f8904c10f8f61bf7e95b640d25ba3075e6cac2c29" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_normalizer" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c56e5ee99d6e3d33bd91c5d85458b6005a22140021cc324cea84dd0e72cff3b4" +dependencies = [ + "icu_collections", + "icu_normalizer_data", + "icu_properties", + "icu_provider", + "smallvec", + "zerovec", +] + +[[package]] +name = "icu_normalizer_data" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da3be0ae77ea334f4da67c12f149704f19f81d1adf7c51cf482943e84a2bad38" + +[[package]] +name = "icu_properties" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bee3b67d0ea5c2cca5003417989af8996f8604e34fb9ddf96208a033901e70de" +dependencies = [ + "icu_collections", + "icu_locale_core", + "icu_properties_data", + "icu_provider", + "zerotrie", + "zerovec", +] + +[[package]] +name = "icu_properties_data" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e2bbb201e0c04f7b4b3e14382af113e17ba4f63e2c9d2ee626b720cbce54a14" + +[[package]] +name = "icu_provider" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "139c4cf31c8b5f33d7e199446eff9c1e02decfc2f0eec2c8d71f65befa45b421" +dependencies = [ + "displaydoc", + "icu_locale_core", + "writeable", + "yoke", + "zerofrom", + "zerotrie", + "zerovec", +] + +[[package]] +name = "id-arena" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d3067d79b975e8844ca9eb072e16b31c3c1c36928edf9c6789548c524d0d954" + +[[package]] +name = "idna" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b0875f23caa03898994f6ddc501886a45c7d3d62d04d2d90788d47be1b1e4de" +dependencies = [ + "idna_adapter", + "smallvec", + "utf8_iter", +] + +[[package]] +name = "idna_adapter" +version = "1.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb68373c0d6620ef8105e855e7745e18b0d00d3bdb07fb532e434244cdb9a714" +dependencies = [ + "icu_normalizer", + "icu_properties", +] + +[[package]] +name = "ignore" +version = "0.4.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b915661dd01db3f05050265b2477bcc6527b3792388e2749b41623cc592be67d" +dependencies = [ + "crossbeam-deque", + "globset", + "log", + "memchr", + "regex-automata", + "same-file", + "walkdir", + "winapi-util", +] + +[[package]] +name = "im-rc" +version = "15.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af1955a75fa080c677d3972822ec4bad316169ab1cfc6c257a942c2265dbe5fe" +dependencies = [ + "bitmaps", + "rand_core 0.6.4", + "rand_xoshiro", + "sized-chunks", + "typenum", + "version_check", +] + +[[package]] +name = "imara-diff" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17d34b7d42178945f775e84bc4c36dde7c1c6cdfea656d3354d009056f2bb3d2" +dependencies = [ + "hashbrown 0.15.5", +] + +[[package]] +name = "indexmap" +version = "2.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d466e9454f08e4a911e14806c24e16fba1b4c121d1ea474396f396069cf949d9" +dependencies = [ + "equivalent", + "hashbrown 0.17.1", + "serde", + "serde_core", +] + +[[package]] +name = "is_executable" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baabb8b4867b26294d818bf3f651a454b6901431711abb96e296245888d6e8c4" +dependencies = [ + "windows-sys 0.60.2", +] + +[[package]] +name = "is_terminal_polyfill" +version = "1.70.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" + +[[package]] +name = "itertools" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b192c782037fadd9cfa75548310488aabdbf3d2da73885b31bd0abd03351285" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f42a60cbdf9a97f5d2305f08a87dc4e09308d1276d28c869c684d7777685682" + +[[package]] +name = "jiff" +version = "0.2.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4603d3033e49e2b0e31229fcab20a5d40089c607d975cd9c80551dc69eed9102" +dependencies = [ + "jiff-static", + "jiff-tzdb-platform", + "log", + "portable-atomic", + "portable-atomic-util", + "serde_core", + "windows-link", +] + +[[package]] +name = "jiff-static" +version = "0.2.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "782d32378dddf207193ac91cefb848ad41abb58195c95168e1291227a0832b47" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "jiff-tzdb" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c900ef84826f1338a557697dc8fc601df9ca9af4ac137c7fb61d4c6f2dfd3076" + +[[package]] +name = "jiff-tzdb-platform" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "875a5a69ac2bab1a891711cf5eccbec1ce0341ea805560dcd90b7a2e925132e8" +dependencies = [ + "jiff-tzdb", +] + +[[package]] +name = "jobserver" +version = "0.1.34" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9afb3de4395d6b3e67a780b6de64b51c978ecf11cb9a462c66be7d4ca9039d33" +dependencies = [ + "getrandom 0.3.4", + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.100" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2025f20d7a4fa7785846e7b63d10a76d3f1cee98ee5cb79ea59703f95e42162" +dependencies = [ + "cfg-if", + "wasm-bindgen", +] + +[[package]] +name = "kstring" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "558bf9508a558512042d3095138b1f7b8fe90c5467d94f9f1da28b3731c5dbd1" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lazy_static" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" + +[[package]] +name = "lazycell" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" + +[[package]] +name = "leb128fmt" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09edd9e8b54e49e587e4f6295a7d29c3ea94d469cb40ab8ca70b288248a81db2" + +[[package]] +name = "libc" +version = "0.2.186" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68ab91017fe16c622486840e4c83c9a37afeff978bd239b5293d61ece587de66" + +[[package]] +name = "libgit2-sys" +version = "0.18.5+1.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "005d6ae6eac1912906073e069f7db60b1fa98e052a68227824afe3e3a1c59ca2" +dependencies = [ + "cc", + "libc", + "libssh2-sys", + "libz-sys", + "openssl-sys", + "pkg-config", +] + +[[package]] +name = "libloading" +version = "0.8.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" +dependencies = [ + "cfg-if", + "windows-link", +] + +[[package]] +name = "libnghttp2-sys" +version = "0.1.13+1.68.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "492e00167f1418c15648144f42bbfc63099806ecee9bf8d09a6353d6b4856b3c" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "libsqlite3-sys" +version = "0.35.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "133c182a6a2c87864fe97778797e46c7e999672690dc9fa3ee8e241aa4a9c13f" +dependencies = [ + "cc", + "pkg-config", + "vcpkg", +] + +[[package]] +name = "libssh2-sys" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "220e4f05ad4a218192533b300327f5150e809b54c4ec83b5a1d91833601811b9" +dependencies = [ + "cc", + "libc", + "libz-sys", + "openssl-sys", + "pkg-config", + "vcpkg", +] + +[[package]] +name = "libz-sys" +version = "1.1.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85bc9657773828b90eeb625adff10eeac83cc21bbfd8e23a03eaa8a33c9e28d9" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + +[[package]] +name = "linux-raw-sys" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a66949e030da00e8c7d4434b251670a91556f4144941d37452769c25d58a53" + +[[package]] +name = "litemap" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92daf443525c4cce67b150400bc2316076100ce0b3686209eb8cf3c31612e6f0" + +[[package]] +name = "lock_api" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "224399e74b87b5f3557511d98dff8b14089b3dadafcab6bb93eab67d3aace965" +dependencies = [ + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "953f07c43838f8e6f9758cab68bf5bed85465e7587ebe0b823f1bcd81978ad3a" + +[[package]] +name = "matchers" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1525a2a28c7f4fa0fc98bb91ae755d1e2d1505079e05539e35bc876b5d65ae9" +dependencies = [ + "regex-automata", +] + +[[package]] +name = "maybe-async" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "746873a384ad60adc5db74471dfaba74bd278afbdcfd81db93fafcdfc8b5ca0c" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "memchr" +version = "2.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88904434abc2901f197fe8cc55f0445e7ded921dba5911dad2e2b39b48e663c4" + +[[package]] +name = "memmap2" +version = "0.9.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "714098028fe011992e1c3962653c96b2d578c4b4bce9036e15ff220319b1e0e3" +dependencies = [ + "libc", +] + +[[package]] +name = "minimal-lexical" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" + +[[package]] +name = "miniz_oxide" +version = "0.8.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" +dependencies = [ + "adler2", + "simd-adler32", +] + +[[package]] +name = "miow" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "536bfad37a309d62069485248eeaba1e8d9853aaf951caaeaed0585a95346f08" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "nix" +version = "0.31.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf20d2fde8ff38632c426f1165ed7436270b44f199fc55284c38276f9db47c3d" +dependencies = [ + "bitflags", + "cfg-if", + "cfg_aliases", + "libc", +] + +[[package]] +name = "nom" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d273983c5a657a70a3e8f2a01329822f3b8c8172b73826411a55751e404a0a4a" +dependencies = [ + "memchr", + "minimal-lexical", +] + +[[package]] +name = "normpath" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9985ef7269fa99f3b12437bb698381da2428743ab90f20393f399fa14cab21a" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "nu-ansi-term" +version = "0.50.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "num-conv" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "521739c6d2bac4aa25192232afe6841231376b2b26d4d9fae5ecf8ca5772e441" + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", +] + +[[package]] +name = "objc2" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a12a8ed07aefc768292f076dc3ac8c48f3781c8f2d5851dd3d98950e8c5a89f" +dependencies = [ + "objc2-encode", +] + +[[package]] +name = "objc2-cloud-kit" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73ad74d880bb43877038da939b7427bba67e9dd42004a18b809ba7d87cee241c" +dependencies = [ + "bitflags", + "objc2", + "objc2-foundation", +] + +[[package]] +name = "objc2-core-data" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b402a653efbb5e82ce4df10683b6b28027616a2715e90009947d50b8dd298fa" +dependencies = [ + "objc2", + "objc2-foundation", +] + +[[package]] +name = "objc2-core-foundation" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2a180dd8642fa45cdb7dd721cd4c11b1cadd4929ce112ebd8b9f5803cc79d536" +dependencies = [ + "bitflags", + "dispatch2", + "objc2", +] + +[[package]] +name = "objc2-core-graphics" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e022c9d066895efa1345f8e33e584b9f958da2fd4cd116792e15e07e4720a807" +dependencies = [ + "bitflags", + "dispatch2", + "objc2", + "objc2-core-foundation", + "objc2-io-surface", +] + +[[package]] +name = "objc2-core-image" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5d563b38d2b97209f8e861173de434bd0214cf020e3423a52624cd1d989f006" +dependencies = [ + "objc2", + "objc2-foundation", +] + +[[package]] +name = "objc2-core-location" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca347214e24bc973fc025fd0d36ebb179ff30536ed1f80252706db19ee452009" +dependencies = [ + "objc2", + "objc2-foundation", +] + +[[package]] +name = "objc2-core-text" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0cde0dfb48d25d2b4862161a4d5fcc0e3c24367869ad306b0c9ec0073bfed92d" +dependencies = [ + "bitflags", + "objc2", + "objc2-core-foundation", + "objc2-core-graphics", +] + +[[package]] +name = "objc2-encode" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef25abbcd74fb2609453eb695bd2f860d389e457f67dc17cafc8b8cbc89d0c33" + +[[package]] +name = "objc2-foundation" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3e0adef53c21f888deb4fa59fc59f7eb17404926ee8a6f59f5df0fd7f9f3272" +dependencies = [ + "bitflags", + "block2", + "libc", + "objc2", + "objc2-core-foundation", +] + +[[package]] +name = "objc2-io-surface" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "180788110936d59bab6bd83b6060ffdfffb3b922ba1396b312ae795e1de9d81d" +dependencies = [ + "bitflags", + "objc2", + "objc2-core-foundation", +] + +[[package]] +name = "objc2-quartz-core" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96c1358452b371bf9f104e21ec536d37a650eb10f7ee379fff67d2e08d537f1f" +dependencies = [ + "bitflags", + "objc2", + "objc2-core-foundation", + "objc2-foundation", +] + +[[package]] +name = "objc2-ui-kit" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d87d638e33c06f577498cbcc50491496a3ed4246998a7fbba7ccb98b1e7eab22" +dependencies = [ + "bitflags", + "block2", + "objc2", + "objc2-cloud-kit", + "objc2-core-data", + "objc2-core-foundation", + "objc2-core-graphics", + "objc2-core-image", + "objc2-core-location", + "objc2-core-text", + "objc2-foundation", + "objc2-quartz-core", + "objc2-user-notifications", +] + +[[package]] +name = "objc2-user-notifications" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9df9128cbbfef73cda168416ccf7f837b62737d748333bfe9ab71c245d76613e" +dependencies = [ + "objc2", + "objc2-foundation", +] + +[[package]] +name = "once_cell" +version = "1.21.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f7c3e4beb33f85d45ae3e3a1792185706c8e16d043238c593331cc7cd313b50" + +[[package]] +name = "once_cell_polyfill" +version = "1.70.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "384b8ab6d37215f3c5301a95a4accb5d64aa607f1fcb26a11b5303878451b4fe" + +[[package]] +name = "opener" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2b03ff07a220d0d0ec9a1f0f238951b7967a5a2e96aefcd21a117b1083415e9" +dependencies = [ + "bstr", + "normpath", + "windows-sys 0.61.2", +] + +[[package]] +name = "openssl-probe" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d05e27ee213611ffe7d6348b942e8f942b37114c00cc03cec254295a4a17852e" + +[[package]] +name = "openssl-sys" +version = "0.9.116" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28a22dc7140cda5f096e5e7724a6962ca81a7f8bfd2979f9b18c11af56318c4" +dependencies = [ + "cc", + "libc", + "pkg-config", + "vcpkg", +] + +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + +[[package]] +name = "orion" +version = "0.17.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e6758747fd1ce1efaf2bd43219ac4aa9e28263b236b2b6a1e486bcd06820707" +dependencies = [ + "fiat-crypto", + "subtle", +] + +[[package]] +name = "os_info" +version = "3.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9cf20a545b305cf1da722b236b5155c9bb35f1d5ceb28c048bd96ca842f41b5b" +dependencies = [ + "android_system_properties", + "log", + "nix", + "objc2", + "objc2-foundation", + "objc2-ui-kit", + "windows-sys 0.61.2", +] + +[[package]] +name = "p384" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fe42f1670a52a47d448f14b6a5c61dd78fce51856e68edaa38f7ae3a46b8d6b6" +dependencies = [ + "ecdsa", + "elliptic-curve", + "primeorder", + "sha2", +] + +[[package]] +name = "parking_lot" +version = "0.12.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93857453250e3077bd71ff98b6a65ea6621a19bb0f559a85248955ac12c45a1a" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2621685985a2ebf1c516881c026032ac7deafcda1a2c9b7850dc81e3dfcb64c1" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-link", +] + +[[package]] +name = "pasetors" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e838401fb2873bad417e6a03179014c748746f67311cb7317ab14fc0881fa9f0" +dependencies = [ + "ct-codecs", + "ed25519-compact", + "getrandom 0.4.2", + "orion", + "p384", + "rand_core 0.6.4", + "regex", + "serde", + "serde_derive", + "serde_json", + "sha2", + "subtle", + "time", + "zeroize", +] + +[[package]] +name = "pathdiff" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df94ce210e5bc13cb6651479fa48d14f601d9858cfe0467f43ae157023b938d3" + +[[package]] +name = "pem-rfc7468" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "88b39c9bfcfc231068454382784bb460aae594343fb030d46e9f50a645418412" +dependencies = [ + "base64ct", +] + +[[package]] +name = "percent-encoding" +version = "2.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b4f627cb1b25917193a259e49bdad08f671f8d9708acfd5fe0a8c1455d87220" + +[[package]] +name = "pin-project-lite" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a89322df9ebe1c1578d689c92318e070967d1042b512afbe49518723f4e6d5cd" + +[[package]] +name = "pkcs8" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" +dependencies = [ + "der", + "spki", +] + +[[package]] +name = "pkg-config" +version = "0.3.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19f132c84eca552bf34cab8ec81f1c1dcc229b811638f9d283dceabe58c5569e" + +[[package]] +name = "portable-atomic" +version = "1.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c33a9471896f1c69cecef8d20cbe2f7accd12527ce60845ff44c153bb2a21b49" + +[[package]] +name = "portable-atomic-util" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2a106d1259c23fac8e543272398ae0e3c0b8d33c88ed73d0cc71b0f1d902618" +dependencies = [ + "portable-atomic", +] + +[[package]] +name = "potential_utf" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0103b1cef7ec0cf76490e969665504990193874ea05c85ff9bab8b911d0a0564" +dependencies = [ + "zerovec", +] + +[[package]] +name = "powerfmt" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" + +[[package]] +name = "ppv-lite86" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" +dependencies = [ + "zerocopy", +] + +[[package]] +name = "prettyplease" +version = "0.2.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" +dependencies = [ + "proc-macro2", + "syn", +] + +[[package]] +name = "primeorder" +version = "0.13.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "353e1ca18966c16d9deb1c69278edbc5f194139612772bd9537af60ac231e1e6" +dependencies = [ + "elliptic-curve", +] + +[[package]] +name = "proc-macro2" +version = "1.0.106" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fd00f0bb2e90d81d1044c2b32617f68fcb9fa3bb7640c23e9c748e53fb30934" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prodash" +version = "30.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6efc566849d3d9d737c5cb06cc50e48950ebe3d3f9d70631490fff3a07b139" +dependencies = [ + "parking_lot", +] + +[[package]] +name = "quote" +version = "1.0.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41f2619966050689382d2b44f664f4bc593e129785a36d6ee376ddf37259b924" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "r-efi" +version = "5.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" + +[[package]] +name = "r-efi" +version = "6.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8dcc9c7d52a811697d2151c701e0d08956f92b0e24136cf4cf27b57a6a0d9bf" + +[[package]] +name = "rand" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44c5af06bb1b7d3216d91932aed5265164bf384dc89cd6ba05cf59a35f5f76ea" +dependencies = [ + "rand_chacha", + "rand_core 0.9.5", +] + +[[package]] +name = "rand_chacha" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" +dependencies = [ + "ppv-lite86", + "rand_core 0.9.5", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom 0.2.17", +] + +[[package]] +name = "rand_core" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76afc826de14238e6e8c374ddcc1fa19e374fd8dd986b0d2af0d02377261d83c" +dependencies = [ + "getrandom 0.3.4", +] + +[[package]] +name = "rand_xoshiro" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f97cdb2a36ed4183de61b2f824cc45c9f1037f28afe0a322e9fff4c108b5aaa" +dependencies = [ + "rand_core 0.6.4", +] + +[[package]] +name = "redox_syscall" +version = "0.5.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed2bf2547551a7053d6fdfafda3f938979645c44812fbfcda098faae3f1a362d" +dependencies = [ + "bitflags", +] + +[[package]] +name = "regex" +version = "1.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1292b7759ae1cb9ec195452d1390a074f0cd8541ab7a5a8c31cd6db45d4a6ba" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e1dd4122fc1595e8162618945476892eefca7b88c52820e74af6262213cae8f" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.8.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6f6ff9a378485b298a5286656da665ba74413d36db0979633275d2e708145d4" + +[[package]] +name = "rfc6979" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8dd2a808d456c4a54e300a23e9f5a67e122c3024119acbfd73e3bf664491cb2" +dependencies = [ + "hmac", + "subtle", +] + +[[package]] +name = "rusqlite" +version = "0.37.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "165ca6e57b20e1351573e3729b958bc62f0e48025386970b6e4d29e7a7e71f3f" +dependencies = [ + "bitflags", + "fallible-iterator", + "fallible-streaming-iterator", + "hashlink", + "libsqlite3-sys", + "smallvec", +] + +[[package]] +name = "rustc-hash" +version = "2.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94300abf3f1ae2e2b8ffb7b58043de3d399c73fa6f4b73826402a5c457614dbe" + +[[package]] +name = "rustc-stable-hash" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "781442f29170c5c93b7185ad559492601acdc71d5bb0706f5868094f45cfcd08" + +[[package]] +name = "rustfix" +version = "0.9.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "864792a841a1d785ba91b8d2a75e1936b40bc517020c3c2958ac403b92e4f00a" +dependencies = [ + "serde", + "serde_json", + "thiserror", + "tracing", +] + +[[package]] +name = "rustix" +version = "1.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6fe4565b9518b83ef4f91bb47ce29620ca828bd32cb7e408f0062e9930ba190" +dependencies = [ + "bitflags", + "errno", + "libc", + "linux-raw-sys", + "windows-sys 0.61.2", +] + +[[package]] +name = "rustversion" +version = "1.0.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "schannel" +version = "0.1.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91c1b7e4904c873ef0710c1f407dde2e6287de2bebc1bbbf7d430bb7cbffd939" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "sec1" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3e97a565f76233a6003f9f5c54be1d9c5bdfa3eccfb189469f11ec4901c47dc" +dependencies = [ + "base16ct", + "der", + "generic-array", + "pkcs8", + "subtle", + "zeroize", +] + +[[package]] +name = "security-framework" +version = "3.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7f4bc775c73d9a02cde8bf7b2ec4c9d12743edf609006c7facc23998404cd1d" +dependencies = [ + "bitflags", + "core-foundation", + "core-foundation-sys", + "libc", + "security-framework-sys", +] + +[[package]] +name = "security-framework-sys" +version = "2.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ce2691df843ecc5d231c0b14ece2acc3efb62c0a398c7e1d875f3983ce020e3" +dependencies = [ + "core-foundation-sys", + "libc", +] + +[[package]] +name = "semver" +version = "1.0.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a7852d02fc848982e0c167ef163aaff9cd91dc640ba85e263cb1ce46fae51cd" +dependencies = [ + "serde", + "serde_core", +] + +[[package]] +name = "serde" +version = "1.0.228" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a8e94ea7f378bd32cbbd37198a4a91436180c5bb472411e48b5ec2e2124ae9e" +dependencies = [ + "serde_core", + "serde_derive", +] + +[[package]] +name = "serde-untagged" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9faf48a4a2d2693be24c6289dbe26552776eb7737074e6722891fadbe6c5058" +dependencies = [ + "erased-serde", + "serde", + "serde_core", + "typeid", +] + +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float", + "serde", +] + +[[package]] +name = "serde_core" +version = "1.0.228" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41d385c7d4ca58e59fc732af25c3983b67ac852c1a25000afe1175de458b67ad" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde_derive" +version = "1.0.228" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "serde_ignored" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "115dffd5f3853e06e746965a20dcbae6ee747ae30b543d91b0e089668bb07798" +dependencies = [ + "serde", + "serde_core", +] + +[[package]] +name = "serde_json" +version = "1.0.150" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8014e44b4736ed0538adeecded0fce2a272f22dc9578a7eb6b2d9993c74cfb9" +dependencies = [ + "itoa", + "memchr", + "serde", + "serde_core", + "zmij", +] + +[[package]] +name = "serde_spanned" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6662b5879511e06e8999a8a235d848113e942c9124f211511b16466ee2995f26" +dependencies = [ + "serde_core", +] + +[[package]] +name = "sha1" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" +dependencies = [ + "cfg-if", + "cpufeatures 0.2.17", + "digest", +] + +[[package]] +name = "sha1-checked" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89f599ac0c323ebb1c6082821a54962b839832b03984598375bff3975b804423" +dependencies = [ + "digest", + "sha1", +] + +[[package]] +name = "sha2" +version = "0.10.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" +dependencies = [ + "cfg-if", + "cpufeatures 0.2.17", + "digest", +] + +[[package]] +name = "sharded-slab" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f40ca3c46823713e0d4209592e8d6e826aa57e928f09752619fc696c499637f6" +dependencies = [ + "lazy_static", +] + +[[package]] +name = "shell-escape" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "45bb67a18fa91266cc7807181f62f9178a6873bfad7dc788c42e6430db40184f" + +[[package]] +name = "shell-words" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc6fe69c597f9c37bfeeeeeb33da3530379845f10be461a66d16d03eca2ded77" + +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + +[[package]] +name = "shlex" +version = "2.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8fadd59c855ef2080decdef8ff161eb6661b86933c9d82e5ba29dc602a55aba" + +[[package]] +name = "signature" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77549399552de45a898a580c1b41d445bf730df867cc44e6c0233bbc4b8329de" +dependencies = [ + "digest", + "rand_core 0.6.4", +] + +[[package]] +name = "simd-adler32" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "703d5c7ef118737c72f1af64ad2f6f8c5e1921f818cdcb97b8fe6fc69bf66214" + +[[package]] +name = "sized-chunks" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16d69225bde7a69b235da73377861095455d298f2b970996eec25ddbb42b3d1e" +dependencies = [ + "bitmaps", + "typenum", +] + +[[package]] +name = "smallvec" +version = "1.15.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ed6a63f02c8539c91a8685a86f4099661ba3da017932f6ebbea6de3f0fa7c90" + +[[package]] +name = "socket2" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52d1cfed4120b4d927bf7c0f86d2087a4a7d6027c906d9f9d525a80573b9be51" +dependencies = [ + "libc", + "windows-sys 0.61.2", +] + +[[package]] +name = "spki" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d91ed6c858b01f942cd56b37a94b3e0a1798290327d1236e4d9cf4eaca44d29d" +dependencies = [ + "base64ct", + "der", +] + +[[package]] +name = "stable_deref_trait" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" + +[[package]] +name = "static_assertions" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "strsim" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" + +[[package]] +name = "subtle" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" + +[[package]] +name = "supports-hyperlinks" +version = "3.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e396b6523b11ccb83120b115a0b7366de372751aa6edf19844dfb13a6af97e91" + +[[package]] +name = "supports-unicode" +version = "3.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7401a30af6cb5818bb64852270bb722533397edcfc7344954a38f420819ece2" + +[[package]] +name = "syn" +version = "2.0.117" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e665b8803e7b1d2a727f4023456bbbbe74da67099c585258af0ad9c5013b9b99" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "synstructure" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tar" +version = "0.4.46" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f6221d9a6003c78398e3b239969f352578258df48c8eb051caadae0015bc840" +dependencies = [ + "filetime", + "libc", +] + +[[package]] +name = "tempfile" +version = "3.27.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32497e9a4c7b38532efcdebeef879707aa9f794296a4f0244f6f69e9bc8574bd" +dependencies = [ + "fastrand", + "getrandom 0.4.2", + "once_cell", + "rustix", + "windows-sys 0.61.2", +] + +[[package]] +name = "terminal_size" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "230a1b821ccbd75b185820a1f1ff7b14d21da1e442e22c0863ea5f08771a8874" +dependencies = [ + "rustix", + "windows-sys 0.61.2", +] + +[[package]] +name = "thiserror" +version = "2.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4288b5bcbc7920c07a1149a35cf9590a2aa808e0bc1eafaade0b80947865fbc4" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "2.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebc4ee7f67670e9b64d05fa4253e753e016c6c95ff35b89b7941d6b856dec1d5" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "thread_local" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f60246a4944f24f6e018aa17cdeffb7818b76356965d03b07d6a9886e8962185" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "time" +version = "0.3.47" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "743bd48c283afc0388f9b8827b976905fb217ad9e647fae3a379a9283c4def2c" +dependencies = [ + "deranged", + "itoa", + "num-conv", + "powerfmt", + "serde_core", + "time-core", + "time-macros", +] + +[[package]] +name = "time-core" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7694e1cfe791f8d31026952abf09c69ca6f6fa4e1a1229e18988f06a04a12dca" + +[[package]] +name = "time-macros" +version = "0.2.27" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e70e4c5a0e0a8a4823ad65dfe1a6930e4f4d756dcd9dd7939022b5e8c501215" +dependencies = [ + "num-conv", + "time-core", +] + +[[package]] +name = "tinystr" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8323304221c2a851516f22236c5722a72eaa19749016521d6dff0824447d96d" +dependencies = [ + "displaydoc", + "zerovec", +] + +[[package]] +name = "tinyvec" +version = "1.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3e61e67053d25a4e82c844e8424039d9745781b3fc4f32b8d55ed50f5f667ef3" +dependencies = [ + "tinyvec_macros", +] + +[[package]] +name = "tinyvec_macros" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" + +[[package]] +name = "toml" +version = "0.9.12+spec-1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf92845e79fc2e2def6a5d828f0801e29a2f8acc037becc5ab08595c7d5e9863" +dependencies = [ + "indexmap", + "serde_core", + "serde_spanned", + "toml_datetime", + "toml_parser", + "toml_writer", + "winnow 0.7.15", +] + +[[package]] +name = "toml_datetime" +version = "0.7.5+spec-1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92e1cfed4a3038bc5a127e35a2d360f145e1f4b971b551a2ba5fd7aedf7e1347" +dependencies = [ + "serde_core", +] + +[[package]] +name = "toml_edit" +version = "0.23.10+spec-1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "84c8b9f757e028cee9fa244aea147aab2a9ec09d5325a9b01e0a49730c2b5269" +dependencies = [ + "indexmap", + "serde_core", + "serde_spanned", + "toml_datetime", + "toml_parser", + "toml_writer", + "winnow 0.7.15", +] + +[[package]] +name = "toml_parser" +version = "1.1.2+spec-1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2abe9b86193656635d2411dc43050282ca48aa31c2451210f4202550afb7526" +dependencies = [ + "winnow 1.0.3", +] + +[[package]] +name = "toml_writer" +version = "1.1.1+spec-1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "756daf9b1013ebe47a8776667b466417e2d4c5679d441c26230efd9ef78692db" + +[[package]] +name = "tracing" +version = "0.1.44" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63e71662fa4b2a2c3a26f570f037eb95bb1f85397f3cd8076caed2f026a6d100" +dependencies = [ + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7490cfa5ec963746568740651ac6781f701c9c5ea257c58e057f3ba8cf69e8da" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tracing-chrome" +version = "0.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf0a738ed5d6450a9fb96e86a23ad808de2b727fd1394585da5cdd6788ffe724" +dependencies = [ + "serde_json", + "tracing-core", + "tracing-subscriber", +] + +[[package]] +name = "tracing-core" +version = "0.1.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db97caf9d906fbde555dd62fa95ddba9eecfd14cb388e4f491a66d74cd5fb79a" +dependencies = [ + "once_cell", + "valuable", +] + +[[package]] +name = "tracing-log" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ee855f1f400bd0e5c02d150ae5de3840039a3f54b025156404e34c23c03f47c3" +dependencies = [ + "log", + "once_cell", + "tracing-core", +] + +[[package]] +name = "tracing-subscriber" +version = "0.3.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cb7f578e5945fb242538965c2d0b04418d38ec25c79d160cd279bf0731c8d319" +dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex-automata", + "sharded-slab", + "smallvec", + "thread_local", + "tracing", + "tracing-core", + "tracing-log", +] + +[[package]] +name = "typeid" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc7d623258602320d5c55d1bc22793b57daff0ec7efc270ea7d55ce1d5f5471c" + +[[package]] +name = "typenum" +version = "1.20.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6f5e870be6c3b371b77fe0ee0bafb859fa4964b4404c27de1d380043c4dda20" + +[[package]] +name = "unicase" +version = "2.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbc4bc3a9f746d862c45cb89d705aa10f187bb96c76001afab07a0d35ce60142" + +[[package]] +name = "unicode-bom" +version = "2.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7eec5d1121208364f6793f7d2e222bf75a915c19557537745b195b253dd64217" + +[[package]] +name = "unicode-ident" +version = "1.0.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6e4313cd5fcd3dad5cafa179702e2b244f760991f45397d14d4ebf38247da75" + +[[package]] +name = "unicode-normalization" +version = "0.1.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5fd4f6878c9cb28d874b009da9e8d183b5abc80117c40bbd187a1fde336be6e8" +dependencies = [ + "tinyvec", +] + +[[package]] +name = "unicode-width" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4ac048d71ede7ee76d585517add45da530660ef4390e49b098733c6e897f254" + +[[package]] +name = "unicode-xid" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebc1c04c71510c7f702b52b7c350734c9ff1295c464a03335b00bb84fc54f853" + +[[package]] +name = "url" +version = "2.5.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff67a8a4397373c3ef660812acab3268222035010ab8680ec4215f38ba3d0eed" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", + "serde", +] + +[[package]] +name = "utf8_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" + +[[package]] +name = "utf8parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" + +[[package]] +name = "valuable" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba73ea9cf16a25df0c8caa16c51acb937d5712a8429db78a3ee29d5dcacd3a65" + +[[package]] +name = "vcpkg" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "accd4ea62f7bb7a82fe23066fb0957d48ef677f6eeb8215f372f52e48bb32426" + +[[package]] +name = "version_check" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" + +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasi" +version = "0.11.1+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" + +[[package]] +name = "wasip2" +version = "1.0.4+wasi-0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b67efb37e106e55ce722a510d6b5f9c17f083e5fc79afc2badeb12cc313d9487" +dependencies = [ + "wit-bindgen 0.57.1", +] + +[[package]] +name = "wasip3" +version = "0.4.0+wasi-0.3.0-rc-2026-01-06" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5428f8bf88ea5ddc08faddef2ac4a67e390b88186c703ce6dbd955e1c145aca5" +dependencies = [ + "wit-bindgen 0.51.0", +] + +[[package]] +name = "wasm-bindgen" +version = "0.2.123" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a254a4b10c19a76f09a27640e7ffbf9bc30bf67e16a3bf28aaefa4920fe81563" +dependencies = [ + "cfg-if", + "once_cell", + "rustversion", + "wasm-bindgen-macro", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.123" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24a40fc75b0ec6f3746ceb10d36f53a93dcd68a93b11b6445983945d79eba0dc" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.123" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "908f34bd9b9ce3d4caf07b72dfab63d61504d156856c6bd3cd87fa350cf3985b" +dependencies = [ + "bumpalo", + "proc-macro2", + "quote", + "syn", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.123" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7acbf7616c27b194bbb550bf77ed0c2c3e5b7fd1260a93082b95fb7f47959b92" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "wasm-encoder" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "990065f2fe63003fe337b932cfb5e3b80e0b4d0f5ff650e6985b1048f62c8319" +dependencies = [ + "leb128fmt", + "wasmparser", +] + +[[package]] +name = "wasm-metadata" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb0e353e6a2fbdc176932bbaab493762eb1255a7900fe0fea1a2f96c296cc909" +dependencies = [ + "anyhow", + "indexmap", + "wasm-encoder", + "wasmparser", +] + +[[package]] +name = "wasmparser" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47b807c72e1bac69382b3a6fb3dbe8ea4c0ed87ff5629b8685ae6b9a611028fe" +dependencies = [ + "bitflags", + "hashbrown 0.15.5", + "indexmap", + "semver", +] + +[[package]] +name = "winapi-util" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" +dependencies = [ + "windows-sys 0.61.2", +] + +[[package]] +name = "windows-link" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" + +[[package]] +name = "windows-sys" +version = "0.60.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" +dependencies = [ + "windows-targets", +] + +[[package]] +name = "windows-sys" +version = "0.61.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ae137229bcbd6cdf0f7b80a31df61766145077ddf49416a728b02cb3921ff3fc" +dependencies = [ + "windows-link", +] + +[[package]] +name = "windows-targets" +version = "0.53.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4945f9f551b88e0d65f3db0bc25c33b8acea4d9e41163edf90dcd0b19f9069f3" +dependencies = [ + "windows-link", + "windows_aarch64_gnullvm", + "windows_aarch64_msvc", + "windows_i686_gnu", + "windows_i686_gnullvm", + "windows_i686_msvc", + "windows_x86_64_gnu", + "windows_x86_64_gnullvm", + "windows_x86_64_msvc", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9d8416fa8b42f5c947f8482c43e7d89e73a173cead56d044f6a56104a6d1b53" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9d782e804c2f632e395708e99a94275910eb9100b2114651e04744e9b125006" + +[[package]] +name = "windows_i686_gnu" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "960e6da069d81e09becb0ca57a65220ddff016ff2d6af6a223cf372a506593a3" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa7359d10048f68ab8b09fa71c3daccfb0e9b559aed648a8f95469c27057180c" + +[[package]] +name = "windows_i686_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e7ac75179f18232fe9c285163565a57ef8d3c89254a30685b57d83a38d326c2" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c3842cdd74a865a8066ab39c8a7a473c0778a3f29370b5fd6b4b9aa7df4a499" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ffa179e2d07eee8ad8f57493436566c7cc30ac536a3379fdf008f47f6bb7ae1" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" + +[[package]] +name = "winnow" +version = "0.7.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df79d97927682d2fd8adb29682d1140b343be4ac0f08fd68b7765d9c059d3945" +dependencies = [ + "memchr", +] + +[[package]] +name = "winnow" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0592e1c9d151f854e6fd382574c3a0855250e1d9b2f99d9281c6e6391af352f1" + +[[package]] +name = "wit-bindgen" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7249219f66ced02969388cf2bb044a09756a083d0fab1e566056b04d9fbcaa5" +dependencies = [ + "wit-bindgen-rust-macro", +] + +[[package]] +name = "wit-bindgen" +version = "0.57.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ebf944e87a7c253233ad6766e082e3cd714b5d03812acc24c318f549614536e" + +[[package]] +name = "wit-bindgen-core" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea61de684c3ea68cb082b7a88508a8b27fcc8b797d738bfc99a82facf1d752dc" +dependencies = [ + "anyhow", + "heck", + "wit-parser", +] + +[[package]] +name = "wit-bindgen-rust" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7c566e0f4b284dd6561c786d9cb0142da491f46a9fbed79ea69cdad5db17f21" +dependencies = [ + "anyhow", + "heck", + "indexmap", + "prettyplease", + "syn", + "wasm-metadata", + "wit-bindgen-core", + "wit-component", +] + +[[package]] +name = "wit-bindgen-rust-macro" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c0f9bfd77e6a48eccf51359e3ae77140a7f50b1e2ebfe62422d8afdaffab17a" +dependencies = [ + "anyhow", + "prettyplease", + "proc-macro2", + "quote", + "syn", + "wit-bindgen-core", + "wit-bindgen-rust", +] + +[[package]] +name = "wit-component" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d66ea20e9553b30172b5e831994e35fbde2d165325bec84fc43dbf6f4eb9cb2" +dependencies = [ + "anyhow", + "bitflags", + "indexmap", + "log", + "serde", + "serde_derive", + "serde_json", + "wasm-encoder", + "wasm-metadata", + "wasmparser", + "wit-parser", +] + +[[package]] +name = "wit-parser" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecc8ac4bc1dc3381b7f59c34f00b67e18f910c2c0f50015669dde7def656a736" +dependencies = [ + "anyhow", + "id-arena", + "indexmap", + "log", + "semver", + "serde", + "serde_derive", + "serde_json", + "unicode-xid", + "wasmparser", +] + +[[package]] +name = "writeable" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ffae5123b2d3fc086436f8834ae3ab053a283cfac8fe0a0b8eaae044768a4c4" + +[[package]] +name = "yoke" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "709fe23a0424b6a435d82152b1bd3fdfb0833487d5fa90d05d42762a9891fef5" +dependencies = [ + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de844c262c8848816172cef550288e7dc6c7b7814b4ee56b3e1553f275f1858e" +dependencies = [ + "proc-macro2", + "quote", + "syn", + "synstructure", +] + +[[package]] +name = "zerocopy" +version = "0.8.52" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce1022995ff5ff5d841ad7d994facc23098cd40152f2c1d11cd607c6f530653f" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.8.52" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ae7f38b72ec2a254e2b87ef277cf2cd4fb97cbebf944faa6f33354da0867930" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "zerofrom" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ec05a11813ea801ff6d75110ad09cd0824ddba17dfe17128ea0d5f68e6c5272" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "11532158c46691caf0f2593ea8358fed6bbf68a0315e80aae9bd41fbade684a1" +dependencies = [ + "proc-macro2", + "quote", + "syn", + "synstructure", +] + +[[package]] +name = "zeroize" +version = "1.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" + +[[package]] +name = "zerotrie" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0f9152d31db0792fa83f70fb2f83148effb5c1f5b8c7686c3459e361d9bc20bf" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", +] + +[[package]] +name = "zerovec" +version = "0.11.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90f911cbc359ab6af17377d242225f4d75119aec87ea711a880987b18cd7b239" +dependencies = [ + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "625dc425cab0dca6dc3c3319506e6593dcb08a9f387ea3b284dbd52a92c40555" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "zlib-rs" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3be3d40e40a133f9c916ee3f9f4fa2d9d63435b5fbe1bfc6d9dae0aa0ada1513" + +[[package]] +name = "zmij" +version = "1.0.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8848ee67ecc8aedbaf3e4122217aff892639231befc6a1b58d29fff4c2cabaa" diff --git a/docs/pyproject.toml b/docs/pyproject.toml index d8fa4f9ec1775..be812d6174f25 100644 --- a/docs/pyproject.toml +++ b/docs/pyproject.toml @@ -5,9 +5,9 @@ requires-python = ">=3.11" dependencies = [ "sphinx>=9,<10", "sphinx-reredirects>=1.1,<2", - "pydata-sphinx-theme>=0.18.0,<1", + "pydata-sphinx-theme>=0.19.0,<1", "myst-parser>=5.1.0,<6", - "maturin>=1.13.3,<2", + "maturin>=1.14.0,<2", "jinja2>=3.1.6,<4", "setuptools>=82.0.1,<83", ] diff --git a/docs/source/library-user-guide/upgrading/55.0.0.md b/docs/source/library-user-guide/upgrading/55.0.0.md index 2ebb6952fe04b..abe99846c12d8 100644 --- a/docs/source/library-user-guide/upgrading/55.0.0.md +++ b/docs/source/library-user-guide/upgrading/55.0.0.md @@ -45,6 +45,58 @@ will now appear as `Decimal128(NULL,10,2)`. Query result values already used human-readable decimal formatting and are unchanged. +### `GroupsAccumulator::merge_batch` no longer takes `opt_filter` + +The `opt_filter` argument has been removed from +`datafusion_expr_common::groups_accumulator::GroupsAccumulator::merge_batch`: + +```diff + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], +- opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()>; +``` + +Aggregate `FILTER` clauses only apply to raw input rows during the partial +(update) phase, so by the time intermediate states are merged there is nothing +left to filter per row. In practice `opt_filter` was always `None` here, so +removing it makes the API self-explanatory and impossible to misuse. + +**Who is affected:** + +- Anyone with a custom `GroupsAccumulator` implementation. +- Anyone calling `merge_batch` directly. + +**Migration guide:** + +Drop the `opt_filter` argument from your `merge_batch` signature and from any +call sites: + +```diff + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], +- opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + // ... + } +``` + +```diff +- acc.merge_batch(values, group_indices, None, total_num_groups)?; ++ acc.merge_batch(values, group_indices, total_num_groups)?; +``` + +If your implementation previously inspected `opt_filter` (for example asserting +it was `None`), that code can simply be deleted. + +See [issue #22775](https://github.com/apache/datafusion/issues/22775) for details. + ### `is_dynamic_physical_expr` is deprecated `datafusion_physical_expr_common::physical_expr::is_dynamic_physical_expr` is diff --git a/docs/source/user-guide/cli/datasources.md b/docs/source/user-guide/cli/datasources.md index 6b1a4887a8a0f..59a6b0aa43284 100644 --- a/docs/source/user-guide/cli/datasources.md +++ b/docs/source/user-guide/cli/datasources.md @@ -132,6 +132,30 @@ select count(*) from hits; 1 row in set. Query took 0.344 seconds. ``` +## Reading from standard input + +On Unix-like systems you can pipe data into the CLI and query it by pointing the +`LOCATION` at the `/dev/stdin` pseudo-file: + +```console +$ cat hits.csv | datafusion-cli -c " +CREATE EXTERNAL TABLE hits STORED AS CSV LOCATION '/dev/stdin' OPTIONS ('format.has_header' 'true'); +SELECT count(*) FROM hits;" +``` + +This works for CSV, JSON, and Parquet. Because standard input is not seekable +(and Parquet stores its metadata at the end of the file), the CLI buffers the +entire input into memory before querying it, so the data must fit in memory. +Standard input is read only once: the buffered contents are reused for any +further tables backed by `/dev/stdin` in the same session. Those tables must +declare the same `STORED AS` format as the first one; a differing format is +rejected with an error. + +The SQL must be passed with `-c`/`--command` or `-f`/`--file` so that standard +input is free to carry the data. In the interactive shell (and when SQL is +piped to the CLI without `-c`/`-f`) standard input carries the SQL itself, and +`LOCATION '/dev/stdin'` returns an error. + **Why Wildcards Are Not Supported** Although wildcards (e.g., _.parquet or \*\*/_.parquet) may work for local diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 86f0f9f3a3cd5..abf1c39510e97 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -120,7 +120,7 @@ The following configuration settings are available: | datafusion.execution.parquet.content_defined_chunking.norm_level | 0 | Normalization level. Increasing this improves deduplication ratio but increases fragmentation. Recommended range is [-3, 3], default is 0. | | datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | | datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | -| datafusion.execution.enable_migration_aggregate | false | Temporary switch for aggregate stream implementations that are being migrated from `GroupedHashAggregateStream`. When set to true, DataFusion tries the migrated implementations when their preconditions are satisfied. When set to false, grouped aggregation falls back to `GroupedHashAggregateStream`. This option will be removed after the migration is finished. See for details. | +| datafusion.execution.enable_migration_aggregate | true | Temporary switch for aggregate stream implementations that are being migrated from `GroupedHashAggregateStream`. When set to true, DataFusion tries the migrated implementations when their preconditions are satisfied. When set to false, grouped aggregation falls back to `GroupedHashAggregateStream`. This option will be removed after the migration is finished. See for details. | | datafusion.execution.spill_compression | uncompressed | Sets the compression codec used when spilling data to disk. Since datafusion writes spill files using the Arrow IPC Stream format, only codecs supported by the Arrow IPC Stream Writer are allowed. Valid values are: uncompressed, lz4_frame, zstd. Note: lz4_frame offers faster (de)compression, but typically results in larger spill files. In contrast, zstd achieves higher compression ratios at the cost of slower (de)compression speed. | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index e5cd6f3d99711..83df7b06fd224 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -5702,6 +5702,7 @@ union_tag(union_expression) - [arrow_try_cast](#arrow_try_cast) - [arrow_typeof](#arrow_typeof) - [cast_to_type](#cast_to_type) +- [file_row_index](#file_row_index) - [get_field](#get_field) - [try_cast_to_type](#try_cast_to_type) - [version](#version) @@ -5885,6 +5886,27 @@ cast_to_type(expression, reference) +-----+ ``` +### `file_row_index` + +Returns the zero-based row offset within the source file +that produced the current row. + +The value is scoped to one file, so rows from different files in the same scan +can have the same row index. This function is intended to be rewritten at +file-scan time. If the input file is not known (for example, if this function +is evaluated outside a file scan, or was not pushed down into one), direct +evaluation returns an error. + +```sql +file_row_index() +``` + +#### Example + +```sql +SELECT file_row_index() FROM t; +``` + ### `get_field` Returns a field within a map or a struct with the given key. diff --git a/snowflake_flatten_validation.sql b/snowflake_flatten_validation.sql deleted file mode 100644 index cae6f5ea59e77..0000000000000 --- a/snowflake_flatten_validation.sql +++ /dev/null @@ -1,219 +0,0 @@ --- ============================================================================ --- Snowflake LATERAL FLATTEN validation queries --- --- Run this file against a real Snowflake instance to verify that the --- Unparser-generated SQL is syntactically and semantically correct. --- --- Each section shows: --- 1. The DataFusion input (SQL parsed by the planner) --- 2. The Snowflake SQL produced by the Unparser --- --- NOTE: The Unparser emits array literals as [1, 2, 3] (DataFusion syntax). --- Snowflake requires ARRAY_CONSTRUCT(1, 2, 3). The queries below use --- ARRAY_CONSTRUCT so they can run directly on Snowflake. The exact Unparser --- output is shown in the "Unparser output:" comment above each query. --- ============================================================================ - --- ---------------------------------------------------------------------------- --- Setup: create and seed test tables --- ---------------------------------------------------------------------------- - -CREATE OR REPLACE TABLE source ( - items ARRAY -); - -INSERT INTO source SELECT PARSE_JSON('[1, 2, 3]'); -INSERT INTO source SELECT PARSE_JSON('["a", "b"]'); -INSERT INTO source SELECT NULL; - -CREATE OR REPLACE TABLE unnest_table ( - array_col ARRAY -); - -INSERT INTO unnest_table SELECT PARSE_JSON('[10, 20, 30]'); -INSERT INTO unnest_table SELECT PARSE_JSON('[40, 50]'); -INSERT INTO unnest_table SELECT NULL; - -CREATE OR REPLACE TABLE multi_array_table ( - column_a ARRAY, - column_b ARRAY -); - -INSERT INTO multi_array_table SELECT PARSE_JSON('[1, 2, 3]'), PARSE_JSON('["x", "y"]'); -INSERT INTO multi_array_table SELECT PARSE_JSON('[4]'), PARSE_JSON('["z"]'); - --- ============================================================================ --- Roundtrip tests: SQL parsed → plan → Snowflake SQL --- ============================================================================ - --- -------------------------------------------------------------------------- --- Test: snowflake_unnest_to_lateral_flatten_simple --- DataFusion input: SELECT * FROM UNNEST([1,2,3]) --- Unparser output: SELECT "_unnest_1"."VALUE" FROM LATERAL FLATTEN(INPUT => [1, 2, 3]) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" -FROM LATERAL FLATTEN(INPUT => ARRAY_CONSTRUCT(1, 2, 3)) AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_implicit_from --- DataFusion input: SELECT UNNEST([1,2,3]) --- Unparser output: SELECT "_unnest_1"."VALUE" FROM LATERAL FLATTEN(INPUT => [1, 2, 3]) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" -FROM LATERAL FLATTEN(INPUT => ARRAY_CONSTRUCT(1, 2, 3)) AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_string_array --- DataFusion input: SELECT * FROM UNNEST(['a','b','c']) --- Unparser output: SELECT "_unnest_1"."VALUE" FROM LATERAL FLATTEN(INPUT => ['a', 'b', 'c']) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" -FROM LATERAL FLATTEN(INPUT => ARRAY_CONSTRUCT('a', 'b', 'c')) AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_select_unnest_with_alias --- DataFusion input: SELECT UNNEST([1,2,3]) as c1 --- Unparser output: SELECT "_unnest_1"."VALUE" AS "c1" FROM LATERAL FLATTEN(INPUT => [1, 2, 3]) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" AS "c1" -FROM LATERAL FLATTEN(INPUT => ARRAY_CONSTRUCT(1, 2, 3)) AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_from_unnest_with_table_alias --- DataFusion input: SELECT * FROM UNNEST([1,2,3]) AS t1 (c1) --- Unparser output: SELECT "t1"."VALUE" FROM LATERAL FLATTEN(INPUT => [1, 2, 3]) AS "t1" --- -------------------------------------------------------------------------- -SELECT "t1"."VALUE" -FROM LATERAL FLATTEN(INPUT => ARRAY_CONSTRUCT(1, 2, 3)) AS "t1"; - --- ============================================================================ --- Plan-built tests: LogicalPlan → Snowflake SQL --- These use a table called "source" with an ARRAY column "items". --- ============================================================================ - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_limit_between_projection_and_unnest --- Plan: Projection → Limit → Unnest → Projection → TableScan --- Unparser output: SELECT "_unnest_1"."VALUE" AS "item" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" LIMIT 5 --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" AS "item" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" -LIMIT 5; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_sort_between_projection_and_unnest --- Plan: Projection → Sort → Unnest → Projection → TableScan --- Unparser output: SELECT "_unnest_1"."VALUE" AS "item" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" ORDER BY "_unnest_1"."VALUE" ASC NULLS FIRST --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" AS "item" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" -ORDER BY "_unnest_1"."VALUE" ASC NULLS FIRST; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_limit_between_projection_and_unnest_with_subquery_alias --- Plan: Projection → Limit → Unnest → SubqueryAlias → Projection → TableScan --- Unparser output: SELECT "_unnest_1"."VALUE" AS "item" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" LIMIT 10 --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" AS "item" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" -LIMIT 10; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_composed_expression_wrapping_unnest --- Plan: Projection(CAST(placeholder AS Int64)) → Unnest → Projection → TableScan --- Unparser output: SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "item_id" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "item_id" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_composed_expression_with_limit --- Plan: Projection(CAST) → Limit → Unnest → Projection → TableScan --- Unparser output: SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "item_id" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" LIMIT 5 --- -------------------------------------------------------------------------- -SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "item_id" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" -LIMIT 5; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_multi_expression_projection --- Plan: Projection([CAST AS Int64, CAST AS Utf8]) → Unnest → Projection → TableScan --- Unparser output: SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "a", CAST("_unnest_1"."VALUE" AS VARCHAR) AS "b" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "a", - CAST("_unnest_1"."VALUE" AS VARCHAR) AS "b" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_multi_expression_with_limit --- Plan: Projection([CAST, CAST]) → Limit → Unnest → Projection → TableScan --- Unparser output: SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "a", CAST("_unnest_1"."VALUE" AS VARCHAR) AS "b" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" LIMIT 10 --- -------------------------------------------------------------------------- -SELECT CAST("_unnest_1"."VALUE" AS BIGINT) AS "a", - CAST("_unnest_1"."VALUE" AS VARCHAR) AS "b" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" -LIMIT 10; - --- -------------------------------------------------------------------------- --- Test: snowflake_unnest_through_subquery_alias --- Plan: Projection → Unnest → SubqueryAlias → Projection → TableScan --- Unparser output: SELECT "_unnest_1"."VALUE" AS "item" FROM "source" CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" AS "item" -FROM "source" -CROSS JOIN LATERAL FLATTEN(INPUT => "source"."items", OUTER => true) AS "_unnest_1"; - --- ============================================================================ --- Roundtrip tests with table columns --- ============================================================================ - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_unnest_from_subselect --- DataFusion input: SELECT UNNEST(array_col) FROM (SELECT array_col FROM unnest_table WHERE array_col IS NOT NULL LIMIT 3) --- Unparser output: SELECT "_unnest_1"."VALUE" FROM (SELECT "unnest_table"."array_col" FROM "unnest_table" WHERE "unnest_table"."array_col" IS NOT NULL LIMIT 3) CROSS JOIN LATERAL FLATTEN(INPUT => "unnest_table"."array_col") AS "_unnest_1" --- -------------------------------------------------------------------------- -SELECT "_unnest_1"."VALUE" -FROM ( - SELECT "unnest_table"."array_col" - FROM "unnest_table" - WHERE "unnest_table"."array_col" IS NOT NULL - LIMIT 3 -) CROSS JOIN LATERAL FLATTEN(INPUT => "unnest_table"."array_col") AS "_unnest_1"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_cross_join_unnest_table_column --- DataFusion input: SELECT * FROM multi_array_table CROSS JOIN UNNEST(column_a) AS a (a) --- Unparser output: SELECT "multi_array_table"."column_a", "multi_array_table"."column_b", "a"."VALUE" FROM "multi_array_table" CROSS JOIN LATERAL FLATTEN(INPUT => "multi_array_table"."column_a") AS "a" --- -------------------------------------------------------------------------- -SELECT "multi_array_table"."column_a", - "multi_array_table"."column_b", - "a"."VALUE" -FROM "multi_array_table" -CROSS JOIN LATERAL FLATTEN(INPUT => "multi_array_table"."column_a") AS "a"; - --- -------------------------------------------------------------------------- --- Test: snowflake_flatten_multiple_unnest_cross_join --- DataFusion input: SELECT a.a, b.b FROM multi_array_table --- CROSS JOIN UNNEST(column_a) AS a (a) --- CROSS JOIN UNNEST(column_b) AS b (b) --- Unparser output: SELECT "a"."VALUE", "b"."VALUE" FROM "multi_array_table" CROSS JOIN LATERAL FLATTEN(INPUT => "multi_array_table"."column_a") AS "a" CROSS JOIN LATERAL FLATTEN(INPUT => "multi_array_table"."column_b") AS "b" --- -------------------------------------------------------------------------- -SELECT "a"."VALUE", - "b"."VALUE" -FROM "multi_array_table" -CROSS JOIN LATERAL FLATTEN(INPUT => "multi_array_table"."column_a") AS "a" -CROSS JOIN LATERAL FLATTEN(INPUT => "multi_array_table"."column_b") AS "b"; - --- ============================================================================ --- Cleanup --- ============================================================================ --- DROP TABLE IF EXISTS source; --- DROP TABLE IF EXISTS unnest_table; --- DROP TABLE IF EXISTS multi_array_table; diff --git a/uv.lock b/uv.lock index f86b732dfd6d5..bdda81c5f9777 100644 --- a/uv.lock +++ b/uv.lock @@ -240,61 +240,61 @@ wheels = [ [[package]] name = "cryptography" -version = "46.0.7" +version = "48.0.1" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "cffi", marker = "platform_python_implementation != 'PyPy'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/47/93/ac8f3d5ff04d54bc814e961a43ae5b0b146154c89c61b47bb07557679b18/cryptography-46.0.7.tar.gz", hash = "sha256:e4cfd68c5f3e0bfdad0d38e023239b96a2fe84146481852dffbcca442c245aa5", size = 750652, upload-time = "2026-04-08T01:57:54.692Z" } +sdist = { url = "https://files.pythonhosted.org/packages/12/45/870e7f4bef50e5f53b9f51d4428aee5290eedf58ba443f16b1ebb7ab8e66/cryptography-48.0.1.tar.gz", hash = "sha256:266f4ee051abb2f725b74ef8072b521ce1feacf685a3364fa6a6b45548db791a", size = 832989, upload-time = "2026-06-09T22:32:31.8Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/0b/5d/4a8f770695d73be252331e60e526291e3df0c9b27556a90a6b47bccca4c2/cryptography-46.0.7-cp311-abi3-macosx_10_9_universal2.whl", hash = "sha256:ea42cbe97209df307fdc3b155f1b6fa2577c0defa8f1f7d3be7d31d189108ad4", size = 7179869, upload-time = "2026-04-08T01:56:17.157Z" }, - { url = "https://files.pythonhosted.org/packages/5f/45/6d80dc379b0bbc1f9d1e429f42e4cb9e1d319c7a8201beffd967c516ea01/cryptography-46.0.7-cp311-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:b36a4695e29fe69215d75960b22577197aca3f7a25b9cf9d165dcfe9d80bc325", size = 4275492, upload-time = "2026-04-08T01:56:19.36Z" }, - { url = "https://files.pythonhosted.org/packages/4a/9a/1765afe9f572e239c3469f2cb429f3ba7b31878c893b246b4b2994ffe2fe/cryptography-46.0.7-cp311-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:5ad9ef796328c5e3c4ceed237a183f5d41d21150f972455a9d926593a1dcb308", size = 4426670, upload-time = "2026-04-08T01:56:21.415Z" }, - { url = "https://files.pythonhosted.org/packages/8f/3e/af9246aaf23cd4ee060699adab1e47ced3f5f7e7a8ffdd339f817b446462/cryptography-46.0.7-cp311-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:73510b83623e080a2c35c62c15298096e2a5dc8d51c3b4e1740211839d0dea77", size = 4280275, upload-time = "2026-04-08T01:56:23.539Z" }, - { url = "https://files.pythonhosted.org/packages/0f/54/6bbbfc5efe86f9d71041827b793c24811a017c6ac0fd12883e4caa86b8ed/cryptography-46.0.7-cp311-abi3-manylinux_2_28_ppc64le.whl", hash = "sha256:cbd5fb06b62bd0721e1170273d3f4d5a277044c47ca27ee257025146c34cbdd1", size = 4928402, upload-time = "2026-04-08T01:56:25.624Z" }, - { url = "https://files.pythonhosted.org/packages/2d/cf/054b9d8220f81509939599c8bdbc0c408dbd2bdd41688616a20731371fe0/cryptography-46.0.7-cp311-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:420b1e4109cc95f0e5700eed79908cef9268265c773d3a66f7af1eef53d409ef", size = 4459985, upload-time = "2026-04-08T01:56:27.309Z" }, - { url = "https://files.pythonhosted.org/packages/f9/46/4e4e9c6040fb01c7467d47217d2f882daddeb8828f7df800cb806d8a2288/cryptography-46.0.7-cp311-abi3-manylinux_2_31_armv7l.whl", hash = "sha256:24402210aa54baae71d99441d15bb5a1919c195398a87b563df84468160a65de", size = 3990652, upload-time = "2026-04-08T01:56:29.095Z" }, - { url = "https://files.pythonhosted.org/packages/36/5f/313586c3be5a2fbe87e4c9a254207b860155a8e1f3cca99f9910008e7d08/cryptography-46.0.7-cp311-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:8a469028a86f12eb7d2fe97162d0634026d92a21f3ae0ac87ed1c4a447886c83", size = 4279805, upload-time = "2026-04-08T01:56:30.928Z" }, - { url = "https://files.pythonhosted.org/packages/69/33/60dfc4595f334a2082749673386a4d05e4f0cf4df8248e63b2c3437585f2/cryptography-46.0.7-cp311-abi3-manylinux_2_34_ppc64le.whl", hash = "sha256:9694078c5d44c157ef3162e3bf3946510b857df5a3955458381d1c7cfc143ddb", size = 4892883, upload-time = "2026-04-08T01:56:32.614Z" }, - { url = "https://files.pythonhosted.org/packages/c7/0b/333ddab4270c4f5b972f980adef4faa66951a4aaf646ca067af597f15563/cryptography-46.0.7-cp311-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:42a1e5f98abb6391717978baf9f90dc28a743b7d9be7f0751a6f56a75d14065b", size = 4459756, upload-time = "2026-04-08T01:56:34.306Z" }, - { url = "https://files.pythonhosted.org/packages/d2/14/633913398b43b75f1234834170947957c6b623d1701ffc7a9600da907e89/cryptography-46.0.7-cp311-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:91bbcb08347344f810cbe49065914fe048949648f6bd5c2519f34619142bbe85", size = 4410244, upload-time = "2026-04-08T01:56:35.977Z" }, - { url = "https://files.pythonhosted.org/packages/10/f2/19ceb3b3dc14009373432af0c13f46aa08e3ce334ec6eff13492e1812ccd/cryptography-46.0.7-cp311-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:5d1c02a14ceb9148cc7816249f64f623fbfee39e8c03b3650d842ad3f34d637e", size = 4674868, upload-time = "2026-04-08T01:56:38.034Z" }, - { url = "https://files.pythonhosted.org/packages/1a/bb/a5c213c19ee94b15dfccc48f363738633a493812687f5567addbcbba9f6f/cryptography-46.0.7-cp311-abi3-win32.whl", hash = "sha256:d23c8ca48e44ee015cd0a54aeccdf9f09004eba9fc96f38c911011d9ff1bd457", size = 3026504, upload-time = "2026-04-08T01:56:39.666Z" }, - { url = "https://files.pythonhosted.org/packages/2b/02/7788f9fefa1d060ca68717c3901ae7fffa21ee087a90b7f23c7a603c32ae/cryptography-46.0.7-cp311-abi3-win_amd64.whl", hash = "sha256:397655da831414d165029da9bc483bed2fe0e75dde6a1523ec2fe63f3c46046b", size = 3488363, upload-time = "2026-04-08T01:56:41.893Z" }, - { url = "https://files.pythonhosted.org/packages/7b/56/15619b210e689c5403bb0540e4cb7dbf11a6bf42e483b7644e471a2812b3/cryptography-46.0.7-cp314-cp314t-macosx_10_9_universal2.whl", hash = "sha256:d151173275e1728cf7839aaa80c34fe550c04ddb27b34f48c232193df8db5842", size = 7119671, upload-time = "2026-04-08T01:56:44Z" }, - { url = "https://files.pythonhosted.org/packages/74/66/e3ce040721b0b5599e175ba91ab08884c75928fbeb74597dd10ef13505d2/cryptography-46.0.7-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:db0f493b9181c7820c8134437eb8b0b4792085d37dbb24da050476ccb664e59c", size = 4268551, upload-time = "2026-04-08T01:56:46.071Z" }, - { url = "https://files.pythonhosted.org/packages/03/11/5e395f961d6868269835dee1bafec6a1ac176505a167f68b7d8818431068/cryptography-46.0.7-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:ebd6daf519b9f189f85c479427bbd6e9c9037862cf8fe89ee35503bd209ed902", size = 4408887, upload-time = "2026-04-08T01:56:47.718Z" }, - { url = "https://files.pythonhosted.org/packages/40/53/8ed1cf4c3b9c8e611e7122fb56f1c32d09e1fff0f1d77e78d9ff7c82653e/cryptography-46.0.7-cp314-cp314t-manylinux_2_28_aarch64.whl", hash = "sha256:b7b412817be92117ec5ed95f880defe9cf18a832e8cafacf0a22337dc1981b4d", size = 4271354, upload-time = "2026-04-08T01:56:49.312Z" }, - { url = "https://files.pythonhosted.org/packages/50/46/cf71e26025c2e767c5609162c866a78e8a2915bbcfa408b7ca495c6140c4/cryptography-46.0.7-cp314-cp314t-manylinux_2_28_ppc64le.whl", hash = "sha256:fbfd0e5f273877695cb93baf14b185f4878128b250cc9f8e617ea0c025dfb022", size = 4905845, upload-time = "2026-04-08T01:56:50.916Z" }, - { url = "https://files.pythonhosted.org/packages/c0/ea/01276740375bac6249d0a971ebdf6b4dc9ead0ee0a34ef3b5a88c1a9b0d4/cryptography-46.0.7-cp314-cp314t-manylinux_2_28_x86_64.whl", hash = "sha256:ffca7aa1d00cf7d6469b988c581598f2259e46215e0140af408966a24cf086ce", size = 4444641, upload-time = "2026-04-08T01:56:52.882Z" }, - { url = "https://files.pythonhosted.org/packages/3d/4c/7d258f169ae71230f25d9f3d06caabcff8c3baf0978e2b7d65e0acac3827/cryptography-46.0.7-cp314-cp314t-manylinux_2_31_armv7l.whl", hash = "sha256:60627cf07e0d9274338521205899337c5d18249db56865f943cbe753aa96f40f", size = 3967749, upload-time = "2026-04-08T01:56:54.597Z" }, - { url = "https://files.pythonhosted.org/packages/b5/2a/2ea0767cad19e71b3530e4cad9605d0b5e338b6a1e72c37c9c1ceb86c333/cryptography-46.0.7-cp314-cp314t-manylinux_2_34_aarch64.whl", hash = "sha256:80406c3065e2c55d7f49a9550fe0c49b3f12e5bfff5dedb727e319e1afb9bf99", size = 4270942, upload-time = "2026-04-08T01:56:56.416Z" }, - { url = "https://files.pythonhosted.org/packages/41/3d/fe14df95a83319af25717677e956567a105bb6ab25641acaa093db79975d/cryptography-46.0.7-cp314-cp314t-manylinux_2_34_ppc64le.whl", hash = "sha256:c5b1ccd1239f48b7151a65bc6dd54bcfcc15e028c8ac126d3fada09db0e07ef1", size = 4871079, upload-time = "2026-04-08T01:56:58.31Z" }, - { url = "https://files.pythonhosted.org/packages/9c/59/4a479e0f36f8f378d397f4eab4c850b4ffb79a2f0d58704b8fa0703ddc11/cryptography-46.0.7-cp314-cp314t-manylinux_2_34_x86_64.whl", hash = "sha256:d5f7520159cd9c2154eb61eb67548ca05c5774d39e9c2c4339fd793fe7d097b2", size = 4443999, upload-time = "2026-04-08T01:57:00.508Z" }, - { url = "https://files.pythonhosted.org/packages/28/17/b59a741645822ec6d04732b43c5d35e4ef58be7bfa84a81e5ae6f05a1d33/cryptography-46.0.7-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:fcd8eac50d9138c1d7fc53a653ba60a2bee81a505f9f8850b6b2888555a45d0e", size = 4399191, upload-time = "2026-04-08T01:57:02.654Z" }, - { url = "https://files.pythonhosted.org/packages/59/6a/bb2e166d6d0e0955f1e9ff70f10ec4b2824c9cfcdb4da772c7dd69cc7d80/cryptography-46.0.7-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:65814c60f8cc400c63131584e3e1fad01235edba2614b61fbfbfa954082db0ee", size = 4655782, upload-time = "2026-04-08T01:57:04.592Z" }, - { url = "https://files.pythonhosted.org/packages/95/b6/3da51d48415bcb63b00dc17c2eff3a651b7c4fed484308d0f19b30e8cb2c/cryptography-46.0.7-cp314-cp314t-win32.whl", hash = "sha256:fdd1736fed309b4300346f88f74cd120c27c56852c3838cab416e7a166f67298", size = 3002227, upload-time = "2026-04-08T01:57:06.91Z" }, - { url = "https://files.pythonhosted.org/packages/32/a8/9f0e4ed57ec9cebe506e58db11ae472972ecb0c659e4d52bbaee80ca340a/cryptography-46.0.7-cp314-cp314t-win_amd64.whl", hash = "sha256:e06acf3c99be55aa3b516397fe42f5855597f430add9c17fa46bf2e0fb34c9bb", size = 3475332, upload-time = "2026-04-08T01:57:08.807Z" }, - { url = "https://files.pythonhosted.org/packages/a7/7f/cd42fc3614386bc0c12f0cb3c4ae1fc2bbca5c9662dfed031514911d513d/cryptography-46.0.7-cp38-abi3-macosx_10_9_universal2.whl", hash = "sha256:462ad5cb1c148a22b2e3bcc5ad52504dff325d17daf5df8d88c17dda1f75f2a4", size = 7165618, upload-time = "2026-04-08T01:57:10.645Z" }, - { url = "https://files.pythonhosted.org/packages/a5/d0/36a49f0262d2319139d2829f773f1b97ef8aef7f97e6e5bd21455e5a8fb5/cryptography-46.0.7-cp38-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:84d4cced91f0f159a7ddacad249cc077e63195c36aac40b4150e7a57e84fffe7", size = 4270628, upload-time = "2026-04-08T01:57:12.885Z" }, - { url = "https://files.pythonhosted.org/packages/8a/6c/1a42450f464dda6ffbe578a911f773e54dd48c10f9895a23a7e88b3e7db5/cryptography-46.0.7-cp38-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:128c5edfe5e5938b86b03941e94fac9ee793a94452ad1365c9fc3f4f62216832", size = 4415405, upload-time = "2026-04-08T01:57:14.923Z" }, - { url = "https://files.pythonhosted.org/packages/9a/92/4ed714dbe93a066dc1f4b4581a464d2d7dbec9046f7c8b7016f5286329e2/cryptography-46.0.7-cp38-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:5e51be372b26ef4ba3de3c167cd3d1022934bc838ae9eaad7e644986d2a3d163", size = 4272715, upload-time = "2026-04-08T01:57:16.638Z" }, - { url = "https://files.pythonhosted.org/packages/b7/e6/a26b84096eddd51494bba19111f8fffe976f6a09f132706f8f1bf03f51f7/cryptography-46.0.7-cp38-abi3-manylinux_2_28_ppc64le.whl", hash = "sha256:cdf1a610ef82abb396451862739e3fc93b071c844399e15b90726ef7470eeaf2", size = 4918400, upload-time = "2026-04-08T01:57:19.021Z" }, - { url = "https://files.pythonhosted.org/packages/c7/08/ffd537b605568a148543ac3c2b239708ae0bd635064bab41359252ef88ed/cryptography-46.0.7-cp38-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:1d25aee46d0c6f1a501adcddb2d2fee4b979381346a78558ed13e50aa8a59067", size = 4450634, upload-time = "2026-04-08T01:57:21.185Z" }, - { url = "https://files.pythonhosted.org/packages/16/01/0cd51dd86ab5b9befe0d031e276510491976c3a80e9f6e31810cce46c4ad/cryptography-46.0.7-cp38-abi3-manylinux_2_31_armv7l.whl", hash = "sha256:cdfbe22376065ffcf8be74dc9a909f032df19bc58a699456a21712d6e5eabfd0", size = 3985233, upload-time = "2026-04-08T01:57:22.862Z" }, - { url = "https://files.pythonhosted.org/packages/92/49/819d6ed3a7d9349c2939f81b500a738cb733ab62fbecdbc1e38e83d45e12/cryptography-46.0.7-cp38-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:abad9dac36cbf55de6eb49badd4016806b3165d396f64925bf2999bcb67837ba", size = 4271955, upload-time = "2026-04-08T01:57:24.814Z" }, - { url = "https://files.pythonhosted.org/packages/80/07/ad9b3c56ebb95ed2473d46df0847357e01583f4c52a85754d1a55e29e4d0/cryptography-46.0.7-cp38-abi3-manylinux_2_34_ppc64le.whl", hash = "sha256:935ce7e3cfdb53e3536119a542b839bb94ec1ad081013e9ab9b7cfd478b05006", size = 4879888, upload-time = "2026-04-08T01:57:26.88Z" }, - { url = "https://files.pythonhosted.org/packages/b8/c7/201d3d58f30c4c2bdbe9b03844c291feb77c20511cc3586daf7edc12a47b/cryptography-46.0.7-cp38-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:35719dc79d4730d30f1c2b6474bd6acda36ae2dfae1e3c16f2051f215df33ce0", size = 4449961, upload-time = "2026-04-08T01:57:29.068Z" }, - { url = "https://files.pythonhosted.org/packages/a5/ef/649750cbf96f3033c3c976e112265c33906f8e462291a33d77f90356548c/cryptography-46.0.7-cp38-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:7bbc6ccf49d05ac8f7d7b5e2e2c33830d4fe2061def88210a126d130d7f71a85", size = 4401696, upload-time = "2026-04-08T01:57:31.029Z" }, - { url = "https://files.pythonhosted.org/packages/41/52/a8908dcb1a389a459a29008c29966c1d552588d4ae6d43f3a1a4512e0ebe/cryptography-46.0.7-cp38-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:a1529d614f44b863a7b480c6d000fe93b59acee9c82ffa027cfadc77521a9f5e", size = 4664256, upload-time = "2026-04-08T01:57:33.144Z" }, - { url = "https://files.pythonhosted.org/packages/4b/fa/f0ab06238e899cc3fb332623f337a7364f36f4bb3f2534c2bb95a35b132c/cryptography-46.0.7-cp38-abi3-win32.whl", hash = "sha256:f247c8c1a1fb45e12586afbb436ef21ff1e80670b2861a90353d9b025583d246", size = 3013001, upload-time = "2026-04-08T01:57:34.933Z" }, - { url = "https://files.pythonhosted.org/packages/d2/f1/00ce3bde3ca542d1acd8f8cfa38e446840945aa6363f9b74746394b14127/cryptography-46.0.7-cp38-abi3-win_amd64.whl", hash = "sha256:506c4ff91eff4f82bdac7633318a526b1d1309fc07ca76a3ad182cb5b686d6d3", size = 3472985, upload-time = "2026-04-08T01:57:36.714Z" }, - { url = "https://files.pythonhosted.org/packages/63/0c/dca8abb64e7ca4f6b2978769f6fea5ad06686a190cec381f0a796fdcaaba/cryptography-46.0.7-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:fc9ab8856ae6cf7c9358430e49b368f3108f050031442eaeb6b9d87e4dcf4e4f", size = 3476879, upload-time = "2026-04-08T01:57:38.664Z" }, - { url = "https://files.pythonhosted.org/packages/3a/ea/075aac6a84b7c271578d81a2f9968acb6e273002408729f2ddff517fed4a/cryptography-46.0.7-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:d3b99c535a9de0adced13d159c5a9cf65c325601aa30f4be08afd680643e9c15", size = 4219700, upload-time = "2026-04-08T01:57:40.625Z" }, - { url = "https://files.pythonhosted.org/packages/6c/7b/1c55db7242b5e5612b29fc7a630e91ee7a6e3c8e7bf5406d22e206875fbd/cryptography-46.0.7-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:d02c738dacda7dc2a74d1b2b3177042009d5cab7c7079db74afc19e56ca1b455", size = 4385982, upload-time = "2026-04-08T01:57:42.725Z" }, - { url = "https://files.pythonhosted.org/packages/cb/da/9870eec4b69c63ef5925bf7d8342b7e13bc2ee3d47791461c4e49ca212f4/cryptography-46.0.7-pp311-pypy311_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:04959522f938493042d595a736e7dbdff6eb6cc2339c11465b3ff89343b65f65", size = 4219115, upload-time = "2026-04-08T01:57:44.939Z" }, - { url = "https://files.pythonhosted.org/packages/f4/72/05aa5832b82dd341969e9a734d1812a6aadb088d9eb6f0430fc337cc5a8f/cryptography-46.0.7-pp311-pypy311_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:3986ac1dee6def53797289999eabe84798ad7817f3e97779b5061a95b0ee4968", size = 4385479, upload-time = "2026-04-08T01:57:46.86Z" }, - { url = "https://files.pythonhosted.org/packages/20/2a/1b016902351a523aa2bd446b50a5bc1175d7a7d1cf90fe2ef904f9b84ebc/cryptography-46.0.7-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:258514877e15963bd43b558917bc9f54cf7cf866c38aa576ebf47a77ddbc43a4", size = 3412829, upload-time = "2026-04-08T01:57:48.874Z" }, + { url = "https://files.pythonhosted.org/packages/1b/bc/ee4137cbbe105652c0ee4252792b78fc8e7afa4b8e61d9d5dc05a7f45731/cryptography-48.0.1-cp311-abi3-macosx_10_9_universal2.whl", hash = "sha256:3e4a1a3232eef2e6c732827d5722db29a0cc8b27af2a4d865b094cf954be9ca1", size = 8008324, upload-time = "2026-06-09T22:31:00.702Z" }, + { url = "https://files.pythonhosted.org/packages/d5/85/6379d42181bfc713094f081360fc5784d6c816b599d45e7f082502d173ce/cryptography-48.0.1-cp311-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:32143b24adb918f078134e1e230f1eb8cc04886b92c28b5f0041aaf3e5699225", size = 4696243, upload-time = "2026-06-09T22:32:33.446Z" }, + { url = "https://files.pythonhosted.org/packages/9c/87/c85d147b53323c7eb4d850920c8901377323c2a0ff8d79c262d4fee89aa2/cryptography-48.0.1-cp311-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:f0d27a5696721ef7a672b8c810f6aded391058e0b9486e63e6d93baf765da691", size = 4713235, upload-time = "2026-06-09T22:31:40.141Z" }, + { url = "https://files.pythonhosted.org/packages/79/58/67cbf8cf1ee7c54b439ca07bbecf8362c07afc11a3724fea70f745784add/cryptography-48.0.1-cp311-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:eb86ce1af36fe65041b6db9a8bb064ee621a7e5fded0f80d475ec243477cd242", size = 4702323, upload-time = "2026-06-09T22:31:42.191Z" }, + { url = "https://files.pythonhosted.org/packages/89/c6/24266ac10c47f6cd2a865f4446062b466da1d1f10b27189eac00e61bf0c9/cryptography-48.0.1-cp311-abi3-manylinux_2_28_ppc64le.whl", hash = "sha256:b024e784ad6c077ee0147b35ea9cbfc1e34e1fd4c1dcca214c2794d73a12df08", size = 5300085, upload-time = "2026-06-09T22:31:58.703Z" }, + { url = "https://files.pythonhosted.org/packages/d2/bb/cc4b78784f97efc8c5874c2a9743708d172be6663024b34a0467885ae0c8/cryptography-48.0.1-cp311-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:3752f2dbc8f07a30aad2932c986cea495b03bb554887828225da104f732852b6", size = 4746137, upload-time = "2026-06-09T22:31:31.01Z" }, + { url = "https://files.pythonhosted.org/packages/1f/52/0c44de3f5267f8fbe8e835138017522a333436166e406f0db9b9e6e3033f/cryptography-48.0.1-cp311-abi3-manylinux_2_31_armv7l.whl", hash = "sha256:bd81490cd5801d755cf97bb68ac191f14b708470b1c7cf4580f669b9c9264cd8", size = 4333867, upload-time = "2026-06-09T22:32:28.096Z" }, + { url = "https://files.pythonhosted.org/packages/9a/2e/772d7adbfa931537bc401640b7cac9976bff689bda187833e5d63b428e49/cryptography-48.0.1-cp311-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:66fd0771e7b9c6dcd44cf1120690d2338d16d72795cf40cae2786a39eba65429", size = 4701805, upload-time = "2026-06-09T22:31:38.284Z" }, + { url = "https://files.pythonhosted.org/packages/f8/a3/b06844f303873493c963caf581c04df31c7035e0c1b0f02c4814d319ec80/cryptography-48.0.1-cp311-abi3-manylinux_2_34_ppc64le.whl", hash = "sha256:3fd2ca57062b241c856670b073487d2e86c4637937ca5601e48f97bf8e11fc8f", size = 5258461, upload-time = "2026-06-09T22:31:04.187Z" }, + { url = "https://files.pythonhosted.org/packages/9f/13/8b765e2e12b07c74941caadb9d1c8fdc006c4dfbf2b8f2d610519758954d/cryptography-48.0.1-cp311-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:0ee6ea481db1ab889cba043ec1eda17bb9c1ea79db6722f779c3667f9f70322f", size = 4745488, upload-time = "2026-06-09T22:32:30.07Z" }, + { url = "https://files.pythonhosted.org/packages/2e/aa/48972bce55049b32a94f4907eda4d75fa385aad8a39506cc2fc72196ecf0/cryptography-48.0.1-cp311-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:f2ceef93cb096aa3c4cc4b5c94ca6131f9196d28c64d6111533402a9b2054d41", size = 4830256, upload-time = "2026-06-09T22:31:43.868Z" }, + { url = "https://files.pythonhosted.org/packages/47/a2/e5079a032fb85cf6005046ca92bbd78b0c82dad2b5751ab8c311659da06f/cryptography-48.0.1-cp311-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:9bd3f92d76217892b15df84ca256c2c113d386fdda7a7d8691aeeced976507c6", size = 4979117, upload-time = "2026-06-09T22:31:05.845Z" }, + { url = "https://files.pythonhosted.org/packages/b7/a0/8f50cae9c74e718ed769d63ed5c74bd0ea830c9550a74629cebd1b9c7bc7/cryptography-48.0.1-cp311-abi3-win32.whl", hash = "sha256:b9a32b876490d66c8bcc9963ef220199569748434ab01a9d6aaeabf88e7f5158", size = 3304154, upload-time = "2026-06-09T22:32:16.845Z" }, + { url = "https://files.pythonhosted.org/packages/c5/69/0572c77dbace6fef72f33755bd52ea399c71367250d366237f8691826b9e/cryptography-48.0.1-cp311-abi3-win_amd64.whl", hash = "sha256:39489bfca54c7a1f6b297efcd8bc608ab92d16c4ca631b0cad4da46724588b24", size = 3817138, upload-time = "2026-06-09T22:32:00.388Z" }, + { url = "https://files.pythonhosted.org/packages/42/06/3e768b4c3bc78201583fa35a0e18f640dd782ff41afba88f8545481a8874/cryptography-48.0.1-cp314-cp314t-macosx_10_9_universal2.whl", hash = "sha256:f817adc181390bd54f2f700107a7419040fb7c1bdf2fc26f36551a06a68c3345", size = 7989830, upload-time = "2026-06-09T22:31:07.8Z" }, + { url = "https://files.pythonhosted.org/packages/8a/13/6476736484b94041110c8340a3eb63962fea4975baea8cb4a512adb44d4d/cryptography-48.0.1-cp314-cp314t-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:d5d30989c6917b478b5817902e85fddaea2261efa8648383d965381ccb9e1ac4", size = 4689201, upload-time = "2026-06-09T22:31:09.745Z" }, + { url = "https://files.pythonhosted.org/packages/79/62/65a87f34d2a431546e2509b85d55e8c90df86d668f6731da64d538512ac2/cryptography-48.0.1-cp314-cp314t-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:df637c05205ea7c1d7fbcbe54bbfea648a52951155f997af13d895d0ecc96991", size = 4702822, upload-time = "2026-06-09T22:32:24.409Z" }, + { url = "https://files.pythonhosted.org/packages/7f/59/810b5204b0a9b10f4b6bc06bd551a8b609803cd931806bc3b71884b225e5/cryptography-48.0.1-cp314-cp314t-manylinux_2_28_aarch64.whl", hash = "sha256:869c3b8a53bfe27147832df48b32adadf558249d50e76cb3769d40e986b13265", size = 4694875, upload-time = "2026-06-09T22:32:08.737Z" }, + { url = "https://files.pythonhosted.org/packages/24/dc/d8ca05ffea724eec6d232ea6f18e74c269eb6bdfdcc9bfba689790d1325f/cryptography-48.0.1-cp314-cp314t-manylinux_2_28_ppc64le.whl", hash = "sha256:e361afba8918070d376df76f408a4f67fec0ee9cff81a99e48fe9a233ef59e17", size = 5290385, upload-time = "2026-06-09T22:31:15.212Z" }, + { url = "https://files.pythonhosted.org/packages/03/8c/3be6cb4da181f5bb6c19cf560c2359d60644a6b5fc5b57854e528f47b296/cryptography-48.0.1-cp314-cp314t-manylinux_2_28_x86_64.whl", hash = "sha256:d069066deead00ac7f090be101be875a06855908f7ec004c27b8fefb4acfb411", size = 4737082, upload-time = "2026-06-09T22:32:22.66Z" }, + { url = "https://files.pythonhosted.org/packages/aa/f6/d5f60a5a1434dbfd949e227fd0065d194c7e6b6ac526b17f5c06152b8231/cryptography-48.0.1-cp314-cp314t-manylinux_2_31_armv7l.whl", hash = "sha256:09f73a725d582cef64b91281a322cd798d14a33b2b6f2b7ad9531dc336d84c02", size = 4325328, upload-time = "2026-06-09T22:32:10.777Z" }, + { url = "https://files.pythonhosted.org/packages/17/b7/ba75dd947a14b6ad907b01ae8f6b5b348cdd1b48142f0063dee9e20c1d9d/cryptography-48.0.1-cp314-cp314t-manylinux_2_34_aarch64.whl", hash = "sha256:15254441469dd6bf027039453288e2072124f8b6603563f5d759e1c9b69273fa", size = 4694530, upload-time = "2026-06-09T22:31:53.105Z" }, + { url = "https://files.pythonhosted.org/packages/62/29/50d6b9e8aff12d8b67afaeb3569335e32dc83a5723e3bbded24fdac9f809/cryptography-48.0.1-cp314-cp314t-manylinux_2_34_ppc64le.whl", hash = "sha256:8ace4507d1e6533c125f4fac754f8bb8b6a74c08e92179dabd7e16571a3efbf3", size = 5245046, upload-time = "2026-06-09T22:31:25.774Z" }, + { url = "https://files.pythonhosted.org/packages/9f/04/618f4115cfc0add0838c82507aa18a346089428da8653ad38b3ff36f5cb3/cryptography-48.0.1-cp314-cp314t-manylinux_2_34_x86_64.whl", hash = "sha256:b4e391975f038e66432328639620a4aff2d307513b004f1ca06d6225bced815c", size = 4736660, upload-time = "2026-06-09T22:32:12.676Z" }, + { url = "https://files.pythonhosted.org/packages/24/9c/06e062462a0de28a3b3911322eded4c16deb9f441b1b7575d3dc59488ab5/cryptography-48.0.1-cp314-cp314t-musllinux_1_2_aarch64.whl", hash = "sha256:42fcd8e26fe555d9b3577a135f5091fefa0aa4e99129c23fb56787a1bd4ada72", size = 4822229, upload-time = "2026-06-09T22:31:17.062Z" }, + { url = "https://files.pythonhosted.org/packages/f4/be/0561971eaaee4b8a0e7d5113c536921063ab91aaf23278ac374eaf881e11/cryptography-48.0.1-cp314-cp314t-musllinux_1_2_x86_64.whl", hash = "sha256:c1400da5e32a43253392277eac7490a60e497d810a63dd5608d71bbd7af507c9", size = 4966364, upload-time = "2026-06-09T22:31:32.842Z" }, + { url = "https://files.pythonhosted.org/packages/a4/27/728c77876f12b000820b69ae490f3c4083775e79e07827e9e60be07ad209/cryptography-48.0.1-cp314-cp314t-win32.whl", hash = "sha256:0df56b056bc17c1b7d6821dfa65216e62bd232d8ab05eb3db44e71d235651471", size = 3278498, upload-time = "2026-06-09T22:31:29.154Z" }, + { url = "https://files.pythonhosted.org/packages/06/e3/79a612c6d7b1e6ee0edd43633d53035bec2cfb78c82b76f7864f39e36f34/cryptography-48.0.1-cp314-cp314t-win_amd64.whl", hash = "sha256:9de21387aa95e2a895823d0745b430bed4f33503ba9ab5e0b5311f33e37d66d2", size = 3798790, upload-time = "2026-06-09T22:31:56.697Z" }, + { url = "https://files.pythonhosted.org/packages/ca/6c/00fa2a95997164c8b2072ce327c23d4ab20809ccc323ea5fab91e53a4bba/cryptography-48.0.1-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:4fdc69f8e4316bcf0c8c8ec1f26f285d12e8142d88d96c876a59a03be3f6ae67", size = 7987408, upload-time = "2026-06-09T22:32:20.777Z" }, + { url = "https://files.pythonhosted.org/packages/b0/d9/45f309a7e4e5f3f8f121d6d3be9e94024a7726ec598d6e08ae04edb2f04d/cryptography-48.0.1-cp39-abi3-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:48fe40804d4caa2288f24e70ca8c64c42dd826da0ad7e4f1b41b2128d679e6c8", size = 4690196, upload-time = "2026-06-09T22:31:54.74Z" }, + { url = "https://files.pythonhosted.org/packages/5f/9f/a1bc8bcc798811b8527eb374bbccf30a3f3e806829d967118222bf1125eb/cryptography-48.0.1-cp39-abi3-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:86be3b1b0b6bf09482fb50a979c508d2950ed95f5621ec77f4e385962006b83a", size = 4696782, upload-time = "2026-06-09T22:31:45.615Z" }, + { url = "https://files.pythonhosted.org/packages/66/c2/81a4fb4e4373c500bb526bc337ac5719dd31dd15b970b84a238168c6aa08/cryptography-48.0.1-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:4ab0a343c807bbcd90c971cd1ecf072937cd01847a9e002bef88fb47ac6be577", size = 4696618, upload-time = "2026-06-09T22:31:11.564Z" }, + { url = "https://files.pythonhosted.org/packages/e5/0b/aa68b221dde92d09cb29a024ede17550ee21e77a404e59fc093c82bb51e1/cryptography-48.0.1-cp39-abi3-manylinux_2_28_ppc64le.whl", hash = "sha256:9621de99d2da096006b629979efd8ae7eb2d8b822488d0c89ee4000c306c59b1", size = 5289970, upload-time = "2026-06-09T22:31:20.368Z" }, + { url = "https://files.pythonhosted.org/packages/78/13/fba657f958d2af66ea959a4ba01212632089249d34af1ae48054136344d7/cryptography-48.0.1-cp39-abi3-manylinux_2_28_x86_64.whl", hash = "sha256:88c852a0ae366e262e5a1744b685e6a433dc8788dd2a277e418bf4904203609d", size = 4731873, upload-time = "2026-06-09T22:31:22.253Z" }, + { url = "https://files.pythonhosted.org/packages/4c/4c/9a964756d24a26b3e34dfcb16f961b89838786e6700b635b0d1e3adff4b6/cryptography-48.0.1-cp39-abi3-manylinux_2_31_armv7l.whl", hash = "sha256:43c5835e2cb98c8733d86f57d6fc879b613f5c3478607281c3e36daffc6dd8a6", size = 4330804, upload-time = "2026-06-09T22:31:36.56Z" }, + { url = "https://files.pythonhosted.org/packages/4b/0f/a10f3a6eb12950a10e3a874070283aa2dd5875b2bfd15fad8a3e17b3f13e/cryptography-48.0.1-cp39-abi3-manylinux_2_34_aarch64.whl", hash = "sha256:fe0180af5bf9236518a087e35bf2d9a347d5f5f51e63c579d683ddff424e3d46", size = 4696217, upload-time = "2026-06-09T22:31:13.351Z" }, + { url = "https://files.pythonhosted.org/packages/f3/6f/5cd12f951165ea73ef85266775d97e4c763b2474ccfd816dd69d3a18d6f8/cryptography-48.0.1-cp39-abi3-manylinux_2_34_ppc64le.whl", hash = "sha256:b7a2d1a937a738a881737cec135a38bb61470589b17515b9f73f571d0ae10401", size = 5245252, upload-time = "2026-06-09T22:32:02.193Z" }, + { url = "https://files.pythonhosted.org/packages/68/ab/8aaa12e4516ec4464033ab79b6f3b592bd5a92102467c4ace8a0d970203f/cryptography-48.0.1-cp39-abi3-manylinux_2_34_x86_64.whl", hash = "sha256:b74ca3b8e5ecdd833bf6a002ca41b4793bb27fb8f1c06ffaf2643c9e9140e31b", size = 4731388, upload-time = "2026-06-09T22:32:04.019Z" }, + { url = "https://files.pythonhosted.org/packages/1b/24/50027ea4dca85ec1f40688f3c24fb32ccacd520583c9592c3cc95628e6fb/cryptography-48.0.1-cp39-abi3-musllinux_1_2_aarch64.whl", hash = "sha256:2c37f2461406063b417837f5f3daab668652acd82423efcd7f0a9f04be972de1", size = 4824186, upload-time = "2026-06-09T22:32:18.707Z" }, + { url = "https://files.pythonhosted.org/packages/52/41/04cb5eb17085ade6f50cc611fb657df6a0f5885350de8764ece89c050197/cryptography-48.0.1-cp39-abi3-musllinux_1_2_x86_64.whl", hash = "sha256:86fe77abb1bd87afb251d4d02ada7ecf53a32cee9b67d976abb2e45a13297475", size = 4964539, upload-time = "2026-06-09T22:31:18.793Z" }, + { url = "https://files.pythonhosted.org/packages/36/bf/ed70785c496e89d7e73b7cda2d21f2447fd6d4e821714b8d04ff217fed92/cryptography-48.0.1-cp39-abi3-win32.whl", hash = "sha256:6b2c0c3e6ccf3ade7750f836ef3ee36eea250cc467d45c256895573ac08cc6f1", size = 3282307, upload-time = "2026-06-09T22:30:53.162Z" }, + { url = "https://files.pythonhosted.org/packages/b3/ff/371ea7d252656ee1eb6d83eeeef3d1d0c6baf1d6497687d081ea03814670/cryptography-48.0.1-cp39-abi3-win_amd64.whl", hash = "sha256:9a49ca6c81417f6a5edb50375a60cccdd70fa0a91a5211829dbea74eba94d2ac", size = 3793408, upload-time = "2026-06-09T22:32:15.191Z" }, + { url = "https://files.pythonhosted.org/packages/a9/d3/eb4e394e587341fdad09a09101fa76478ead3a78b0ad63e55c22f0d75c02/cryptography-48.0.1-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:08a597acce1ff37f347400087776599e2348a3a8bc53b44120e463cd274efe4a", size = 3951747, upload-time = "2026-06-09T22:31:23.871Z" }, + { url = "https://files.pythonhosted.org/packages/e0/4a/3f43451b4f858bfceaaaffc649e6e787e8d4fb332a1d443af39ab02cc8f1/cryptography-48.0.1-pp311-pypy311_pp73-manylinux_2_28_aarch64.whl", hash = "sha256:735824ec41b7f74a7c45fb1591349333e4c696cb6c044e5f46356e560143e4cd", size = 4641226, upload-time = "2026-06-09T22:31:02.532Z" }, + { url = "https://files.pythonhosted.org/packages/73/4e/855584c2c23b09e4ce2d3b9c30e983e679cd60b068c513c6bbdb91e11782/cryptography-48.0.1-pp311-pypy311_pp73-manylinux_2_28_x86_64.whl", hash = "sha256:92a46e1d638daa264ba2971c0b0489c9409787943efae4d60ffda3d091ef832c", size = 4668958, upload-time = "2026-06-09T22:32:06.213Z" }, + { url = "https://files.pythonhosted.org/packages/42/3b/d35750e41d803d1e516fd6d6011f065424924da7af1748cef4cc9cb3ede1/cryptography-48.0.1-pp311-pypy311_pp73-manylinux_2_34_aarch64.whl", hash = "sha256:7e234ac052af99f2700826a5c29ea99d9c1b1f80341cde62d11c8154dc8e0bd9", size = 4640793, upload-time = "2026-06-09T22:32:26.331Z" }, + { url = "https://files.pythonhosted.org/packages/ca/aa/cdb7181fe865285e87e96825aaab239400f1de0c3bfba9bd9769b79f1a92/cryptography-48.0.1-pp311-pypy311_pp73-manylinux_2_34_x86_64.whl", hash = "sha256:33842cf0888951cef5bc7ac724ab844a42044c1727b967b7f8997289a0464f92", size = 4668505, upload-time = "2026-06-09T22:31:27.534Z" }, + { url = "https://files.pythonhosted.org/packages/5d/8c/ce3823c06c2804f194f9e64f0d67fa3f4094a39f2bb1a990cd03603af8fc/cryptography-48.0.1-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:6184ca7b174f28d7c703f1290d4b297217c45355f77a98f67e9b7f14549ac54a", size = 3742204, upload-time = "2026-06-09T22:31:34.773Z" }, ] [[package]] @@ -350,8 +350,8 @@ dependencies = [ requires-dist = [ { name = "jinja2", specifier = ">=3.1.6,<4" }, { name = "maturin", specifier = ">=1.13.3,<2" }, - { name = "myst-parser", specifier = ">=5,<6" }, - { name = "pydata-sphinx-theme", specifier = ">=0.17.1,<1" }, + { name = "myst-parser", specifier = ">=5.1.0,<6" }, + { name = "pydata-sphinx-theme", specifier = ">=0.18.0,<1" }, { name = "setuptools", specifier = ">=82.0.1,<83" }, { name = "sphinx", specifier = ">=9,<10" }, { name = "sphinx-reredirects", specifier = ">=1.1,<2" }, @@ -465,14 +465,14 @@ wheels = [ [[package]] name = "markdown-it-py" -version = "4.0.0" +version = "4.2.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "mdurl" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/5b/f5/4ec618ed16cc4f8fb3b701563655a69816155e79e24a17b651541804721d/markdown_it_py-4.0.0.tar.gz", hash = "sha256:cb0a2b4aa34f932c007117b194e945bd74e0ec24133ceb5bac59009cda1cb9f3", size = 73070, upload-time = "2025-08-11T12:57:52.854Z" } +sdist = { url = "https://files.pythonhosted.org/packages/06/ff/7841249c247aa650a76b9ee4bbaeae59370dc8bfd2f6c01f3630c35eb134/markdown_it_py-4.2.0.tar.gz", hash = "sha256:04a21681d6fbb623de53f6f364d352309d4094dd4194040a10fd51833e418d49", size = 82454, upload-time = "2026-05-07T12:08:28.36Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/94/54/e7d793b573f298e1c9013b8c4dade17d481164aa517d1d7148619c2cedbf/markdown_it_py-4.0.0-py3-none-any.whl", hash = "sha256:87327c59b172c5011896038353a81343b6754500a08cd7a4973bb48c6d578147", size = 87321, upload-time = "2025-08-11T12:57:51.923Z" }, + { url = "https://files.pythonhosted.org/packages/b3/81/4da04ced5a082363ecfa159c010d200ecbd959ae410c10c0264a38cac0f5/markdown_it_py-4.2.0-py3-none-any.whl", hash = "sha256:9f7ebbcd14fe59494226453aed97c1070d83f8d24b6fc3a3bcf9a38092641c4a", size = 91687, upload-time = "2026-05-07T12:08:27.182Z" }, ] [[package]] @@ -572,14 +572,14 @@ wheels = [ [[package]] name = "mdit-py-plugins" -version = "0.5.0" +version = "0.6.1" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "markdown-it-py" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/b2/fd/a756d36c0bfba5f6e39a1cdbdbfdd448dc02692467d83816dff4592a1ebc/mdit_py_plugins-0.5.0.tar.gz", hash = "sha256:f4918cb50119f50446560513a8e311d574ff6aaed72606ddae6d35716fe809c6", size = 44655, upload-time = "2025-08-11T07:25:49.083Z" } +sdist = { url = "https://files.pythonhosted.org/packages/59/fc/f8d0863f8862f25602c0404d75568e89fb6b4109804645e5cdfb1be5cf56/mdit_py_plugins-0.6.1.tar.gz", hash = "sha256:a2bca0f039f39dbd35fb74ae1b5f998608c437463371f0ff7f49a19a17a114d0", size = 56114, upload-time = "2026-05-13T09:03:38.91Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/fb/86/dd6e5db36df29e76c7a7699123569a4a18c1623ce68d826ed96c62643cae/mdit_py_plugins-0.5.0-py3-none-any.whl", hash = "sha256:07a08422fc1936a5d26d146759e9155ea466e842f5ab2f7d2266dd084c8dab1f", size = 57205, upload-time = "2025-08-11T07:25:47.597Z" }, + { url = "https://files.pythonhosted.org/packages/a5/69/6da5581c6a7fede7dc261bf4e67d6adca4196f176b43288b55b3db395b6e/mdit_py_plugins-0.6.1-py3-none-any.whl", hash = "sha256:214c82fb2ac524472ab6a5bcab1de80f73b50443e187f401bfd77efbc7c6481d", size = 66663, upload-time = "2026-05-13T09:03:37.76Z" }, ] [[package]] @@ -593,7 +593,7 @@ wheels = [ [[package]] name = "myst-parser" -version = "5.0.0" +version = "5.1.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "docutils" }, @@ -604,9 +604,9 @@ dependencies = [ { name = "sphinx", version = "9.0.4", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version < '3.12'" }, { name = "sphinx", version = "9.1.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.12'" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/33/fa/7b45eef11b7971f0beb29d27b7bfe0d747d063aa29e170d9edd004733c8a/myst_parser-5.0.0.tar.gz", hash = "sha256:f6f231452c56e8baa662cc352c548158f6a16fcbd6e3800fc594978002b94f3a", size = 98535, upload-time = "2026-01-15T09:08:18.036Z" } +sdist = { url = "https://files.pythonhosted.org/packages/21/dc/603751677fff302f34396e206b610f556a59d7fe58b9a2145f54e96b48e8/myst_parser-5.1.0.tar.gz", hash = "sha256:ab69322dc6719dcc7f296479dbb70181b66df6ed315064f92dbc85c0e1bf2f02", size = 101182, upload-time = "2026-05-13T09:38:19.361Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/d3/ac/686789b9145413f1a61878c407210e41bfdb097976864e0913078b24098c/myst_parser-5.0.0-py3-none-any.whl", hash = "sha256:ab31e516024918296e169139072b81592336f2fef55b8986aa31c9f04b5f7211", size = 84533, upload-time = "2026-01-15T09:08:16.788Z" }, + { url = "https://files.pythonhosted.org/packages/09/dc/f3dfb7488b770f3f67e6545085bf2abea5172e88f57b8ad25ef860ca704c/myst_parser-5.1.0-py3-none-any.whl", hash = "sha256:9c91c52b3cdb4d94a6506e4fab4e2f296c7623a0da0dcbe6de1565c3dad67a8a", size = 85817, upload-time = "2026-05-13T09:38:17.904Z" }, ] [[package]] @@ -758,7 +758,7 @@ wheels = [ [[package]] name = "pydata-sphinx-theme" -version = "0.17.1" +version = "0.18.0" source = { registry = "https://pypi.org/simple" } dependencies = [ { name = "accessible-pygments" }, @@ -770,9 +770,9 @@ dependencies = [ { name = "sphinx", version = "9.1.0", source = { registry = "https://pypi.org/simple" }, marker = "python_full_version >= '3.12'" }, { name = "typing-extensions" }, ] -sdist = { url = "https://files.pythonhosted.org/packages/ce/f7/c74c7100a7f4c0f77b5dcacb7dfdb8fee774fb70e487dd97acba2b930774/pydata_sphinx_theme-0.17.1.tar.gz", hash = "sha256:2cfc1d926c753c77039b7ee53f0ccebcbee5e81f0db61432b01cbb10ad7fd0af", size = 4991415, upload-time = "2026-04-21T13:00:34.263Z" } +sdist = { url = "https://files.pythonhosted.org/packages/ad/81/b3fdc8b74d0cfed9e623a0fef9932376800da5daa1a85d1224cac4c131a3/pydata_sphinx_theme-0.18.0.tar.gz", hash = "sha256:b4abc95ab02600872e060db07c79e056e87b7ea653ab1ffd0e0b1fa75a3003d4", size = 5004260, upload-time = "2026-05-20T08:32:28.897Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/e2/bc/2cb8c78300ce1ace4eeac3b3522218cea2c2053bfa6b4e32cc972a477f9a/pydata_sphinx_theme-0.17.1-py3-none-any.whl", hash = "sha256:320b022d7808bdf5920d9a28e573f27aace9b23e1af6ca103eecc752411df492", size = 6823346, upload-time = "2026-04-21T13:00:31.978Z" }, + { url = "https://files.pythonhosted.org/packages/a9/cd/e0eda602060f9dc99068f8e54490812d9d34ebb134043ff0ae594cf721a4/pydata_sphinx_theme-0.18.0-py3-none-any.whl", hash = "sha256:fbe5401f26642d487e3c5b6dfcbf69b3b1d579e80dcc479a429632abe0a13929", size = 6200747, upload-time = "2026-05-20T08:32:26.646Z" }, ] [[package]] @@ -802,11 +802,11 @@ wheels = [ [[package]] name = "pyjwt" -version = "2.12.0" +version = "2.13.0" source = { registry = "https://pypi.org/simple" } -sdist = { url = "https://files.pythonhosted.org/packages/a8/10/e8192be5f38f3e8e7e046716de4cae33d56fd5ae08927a823bb916be36c1/pyjwt-2.12.0.tar.gz", hash = "sha256:2f62390b667cd8257de560b850bb5a883102a388829274147f1d724453f8fb02", size = 102511, upload-time = "2026-03-12T17:15:30.831Z" } +sdist = { url = "https://files.pythonhosted.org/packages/3b/81/58d0ac84e1ef3a3843791d6954d94c0b33d526c75eeb1efbce9d0a4c4077/pyjwt-2.13.0.tar.gz", hash = "sha256:41571c89ca91598c79e8ef18a2d07367d4810fbbd6f637794879baf1b7703423", size = 107515, upload-time = "2026-05-21T19:54:36.618Z" } wheels = [ - { url = "https://files.pythonhosted.org/packages/15/70/70f895f404d363d291dcf62c12c85fdd47619ad9674ac0f53364d035925a/pyjwt-2.12.0-py3-none-any.whl", hash = "sha256:9bb459d1bdd0387967d287f5656bf7ec2b9a26645d1961628cda1764e087fd6e", size = 29700, upload-time = "2026-03-12T17:15:29.257Z" }, + { url = "https://files.pythonhosted.org/packages/a3/5e/ecf12fdb62546d64385c158514e9b2b671f7832108ef2ecd2020ce0af2d1/pyjwt-2.13.0-py3-none-any.whl", hash = "sha256:66adcc2aff09b3f1bbd95fc1e1577df8ac8723c978552fd43304c8a290ac5728", size = 31274, upload-time = "2026-05-21T19:54:35.362Z" }, ] [package.optional-dependencies]