diff --git a/contrib/delta/dev/diffs/4.1.0.diff b/contrib/delta/dev/diffs/4.1.0.diff new file mode 100644 index 0000000000..692dd5bd3f --- /dev/null +++ b/contrib/delta/dev/diffs/4.1.0.diff @@ -0,0 +1,232 @@ +diff --git a/build.sbt b/build.sbt +index 6f16864..e623a08 100644 +--- a/build.sbt ++++ b/build.sbt +@@ -50,6 +50,9 @@ val internalModuleNames = settingKey[Set[String]]("Internal module artifact name + + // Spark version to delta-spark and its dependent modules + // For more information see CrossSparkVersions.scala ++// Comet regression testing (added by dev/diffs/delta/4.1.0.diff in Apache DataFusion Comet) ++val cometVersion = "0.17.0-SNAPSHOT" ++ThisBuild / resolvers += Resolver.mavenLocal + val sparkVersion = settingKey[String]("Spark version") + + // Dependent library versions +@@ -617,6 +620,19 @@ lazy val spark = (project in file("spark-unified")) + "org.apache.spark" %% "spark-sql" % sparkVersion.value % "test" classifier "tests", + "org.apache.spark" %% "spark-hive" % sparkVersion.value % "test" classifier "tests", + "org.mockito" % "mockito-inline" % "4.11.0" % "test", ++ // Comet regression testing — pulls comet-spark for Spark 4.1 + Scala 2.13 from ++ // the user's mavenLocal repo. The published artifact is what `mvn install ++ // -Pspark-4.1 -Pcontrib-delta` produces; the contrib's Delta wiring is bundled ++ // into that JAR. No separate Delta-specific Comet artifact. ++ // ++ // `exclude(comet-contrib-delta-deps)`: the published comet-spark pom lists this ++ // as a transitive (advertising `delta-spark` provided-scope to end users), but ++ // its own pom has un-interpolated `${spark.version.short}` / `${scala.binary.version}` ++ // in `` and ``. Maven re-interpolates from filename, SBT ++ // doesn't — so SBT fails resolving the parent. Delta's own build already ++ // supplies delta-spark on the test classpath, so the exclude is safe here. ++ ("org.apache.datafusion" % s"comet-spark-spark4.1_${scalaBinaryVersion.value}" % cometVersion % "test") ++ .exclude("org.apache.datafusion", s"comet-contrib-delta-deps-spark4.1_${scalaBinaryVersion.value}"), + ), + + Test / testOptions += Tests.Argument("-oDF"), +@@ -636,7 +652,15 @@ lazy val spark = (project in file("spark-unified")) + "-Ddelta.log.cacheSize=3", + "-Dspark.databricks.delta.delta.log.cacheSize=3", + "-Dspark.sql.sources.parallelPartitionDiscovery.parallelism=5", +- "-Xmx1024m" ++ // Bumped from 1024m: 1GB is too tight for Spark + Delta + parquet I/O + Comet ++ // native library + offheap. Caused GC thrashing on memory-heavy suites (DV, ++ // Merge, CDC) under the 3.3.2 regression sweep — keeping 4g for 4.1.0. ++ "-Xmx4g", ++ // Match stock Delta CI: run on UTC so Delta's force-verify-all-files-in-CRC ++ // path (triggered by non-UTC user.timezone) stays disabled. Otherwise ++ // ChecksumSuite tests fail because `TimeZone.setDefault(UTC)` doesn't ++ // update `System.getProperty("user.timezone")`. ++ "-Duser.timezone=UTC" + ), + + // Required for testing table features see https://github.com/delta-io/delta/issues/1602 +diff --git a/build/sbt-config/repositories b/build/sbt-config/repositories +index 2853417..62ff4b6 100644 +--- a/build/sbt-config/repositories ++++ b/build/sbt-config/repositories +@@ -13,3 +13,12 @@ + typesafe-releases: https://repo.typesafe.com/typesafe/releases/ + apache-snapshot: https://repository.apache.org/content/groups/snapshots/ + jitpack: https://jitpack.io ++ # Comet regression testing: dev/run-delta-regression.sh publishes Comet's just-built ++ # artifacts to an ISOLATED dir under $TMPDIR rather than ~/.m2/repository/. Pointing ++ # SBT directly at ~/.m2 triggers a coursier sticky-resolver bug: when an unrelated ++ # transitive (parquet/guava/azure/...) has an orphan pom-only entry in ~/.m2 from a ++ # prior `mvn` run, coursier resolves the POM at local-m2, then refuses to fall ++ # through to maven-central for the JAR — failing the build on artifacts that have ++ # nothing to do with Comet. The isolated dir contains only `org.apache.datafusion:*` ++ # so there are no unrelated POMs to mistakenly match. ++ local-comet: file:///tmp/comet-published-4.1/ +diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/CometSmokeTest.scala b/spark/src/test/scala/org/apache/spark/sql/delta/CometSmokeTest.scala +new file mode 100644 +index 0000000..013ee98 +--- /dev/null ++++ b/spark/src/test/scala/org/apache/spark/sql/delta/CometSmokeTest.scala +@@ -0,0 +1,82 @@ ++/* ++ * Copyright (2021) The Delta Lake Project Authors. ++ * ++ * Licensed under the Apache License, Version 2.0 (the "License"); ++ * you may not use this file except in compliance with the License. ++ * You may obtain a copy of the License at ++ * ++ * http://www.apache.org/licenses/LICENSE-2.0 ++ * ++ * Unless required by applicable law or agreed to in writing, software ++ * distributed under the License is distributed on an "AS IS" BASIS, ++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. ++ * See the License for the specific language governing permissions and ++ * limitations under the License. ++ */ ++ ++package org.apache.spark.sql.delta ++ ++import org.apache.spark.sql.{QueryTest, Row} ++import org.apache.spark.sql.delta.test.DeltaSQLCommandTest ++import org.apache.spark.sql.test.SharedSparkSession ++ ++/** ++ * Smoke test asserting that Comet is actually loaded and executing queries when the ++ * Delta regression diff is applied. Catches silent configuration drift where Comet ++ * is on the classpath but not wired into the physical plan (e.g. a typo in ++ * `spark.plugins` that Spark silently ignores). ++ * ++ * Added by the Comet regression diff at `dev/diffs/delta/4.1.0.diff` in ++ * Apache DataFusion Comet. ++ */ ++class CometSmokeTest extends QueryTest with SharedSparkSession with DeltaSQLCommandTest { ++ ++ test("Comet plugin is registered in SparkConf") { ++ val plugins = spark.conf.get("spark.plugins") ++ assert(plugins.contains("CometPlugin"), ++ s"Comet plugin not registered. spark.plugins=$plugins") ++ } ++ ++ test("Delta streaming source read returns the micro-batch AddFiles") { ++ // Regression guard for the pre-materialized FileIndex path: Delta's streaming ++ // micro-batch gives us an exact `TahoeBatchFileIndex` with the AddFiles for ++ // [startOffset, endOffset]. The contrib's native Delta scan must honour that ++ // list instead of re-running kernel log replay against the snapshot root ++ // (which would return an empty or different set for a streaming batch). ++ val src = "comet_stream_src" ++ try { ++ spark.sql(s"CREATE TABLE $src (key INT, value INT) USING DELTA") ++ var collected: Seq[org.apache.spark.sql.Row] = Seq.empty ++ val sw = spark.readStream.table(src).writeStream ++ .format("console") ++ .foreachBatch { (df: org.apache.spark.sql.DataFrame, _: Long) => ++ collected = df.collect().toSeq ++ } ++ .outputMode("append") ++ .start() ++ spark.sql(s"INSERT INTO $src(key, value) VALUES(0, 42)") ++ sw.processAllAvailable() ++ sw.stop() ++ assert(collected.nonEmpty, "Streaming Delta source produced no rows") ++ assert(collected.map(r => (r.getInt(0), r.getInt(1))) == Seq((0, 42))) ++ } finally { ++ spark.sql(s"DROP TABLE IF EXISTS $src") ++ } ++ } ++ ++ test("Comet operators appear in Delta query physical plan") { ++ withTempDir { dir => ++ val path = dir.getCanonicalPath ++ spark.range(10).toDF("id") ++ .write.format("delta").save(path) ++ ++ val df = spark.read.format("delta").load(path).filter("id > 2") ++ val planString = df.queryExecution.executedPlan.toString ++ ++ assert(planString.contains("Comet"), ++ s"No Comet operators found in physical plan. Is Comet actually wired in?\n$planString") ++ ++ checkAnswer(df, (3 until 10).map(i => Row(i))) ++ } ++ } ++} +diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLCommandTest.scala b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLCommandTest.scala +index cb28a4f..44a2a37 100644 +--- a/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLCommandTest.scala ++++ b/spark/src/test/scala/org/apache/spark/sql/delta/test/DeltaSQLCommandTest.scala +@@ -35,5 +35,22 @@ trait DeltaSQLCommandTest extends SharedSparkSession { + classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + classOf[DeltaCatalog].getName) ++ // --- Comet regression wiring (added by dev/diffs/delta/4.1.0.diff) --- ++ // CometPlugin registers the native lib + memory manager. ServiceLoader pulls ++ // the Delta contrib's DeltaScanRuleExtension / DeltaOperatorSerdeExtension out ++ // of META-INF/services in comet-spark.jar at first use. ++ .set("spark.plugins", "org.apache.spark.CometPlugin") ++ .set("spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .set("spark.comet.enabled", "true") ++ .set("spark.comet.exec.enabled", "true") ++ .set("spark.comet.exec.shuffle.enabled", "true") ++ .set("spark.comet.scan.enabled", "true") ++ // Enable the contrib's native Delta scan path. Key is defined in ++ // org.apache.comet.contrib.delta.DeltaConf. ++ .set("spark.comet.scan.deltaNative.enabled", "true") ++ .set("spark.comet.explainFallback.enabled", "true") ++ .set("spark.memory.offHeap.enabled", "true") ++ .set("spark.memory.offHeap.size", "10g") + } + } +diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/test/ScanReportHelper.scala b/spark/src/test/scala/org/apache/spark/sql/delta/test/ScanReportHelper.scala +index e7c4b90..32246a2 100644 +--- a/spark/src/test/scala/org/apache/spark/sql/delta/test/ScanReportHelper.scala ++++ b/spark/src/test/scala/org/apache/spark/sql/delta/test/ScanReportHelper.scala +@@ -44,6 +44,11 @@ trait ScanReportHelper extends SharedSparkSession with AdaptiveSparkPlanHelper { + collectWithSubqueries(plan)({ + case fs: FileSourceScanExec => Seq(fs) + case cached: InMemoryTableScanExec => collectScans(cached.relation.cacheBuilder.cachedPlan) ++ // Comet regression: Comet rewrites `FileSourceScanExec` into its own leaf ++ // variants. Tests that inspect scan metrics would otherwise see an empty ++ // list and fail with `MatchError: ArrayBuffer()`. ++ case c: org.apache.spark.sql.comet.CometScanExec => Seq(c.wrapped) ++ case c: org.apache.spark.sql.comet.CometDeltaNativeScanExec => Seq(c.originalPlan) + }).flatten + } + +diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/test/TestsStatistics.scala b/spark/src/test/scala/org/apache/spark/sql/delta/test/TestsStatistics.scala +index 93c7da9..755fae9 100644 +--- a/spark/src/test/scala/org/apache/spark/sql/delta/test/TestsStatistics.scala ++++ b/spark/src/test/scala/org/apache/spark/sql/delta/test/TestsStatistics.scala +@@ -79,6 +79,13 @@ trait TestsStatistics { self: DeltaSQLTestUtils => + case f: FileSourceScanExec => Some(f) + case InputAdapter(f: FileSourceScanExec) => Some(f) + case ColumnarToRowExec(InputAdapter(f: FileSourceScanExec)) => Some(f) ++ // Comet regression: unwrap the Comet scan variants Comet's rules produce in ++ // place of Spark's `FileSourceScanExec`. Tests that search for the scan in ++ // the executed plan (e.g. to read its metrics) are otherwise blind to them. ++ case c: org.apache.spark.sql.comet.CometScanExec => Some(c.wrapped) ++ case c: org.apache.spark.sql.comet.CometDeltaNativeScanExec => Some(c.originalPlan) ++ case org.apache.spark.sql.comet.CometNativeColumnarToRowExec(inner) => ++ unapply(inner) + case _ => None + } + } +diff --git a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +--- a/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala ++++ b/spark/src/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +@@ -118,6 +118,11 @@ + val query = spark.read.format("delta").load(testPath).where("part = 1") + val fileScans = query.queryExecution.executedPlan.collect { + case f: FileSourceScanExec => f ++ // Comet regression: Comet's planner replaces `FileSourceScanExec` with ++ // `CometDeltaNativeScanExec`. Its `metrics` map aliases `numFiles` to the ++ // post-pruning task count so this assertion exercises the same partition-skip ++ // semantics from the Comet path. ++ case c: org.apache.spark.sql.comet.CometDeltaNativeScanExec => c + } + + // Force the query to read files and generate metrics diff --git a/contrib/delta/dev/run-regression.sh b/contrib/delta/dev/run-regression.sh new file mode 100755 index 0000000000..c064c2f786 --- /dev/null +++ b/contrib/delta/dev/run-regression.sh @@ -0,0 +1,206 @@ +#!/usr/bin/env bash +# 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. + +# Run Delta Lake's own test suite with Comet enabled as a regression check. +# Mirrors what .github/workflows/delta_regression_test.yml does in CI. +# +# This is the PR2 (contrib) variant: the install step bundles the Delta +# contrib via `-Pcontrib-delta` so the comet-spark JAR being installed +# carries DeltaScanRuleExtension/DeltaOperatorSerdeExtension and the +# matching JNI symbols (built into libcomet via `--features contrib-delta` +# on the native crate). Without `-Pcontrib-delta` the installed comet-spark +# JAR has no Delta wiring and Delta tests would just exercise vanilla Spark. +# +# Usage: +# dev/run-delta-regression.sh [DELTA_VERSION] [TEST_FILTER] +# +# Examples: +# dev/run-delta-regression.sh # smoke on default (4.1.0) +# dev/run-delta-regression.sh 4.1.0 # smoke on Delta 4.1.0 +# dev/run-delta-regression.sh 4.1.0 full # full Delta test suite +# dev/run-delta-regression.sh 4.1.0 DeltaTimeTravelSuite # one specific test class +# DELTA_WORKDIR=/tmp/my-delta dev/run-delta-regression.sh # reuse a checkout + +set -euo pipefail + +DELTA_VERSION="${1:-4.1.0}" +TEST_FILTER="${2:-smoke}" + +# Map Delta version -> Spark short version -> SBT module +case "$DELTA_VERSION" in + 2.4.0) SPARK_SHORT="3.4"; SBT_MODULE="core" ;; + 3.3.2) SPARK_SHORT="3.5"; SBT_MODULE="spark" ;; + 4.0.0) SPARK_SHORT="4.0"; SBT_MODULE="spark" ;; + 4.1.0) SPARK_SHORT="4.1"; SBT_MODULE="spark" ;; + *) + echo "Error: unsupported Delta version '$DELTA_VERSION'" + echo "Supported: 2.4.0 (Spark 3.4), 3.3.2 (Spark 3.5), 4.0.0 (Spark 4.0), 4.1.0 (Spark 4.1)" + exit 1 + ;; +esac + +# Script lives at contrib/delta/dev/run-regression.sh, so COMET_ROOT is three levels up. +COMET_ROOT="$(cd "$(dirname "${BASH_SOURCE[0]}")/../../.." && pwd)" +DIFF_FILE="$COMET_ROOT/contrib/delta/dev/diffs/${DELTA_VERSION}.diff" +DELTA_WORKDIR="${DELTA_WORKDIR:-${TMPDIR:-/tmp}/delta-regression-${DELTA_VERSION}}" + +if [[ ! -f "$DIFF_FILE" ]]; then + echo "Error: diff file not found: $DIFF_FILE" + exit 1 +fi + +echo "==========================================" +echo "Delta regression run (contrib variant)" +echo " Delta version : $DELTA_VERSION" +echo " Spark profile : spark-$SPARK_SHORT" +echo " SBT module : $SBT_MODULE" +echo " Test filter : $TEST_FILTER" +echo " Work dir : $DELTA_WORKDIR" +echo " Comet root : $COMET_ROOT" +echo "==========================================" + +# Step 1: build + install Comet to local Maven repo for the target Spark profile, +# with the Delta contrib bundled into comet-spark.jar. +# +# `FAST=1` skips plugin checks that aren't relevant during iteration: +# - drop `-Prelease` (no source/javadoc/scaladoc jars, no GPG prep) +# - skip spotless check (run `mvn spotless:apply` manually before commit) +# - skip Apache RAT license header check +# - skip javadoc / scaladoc generation +# - skip source jar packaging +# Together these save ~60-120s per iteration. The canonical (no-FAST) invocation +# still runs the full lifecycle so CI parity is preserved. +echo +echo "[1/4] Building and installing Comet (spark-$SPARK_SHORT, contrib-delta)..." +cd "$COMET_ROOT" +# Spark 4.1 requires Java 17 (java.lang.Record). Comet's parent pom defaults +# java.version=11 — overriding here so the install works regardless of which JDK +# is on JAVA_HOME, as long as that JDK is ≥17. +JAVA_OVERRIDE=( + -Djava.version=17 + -Dmaven.compiler.source=17 + -Dmaven.compiler.target=17 +) +if [[ -n "${FAST:-}" ]]; then + echo " FAST=1: skipping spotless/RAT/javadoc/source-jar plugins" + # Override `jni.dir` -> `native/target/release` because Comet's parent pom defaults it + # to `native/target/debug`. The non-FAST path implicitly fixes this via `-Prelease`, + # but FAST=1 drops that profile (it pulls in shade/javadoc), so without this override + # mvn bundles a stale debug-tree dylib and contrib-delta's `#[ctor]` planner registration + # silently goes missing -- every Delta scan then fails with "No contrib planner + # registered for ContribOp.kind=delta-scan" at runtime. + ./mvnw install -DskipTests -Pspark-"$SPARK_SHORT" -Pcontrib-delta \ + "${JAVA_OVERRIDE[@]}" \ + -Djni.dir="$COMET_ROOT/native/target/release" \ + -Dspotless.check.skip=true \ + -Drat.skip=true \ + -Dmaven.javadoc.skip=true \ + -Dmaven.source.skip=true +else + ./mvnw install -Prelease -DskipTests -Pspark-"$SPARK_SHORT" -Pcontrib-delta \ + "${JAVA_OVERRIDE[@]}" +fi + +# Sync Comet's just-installed artifacts to an ISOLATED publish dir. Pointing SBT +# directly at ~/.m2/repository/ triggers coursier's sticky-resolver: orphan +# pom-only entries left over from `mvn dependency:resolve` runs make it look for +# unrelated transitive JARs (parquet, guava, azure, ...) at local-m2 and refuse +# to fall through to maven-central. Isolating Comet's artifacts in a dedicated +# directory means local-comet only matches `org.apache.datafusion:*` -- no +# orphans to mistake. +# +# Hard-coded under /tmp (not $TMPDIR) because the path is also referenced in +# dev/diffs/delta/.diff (build/sbt-config/repositories), which +# the diff applies into the Delta checkout. macOS's $TMPDIR is per-user under +# /var/folders/..., so substituting it here would diverge from the diff's +# literal path. +COMET_PUBLISH_DIR="${COMET_PUBLISH_DIR:-/tmp/comet-published-${SPARK_SHORT}}" +echo +echo "[1.5/4] Syncing Comet artifacts to $COMET_PUBLISH_DIR..." +rm -rf "$COMET_PUBLISH_DIR" +mkdir -p "$COMET_PUBLISH_DIR/org/apache/datafusion" +rsync -a "$HOME/.m2/repository/org/apache/datafusion/" "$COMET_PUBLISH_DIR/org/apache/datafusion/" +echo " Published: $(ls -1 "$COMET_PUBLISH_DIR/org/apache/datafusion/" | wc -l | tr -d ' ') Comet modules" + +# Step 2: clone Delta (or reuse existing checkout). +# +# `git clean -fd` here is intentional and cheap (sub-second): it removes +# untracked files left from the previous diff apply but respects gitignore, +# so Delta's `target/` (and SBT's zinc cache inside it) is preserved. +echo +echo "[2/4] Cloning Delta $DELTA_VERSION..." +if [[ -d "$DELTA_WORKDIR/.git" ]]; then + echo " Reusing existing checkout at $DELTA_WORKDIR" + cd "$DELTA_WORKDIR" + git fetch --depth 1 origin "refs/tags/v$DELTA_VERSION:refs/tags/v$DELTA_VERSION" 2>/dev/null || true + git checkout -f "v$DELTA_VERSION" + git clean -fd + rm -rf spark/spark-warehouse +else + rm -rf "$DELTA_WORKDIR" + git clone --depth 1 --branch "v$DELTA_VERSION" https://github.com/delta-io/delta.git "$DELTA_WORKDIR" + cd "$DELTA_WORKDIR" +fi + +# Step 3: apply the Comet diff. +echo +echo "[3/4] Applying diff $DIFF_FILE..." +git apply "$DIFF_FILE" + +# Step 4: run tests. +echo +echo "[4/4] Running tests..." +export SPARK_LOCAL_IP="${SPARK_LOCAL_IP:-localhost}" +# Skip Delta's javaunidoc generation. Delta's `configureUnidoc` wires +# `(Test / test) := (Test / test) dependsOn (Compile / unidoc)`, and the +# javaunidoc step compiles auto-generated Java stubs from Scala test sources +# that fail to resolve `org.apache.spark.sql.test.SQLTestData` etc. -- Delta's +# own gap, not ours. Setting DISABLE_UNIDOC=1 short-circuits the helper +# (Unidoc.scala line 52) so the test target runs directly. +export DISABLE_UNIDOC=1 + +# Delta 4.1.0 mandates Java 17; Comet itself builds fine on 17+. If the user +# is iterating with a newer JDK on Comet, point this at a JDK 17 install for +# SBT. Typical usage: `DELTA_JAVA_HOME=$(/usr/libexec/java_home -v 17)`. +if [[ -n "${DELTA_JAVA_HOME:-}" ]]; then + echo " Using DELTA_JAVA_HOME=$DELTA_JAVA_HOME for SBT" + export JAVA_HOME="$DELTA_JAVA_HOME" + export PATH="$DELTA_JAVA_HOME/bin:$PATH" +fi + +# Reset Gradle daemon + script cache. A daemon started with an older JDK +# sticks around and will be reused by Delta's `./gradlew` inside +# `icebergShaded/assembly`, and Gradle's compiled-build-script cache stores +# classfiles whose major version matches the JDK of the earlier run. +pkill -f 'GradleDaemon' 2>/dev/null || true +rm -rf ~/.gradle/caches/7.5.1/scripts ~/.gradle/caches/7.6.3/scripts 2>/dev/null || true + +case "$TEST_FILTER" in + smoke) + build/sbt "$SBT_MODULE/testOnly org.apache.spark.sql.delta.CometSmokeTest" + ;; + full) + build/sbt "$SBT_MODULE/test" + ;; + *) + build/sbt "$SBT_MODULE/testOnly $TEST_FILTER" + ;; +esac + +echo +echo "Done." diff --git a/contrib/delta/dev/run-test.sh b/contrib/delta/dev/run-test.sh new file mode 100755 index 0000000000..9b8c67731d --- /dev/null +++ b/contrib/delta/dev/run-test.sh @@ -0,0 +1,51 @@ +#!/bin/bash +# +# 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. +# +# Run one or more Delta scalatest test selectors via build/sbt in the extracted +# Delta regression checkout (Delta 4.1.0 / Spark 4.1 by default; override with +# DELTA_VERSION). +# +# Usage: dev/run-delta-test.sh 'org.apache.spark.sql.delta.SomeSuite -- -z "test substring"' [...more testOnly selectors] +# +# Each argument is passed as a separate `spark/testOnly` command. Output goes to +# target/delta-regression-logs/test-.log (relative to this repo). +set -euo pipefail +REPO_ROOT="$(cd "$(dirname "$0")/../../.." && pwd)" +DELTA_VERSION="${DELTA_VERSION:-4.1.0}" +DELTA_DIR="${DELTA_DIR:-${DELTA_WORKDIR:-${TMPDIR:-/tmp}/delta-regression-${DELTA_VERSION}}}" +export JAVA_HOME="${JAVA_HOME:-$HOME/jdks/jdk-17.0.18+8/Contents/Home}" +export SPARK_LOCAL_IP=127.0.0.1 +export RUST_BACKTRACE=1 + +if [ $# -lt 1 ]; then + echo "usage: $0 'SuiteClass -- -z \"name\"' [...]" + exit 2 +fi + +LOG="$REPO_ROOT/target/delta-regression-logs/test-$(date +%Y%m%d-%H%M%S).log" +mkdir -p "$(dirname "$LOG")" + +cmds=() +for sel in "$@"; do + cmds+=("spark/testOnly $sel") +done + +cd "$DELTA_DIR" +echo "==> logging to $LOG" +build/sbt "${cmds[@]}" 2>&1 | tee "$LOG" diff --git a/contrib/delta/native/Cargo.lock b/contrib/delta/native/Cargo.lock new file mode 100644 index 0000000000..6544c9a9ce --- /dev/null +++ b/contrib/delta/native/Cargo.lock @@ -0,0 +1,4495 @@ +# 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 = "ahash" +version = "0.8.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" +dependencies = [ + "cfg-if", + "const-random", + "getrandom 0.3.4", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddd31a130427c27518df266943a5308ed92d4b226cc639f5a8f1002816174301" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[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 = "anyhow" +version = "1.0.102" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f202df86484c868dbad7eaa557ef785d5c66295e41b460ef922eca0723b842c" + +[[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 = "arrow" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3bd47f2a6ddc39244bd722a27ee5da66c03369d087b9e024eafdb03e98b98ea7" +dependencies = [ + "arrow-arith 57.3.1", + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-cast 57.3.1", + "arrow-csv 57.3.1", + "arrow-data 57.3.1", + "arrow-ipc 57.3.1", + "arrow-json 57.3.1", + "arrow-ord 57.3.1", + "arrow-row 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "arrow-string 57.3.1", +] + +[[package]] +name = "arrow" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "378530e55cd479eda3c14eb345310799717e6f76d0c332041e8487022166b471" +dependencies = [ + "arrow-arith 58.3.0", + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-cast 58.3.0", + "arrow-csv 58.3.0", + "arrow-data 58.3.0", + "arrow-ipc 58.3.0", + "arrow-json 58.3.0", + "arrow-ord 58.3.0", + "arrow-row 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", + "arrow-string 58.3.0", +] + +[[package]] +name = "arrow-arith" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c7bbd679c5418b8639b92be01f361d60013c4906574b578b77b63c78356594c" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "num-traits", +] + +[[package]] +name = "arrow-arith" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0ab212d2c1886e802f51c5212d78ebbcbb0bec980fff9dadc1eb8d45cd0b738" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "chrono", + "num-traits", +] + +[[package]] +name = "arrow-array" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8a4ab47b3f3eac60f7fd31b81e9028fda018607bcc63451aca4f2b755269862" +dependencies = [ + "ahash", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "chrono-tz", + "half", + "hashbrown 0.16.1", + "num-complex", + "num-integer", + "num-traits", +] + +[[package]] +name = "arrow-array" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfd33d3e92f207444098c75b42de99d329562be0cf686b307b097cc52b4e999e" +dependencies = [ + "ahash", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "chrono", + "chrono-tz", + "half", + "hashbrown 0.17.1", + "num-complex", + "num-integer", + "num-traits", +] + +[[package]] +name = "arrow-buffer" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d18b89b4c4f4811d0858175e79541fe98e33e18db3b011708bc287b1240593f" +dependencies = [ + "bytes", + "half", + "num-bigint", + "num-traits", +] + +[[package]] +name = "arrow-buffer" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c6cd424c2693bcdbc150d843dc9d4d137dd2de4782ce6df491ad11a3a0416c0" +dependencies = [ + "bytes", + "half", + "num-bigint", + "num-traits", +] + +[[package]] +name = "arrow-cast" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "722b5c41dd1d14d0a879a1bce92c6fe33f546101bb2acce57a209825edd075b3" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-ord 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "atoi", + "base64", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num-traits", + "ryu", +] + +[[package]] +name = "arrow-cast" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c5aefb56a2c02e9e2b30746241058b85f8983f0fcff2ba0c6d09006e1cded7f" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-ord 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", + "atoi", + "base64", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num-traits", + "ryu", +] + +[[package]] +name = "arrow-csv" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "27ddb80a4848e03b1655af496d5ac2563a779e5742fcb48f2ca2e089c9cd2197" +dependencies = [ + "arrow-array 57.3.1", + "arrow-cast 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "csv", + "csv-core", + "regex", +] + +[[package]] +name = "arrow-csv" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e94e8cf7e517657a52b91ea1263acf38c4ca62a84655d72458a3359b12ab97de" +dependencies = [ + "arrow-array 58.3.0", + "arrow-cast 58.3.0", + "arrow-schema 58.3.0", + "chrono", + "csv", + "csv-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1683705c63dcf0d18972759eda48489028cbbff67af7d6bef2c6b7b74ab778a" +dependencies = [ + "arrow-buffer 57.3.1", + "arrow-schema 57.3.1", + "half", + "num-integer", + "num-traits", +] + +[[package]] +name = "arrow-data" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c88210023a2bfee1896af366309a3028fc3bcbd6515fa29a7990ee1baa08ee0" +dependencies = [ + "arrow-buffer 58.3.0", + "arrow-schema 58.3.0", + "half", + "num-integer", + "num-traits", +] + +[[package]] +name = "arrow-ipc" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8cf72d04c07229fbf4dbebe7145cac37d7cf7ec582fe705c6b92cb314af096ab" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "flatbuffers", +] + +[[package]] +name = "arrow-ipc" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "238438f0834483703d88896db6fe5a7138b2230debc31b34c0336c2996e3c64f" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", + "flatbuffers", + "lz4_flex 0.13.1", +] + +[[package]] +name = "arrow-json" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a84a905f41fedfcd7679813c89a61dc369c0f932b27aa8dcc6aa051cc781a97d" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-cast 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "half", + "indexmap", + "itoa", + "lexical-core", + "memchr", + "num-traits", + "ryu", + "serde_core", + "serde_json", + "simdutf8", +] + +[[package]] +name = "arrow-json" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "205ca2119e6d679d5c133c6f30e68f027738d95ed948cf77677ea69c7800036b" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-cast 58.3.0", + "arrow-ord 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", + "chrono", + "half", + "indexmap", + "itoa", + "lexical-core", + "memchr", + "num-traits", + "ryu", + "serde_core", + "serde_json", + "simdutf8", +] + +[[package]] +name = "arrow-ord" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "082342947d4e5a2bcccf029a0a0397e21cb3bb8421edd9571d34fb5dd2670256" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", +] + +[[package]] +name = "arrow-ord" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bffd8fd2579286a5d63bac898159873e5094a79009940bcb42bbfce4f19f1d0" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", +] + +[[package]] +name = "arrow-row" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3a931b520a2a5e22033e01a6f2486b4cdc26f9106b759abeebc320f125e94d7" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "half", +] + +[[package]] +name = "arrow-row" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bab5994731204603c73ba69267616c50f80780774c6bb0476f1f830625115e0c" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "half", +] + +[[package]] +name = "arrow-schema" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4cf0d4a6609679e03002167a61074a21d7b1ad9ea65e462b2c0a97f8a3b2bc6" +dependencies = [ + "bitflags", +] + +[[package]] +name = "arrow-schema" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f633dbfdf39c039ada1bf9e34c694816eb71fbb7dc78f613993b7245e078a1ed" +dependencies = [ + "bitflags", + "serde_core", + "serde_json", +] + +[[package]] +name = "arrow-select" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b320d86a9806923663bb0fd9baa65ecaba81cb0cd77ff8c1768b9716b4ef891" +dependencies = [ + "ahash", + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "num-traits", +] + +[[package]] +name = "arrow-select" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8cd065c54172ac787cf3f2f8d4107e0d3fdc26edba76fdf4f4cc170258942222" +dependencies = [ + "ahash", + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "num-traits", +] + +[[package]] +name = "arrow-string" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b493e99162e5764077e7823e50ba284858d365922631c7aaefe9487b1abd02c2" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "memchr", + "num-traits", + "regex", + "regex-syntax", +] + +[[package]] +name = "arrow-string" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29dd7cda3ab9692f43a2e4acc444d760cc17b12bb6d8232ddf64e9bab7c06b42" +dependencies = [ + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", + "memchr", + "num-traits", + "regex", + "regex-syntax", +] + +[[package]] +name = "async-trait" +version = "0.1.89" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "atoi" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" +dependencies = [ + "num-traits", +] + +[[package]] +name = "atomic-waker" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" + +[[package]] +name = "autocfg" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" + +[[package]] +name = "base64" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" + +[[package]] +name = "bigdecimal" +version = "0.4.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d6867f1565b3aad85681f1015055b087fcfd840d6aeee6eee7f2da317603695" +dependencies = [ + "autocfg", + "libm", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "bitflags" +version = "2.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4512299f36f043ab09a583e57bceb5a5aab7a73db1805848e8fef3c9e8c78b3" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[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 = "brotli" +version = "8.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4bd8b9603c7aa97359dbd97ecf258968c95f3adddd6db2f7e7a5bef101c84560" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor", +] + +[[package]] +name = "brotli-decompressor" +version = "5.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "874bb8112abecc98cbd6d81ea4fa7e94fb9449648c93cc89aa40c81c24d7de03" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5d20789868f4b01b2f2caec9f5c4e0213b41e3e5702a50157d699ae31ced2fcb" + +[[package]] +name = "bytemuck" +version = "1.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8efb64bd706a16a1bdde310ae86b351e4d21550d98d056f22f8a7f7a2183fec" + +[[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 = "cc" +version = "1.2.62" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1dce859f0832a7d088c4f1119888ab94ef4b5d6795d1ce05afb7fe159d79f98" +dependencies = [ + "find-msvc-tools", + "jobserver", + "libc", + "shlex", +] + +[[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 = "chacha20" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f8d983286843e49675a4b7a2d174efe136dc93a18d69130dd18198a6c167601" +dependencies = [ + "cfg-if", + "cpufeatures 0.3.0", + "rand_core 0.10.1", +] + +[[package]] +name = "chrono" +version = "0.4.44" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c673075a2e0e5f4a1dde27ce9dee1ea4558c7ffe648f576438a20ca1d2acc4b0" +dependencies = [ + "iana-time-zone", + "js-sys", + "num-traits", + "serde", + "wasm-bindgen", + "windows-link", +] + +[[package]] +name = "chrono-tz" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6139a8597ed92cf816dfb33f5dd6cf0bb93a6adc938f11039f371bc5bcd26c3" +dependencies = [ + "chrono", + "phf", +] + +[[package]] +name = "combine" +version = "4.6.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "comet-contrib-delta" +version = "0.17.0" +dependencies = [ + "arrow 58.3.0", + "chrono", + "chrono-tz", + "datafusion", + "datafusion-comet-jni-bridge", + "datafusion-comet-proto", + "delta_kernel", + "futures", + "jni", + "log", + "object_store 0.12.5", + "object_store 0.13.2", + "parquet 58.3.0", + "prost", + "roaring 0.10.12", + "tempfile", + "thiserror", + "tokio", + "url", +] + +[[package]] +name = "comfy-table" +version = "7.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "958c5d6ecf1f214b4c2bbbbf6ab9523a864bd136dcf71a7e8904799acfe1ad47" +dependencies = [ + "crossterm", + "unicode-segmentation", + "unicode-width", +] + +[[package]] +name = "const-random" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom 0.2.17", + "once_cell", + "tiny-keccak", +] + +[[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 = "crc" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5eb8a2a1cd12ab0d987a5d5e825195d372001a4094a0376319d5a0ad71c1ba0d" +dependencies = [ + "crc-catalog", +] + +[[package]] +name = "crc-catalog" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "217698eaf96b4a3f0bc4f3662aaa55bdf913cd54d7204591faa790070c6d0853" + +[[package]] +name = "crossbeam-utils" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" + +[[package]] +name = "crossterm" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8b9f2e4c67f833b660cdb0a3523065869fb35570177239812ed4c905aeff87b" +dependencies = [ + "bitflags", + "crossterm_winapi", + "document-features", + "parking_lot", + "rustix", + "winapi", +] + +[[package]] +name = "crossterm_winapi" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acdd7c62a3665c7f6830a51635d9ac9b23ed385797f70a83bb8bafe9c572ab2b" +dependencies = [ + "winapi", +] + +[[package]] +name = "crunchy" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" + +[[package]] +name = "crypto-common" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78c8292055d1c1df0cce5d180393dc8cce0abec0a7102adb6c7b1eef6016d60a" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52cd9d68cf7efc6ddfaaee42e7288d3a99d613d4b50f76ce9827ae0c6e14f938" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde_core", +] + +[[package]] +name = "csv-core" +version = "0.1.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "704a3c26996a80471189265814dbc2c257598b96b8a7feae2d31ace646bb9782" +dependencies = [ + "memchr", +] + +[[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 = "datafusion" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93db0e623840612f7f2cd757f7e8a8922064192363732c88692e0870016e141b" +dependencies = [ + "arrow 58.3.0", + "arrow-schema 58.3.0", + "async-trait", + "bytes", + "chrono", + "datafusion-catalog", + "datafusion-catalog-listing", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-datasource-arrow", + "datafusion-datasource-csv", + "datafusion-datasource-json", + "datafusion-datasource-parquet", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-nested", + "datafusion-functions-table", + "datafusion-functions-window", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-expr-adapter", + "datafusion-physical-expr-common", + "datafusion-physical-optimizer", + "datafusion-physical-plan", + "datafusion-session", + "datafusion-sql", + "futures", + "itertools", + "log", + "object_store 0.13.2", + "parking_lot", + "parquet 58.3.0", + "rand 0.9.4", + "regex", + "tempfile", + "tokio", + "url", + "uuid", +] + +[[package]] +name = "datafusion-catalog" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37cefde60b26a7f4ff61e9d2ff2833322f91df2b568d7238afe67bde5bdffb66" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "dashmap", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "itertools", + "log", + "object_store 0.13.2", + "parking_lot", + "tokio", +] + +[[package]] +name = "datafusion-catalog-listing" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17e112307715d6a7a331111a4c2330ff54bc237183511c319e3708a4cff431fb" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-adapter", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "futures", + "itertools", + "log", + "object_store 0.13.2", +] + +[[package]] +name = "datafusion-comet-common" +version = "0.17.0" +dependencies = [ + "arrow 58.3.0", + "datafusion", + "serde", + "serde_json", + "thiserror", +] + +[[package]] +name = "datafusion-comet-jni-bridge" +version = "0.17.0" +dependencies = [ + "arrow 58.3.0", + "datafusion", + "datafusion-comet-common", + "jni", + "lazy_static", + "once_cell", + "parquet 58.3.0", + "paste", + "prost", + "regex", + "thiserror", +] + +[[package]] +name = "datafusion-comet-proto" +version = "0.17.0" +dependencies = [ + "prost", + "prost-build", +] + +[[package]] +name = "datafusion-common" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d72a11ca44a95e1081870d3abb80c717496e8a7acb467a1d3e932bb636af5cc2" +dependencies = [ + "ahash", + "arrow 58.3.0", + "arrow-ipc 58.3.0", + "chrono", + "half", + "hashbrown 0.16.1", + "indexmap", + "itertools", + "libc", + "log", + "object_store 0.13.2", + "parquet 58.3.0", + "paste", + "sqlparser", + "tokio", + "web-time", +] + +[[package]] +name = "datafusion-common-runtime" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89f4afaed29670ec4fd6053643adc749fe3f4bc9d1ce1b8c5679b22c67d12def" +dependencies = [ + "futures", + "log", + "tokio", +] + +[[package]] +name = "datafusion-datasource" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e9fb386e1691355355a96419978a0022b7947b44d4a24a6ea99f00b6b485cbb6" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "bytes", + "chrono", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-adapter", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "glob", + "itertools", + "log", + "object_store 0.13.2", + "rand 0.9.4", + "tokio", + "url", +] + +[[package]] +name = "datafusion-datasource-arrow" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ffa6c52cfed0734c5f93754d1c0175f558175248bf686c944fb05c373e5fc096" +dependencies = [ + "arrow 58.3.0", + "arrow-ipc 58.3.0", + "async-trait", + "bytes", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "itertools", + "object_store 0.13.2", + "tokio", +] + +[[package]] +name = "datafusion-datasource-csv" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "503f29e0582c1fc189578d665ff57d9300da1f80c282777d7eb67bb79fb8cdca" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "bytes", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "object_store 0.13.2", + "regex", + "tokio", +] + +[[package]] +name = "datafusion-datasource-json" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e33804749abc8d0c8cb7473228483cb8070e524c6f6086ee1b85a64debe2b3d2" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "bytes", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "object_store 0.13.2", + "serde_json", + "tokio", + "tokio-stream", +] + +[[package]] +name = "datafusion-datasource-parquet" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a8e0365e0e08e8ff94d912f0ababcf9065a1a304018ba90b1fc83c855b4997" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "bytes", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr", + "datafusion-physical-expr-adapter", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-pruning", + "datafusion-session", + "futures", + "itertools", + "log", + "object_store 0.13.2", + "parking_lot", + "parquet 58.3.0", + "tokio", +] + +[[package]] +name = "datafusion-doc" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de6ac0df1662b9148ad3c987978b32cbec7c772f199b1d53520c8fa764a87ee" + +[[package]] +name = "datafusion-execution" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c03c7fbdaefcca4ef6ffe425a5fc2325763bfb426599bb0bf4536466efabe709" +dependencies = [ + "arrow 58.3.0", + "arrow-buffer 58.3.0", + "async-trait", + "chrono", + "dashmap", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr-common", + "futures", + "log", + "object_store 0.13.2", + "parking_lot", + "rand 0.9.4", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "574b9b6977fedbd2a611cbff12e5caf90f31640ad9dc5870f152836d94bad0dd" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-doc", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr-common", + "indexmap", + "itertools", + "paste", + "serde_json", + "sqlparser", +] + +[[package]] +name = "datafusion-expr-common" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d7c3adf3db8bf61e92eb90cb659c8e8b734593a8f7c8e12a843c7ddba24b87e" +dependencies = [ + "arrow 58.3.0", + "datafusion-common", + "indexmap", + "itertools", + "paste", +] + +[[package]] +name = "datafusion-functions" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28aa4e10384e782774b10e72aca4d93ef7b31aa653095d9d4536b0a3dbc51b6" +dependencies = [ + "arrow 58.3.0", + "arrow-buffer 58.3.0", + "base64", + "blake2", + "blake3", + "chrono", + "chrono-tz", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", + "hex", + "itertools", + "log", + "md-5", + "memchr", + "num-traits", + "rand 0.9.4", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-functions-aggregate" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00aa6217e56098ba84e0a338176fe52f0a84cca398021512c6c8c5eff806d0ad" +dependencies = [ + "ahash", + "arrow 58.3.0", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "half", + "log", + "num-traits", + "paste", +] + +[[package]] +name = "datafusion-functions-aggregate-common" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b511250349407db7c43832ab2de63f5557b19a20dfd236b39ca2c04468b50d47" +dependencies = [ + "ahash", + "arrow 58.3.0", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", +] + +[[package]] +name = "datafusion-functions-nested" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef13a858e20d50f0a9bb5e96e7ac82b4e7597f247515bccca4fdd2992df0212a" +dependencies = [ + "arrow 58.3.0", + "arrow-ord 58.3.0", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr-common", + "hashbrown 0.16.1", + "itertools", + "itoa", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-table" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b40d3f5bbb3905f9ccb1ce9485a9595c77b69758a7c24d3ba79e334ff51e7e" +dependencies = [ + "arrow 58.3.0", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", + "paste", +] + +[[package]] +name = "datafusion-functions-window" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4e88ec9d57c9b685d02f58bfee7be62d72610430ddcedb82a08e5d9925dbfb6" +dependencies = [ + "arrow 58.3.0", + "datafusion-common", + "datafusion-doc", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-window-common" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8307bb93519b1a91913723a1130cfafeee3f72200d870d88e91a6fc5470ede5c" +dependencies = [ + "datafusion-common", + "datafusion-physical-expr-common", +] + +[[package]] +name = "datafusion-macros" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e367e6a71051d0ebdd29b2f85d12059b38b1d1f172c6906e80016da662226bd" +dependencies = [ + "datafusion-doc", + "quote", + "syn", +] + +[[package]] +name = "datafusion-optimizer" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e929015451a67f77d9d8b727b2bf3a40c4445fdef6cdc53281d7d97c76888ace" +dependencies = [ + "arrow 58.3.0", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "indexmap", + "itertools", + "log", + "regex", + "regex-syntax", +] + +[[package]] +name = "datafusion-physical-expr" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b1e68aba7a4b350401cfdf25a3d6f989ad898a7410164afe9ca52080244cb59" +dependencies = [ + "ahash", + "arrow 58.3.0", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", + "half", + "hashbrown 0.16.1", + "indexmap", + "itertools", + "parking_lot", + "paste", + "petgraph", + "tokio", +] + +[[package]] +name = "datafusion-physical-expr-adapter" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea22315f33cf2e0adc104e8ec42e285f6ed93998d565c65e82fec6a9ee9f9db4" +dependencies = [ + "arrow 58.3.0", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "itertools", +] + +[[package]] +name = "datafusion-physical-expr-common" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b04b45ea8ad3ac2d78f2ea2a76053e06591c9629c7a603eda16c10649ecf4362" +dependencies = [ + "ahash", + "arrow 58.3.0", + "chrono", + "datafusion-common", + "datafusion-expr-common", + "hashbrown 0.16.1", + "indexmap", + "itertools", + "parking_lot", +] + +[[package]] +name = "datafusion-physical-optimizer" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7cb13397809a425918f608dfe8653f332015a3e330004ab191b4404187238b95" +dependencies = [ + "arrow 58.3.0", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-pruning", + "itertools", +] + +[[package]] +name = "datafusion-physical-plan" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5edc023675791af9d5fb4cc4c24abf5f7bd3bd4dcf9e5bd90ea1eff6976dcc79" +dependencies = [ + "ahash", + "arrow 58.3.0", + "arrow-ord 58.3.0", + "arrow-schema 58.3.0", + "async-trait", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "futures", + "half", + "hashbrown 0.16.1", + "indexmap", + "itertools", + "log", + "num-traits", + "parking_lot", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "datafusion-pruning" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac8c76860e355616555081cab5968cec1af7a80701ff374510860bcd567e365a" +dependencies = [ + "arrow 58.3.0", + "datafusion-common", + "datafusion-datasource", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "itertools", + "log", +] + +[[package]] +name = "datafusion-session" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5412111aa48e2424ba926112e192f7a6b7e4ccb450145d25ce5ede9f19dc491e" +dependencies = [ + "async-trait", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", +] + +[[package]] +name = "datafusion-sql" +version = "53.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa0d133ddf8b9b3b872acac900157f783e7b879fe9a6bccf389abebbfac45ec1" +dependencies = [ + "arrow 58.3.0", + "bigdecimal", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-functions-nested", + "indexmap", + "log", + "regex", + "sqlparser", +] + +[[package]] +name = "delta_kernel" +version = "0.19.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06f7fc164b1557731fcc68a198e813811a000efade0f112d4f0a002e65042b83" +dependencies = [ + "arrow 57.3.1", + "bytes", + "chrono", + "comfy-table", + "crc", + "delta_kernel_derive", + "futures", + "indexmap", + "itertools", + "object_store 0.12.5", + "parquet 57.3.1", + "reqwest", + "roaring 0.11.4", + "rustc_version", + "serde", + "serde_json", + "strum", + "thiserror", + "tokio", + "tracing", + "url", + "uuid", + "z85", +] + +[[package]] +name = "delta_kernel_derive" +version = "0.19.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86815a2c475835751ffa9b8d9ac8ed86cf86294304c42bedd1103d54f25ecbfe" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "displaydoc" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "document-features" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4b8a88685455ed29a21542a33abd9cb6510b6b129abadabdcef0f4c55bc8f61" +dependencies = [ + "litrs", +] + +[[package]] +name = "either" +version = "1.15.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48c757948c5ede0e46177b7add2e67155f70e33c07fea8284df6576da70b3719" + +[[package]] +name = "equivalent" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" + +[[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 = "fastrand" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f1f227452a390804cdb637b74a86990f2a7d7ba4b7d5693aac9b4dd6defd8d6" + +[[package]] +name = "find-msvc-tools" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5baebc0774151f905a1a2cc41989300b1e6fbb29aff0ceffa1064fdd3088d582" + +[[package]] +name = "fixedbitset" +version = "0.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" + +[[package]] +name = "flatbuffers" +version = "25.12.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35f6839d7b3b98adde531effaf34f0c2badc6f4735d26fe74709d8e513a96ef3" +dependencies = [ + "bitflags", + "rustc_version", +] + +[[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 = "futures" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b147ee9d1f6d097cef9ce628cd2ee62288d963e16fb287bd9286455b241382d" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07bbe89c50d7a535e539b8c17bc0b49bdb77747034daa8087407d655f3f7cc1d" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e3450815272ef58cec6d564423f6e755e25379b217b0bc688e295ba24df6b1d" + +[[package]] +name = "futures-executor" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf29c38818342a3b26b5b923639e7b1f4a61fc5e76102d4b1981c6dc7a7579d" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cecba35d7ad927e23624b22ad55235f2239cfa44fd10428eecbeba6d6a717718" + +[[package]] +name = "futures-macro" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e835b70203e41293343137df5c0664546da5745f82ec9b84d40be8336958447b" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "futures-sink" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c39754e157331b013978ec91992bde1ac089843443c49cbc7f46150b0fad0893" + +[[package]] +name = "futures-task" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "037711b3d59c33004d3856fbdc83b99d4ff37a24768fa1be9ce3538a1cde4393" + +[[package]] +name = "futures-util" +version = "0.3.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "389ca41296e6190b48053de0321d02a77f32f8a5d2461dd38762c0593805c6d6" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff2abc00be7fca6ebc474524697ae276ad847ad0a6b3faa4bcb027e9a4614ad0" +dependencies = [ + "cfg-if", + "js-sys", + "libc", + "wasi", + "wasm-bindgen", +] + +[[package]] +name = "getrandom" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" +dependencies = [ + "cfg-if", + "js-sys", + "libc", + "r-efi 5.3.0", + "wasip2", + "wasm-bindgen", +] + +[[package]] +name = "getrandom" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0de51e6874e94e7bf76d726fc5d13ba782deca734ff60d5bb2fb2607c7406555" +dependencies = [ + "cfg-if", + "libc", + "r-efi 6.0.0", + "rand_core 0.10.1", + "wasip2", + "wasip3", +] + +[[package]] +name = "glob" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" + +[[package]] +name = "h2" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "171fefbc92fe4a4de27e0698d6a5b392d6a0e333506bc49133760b3bcf948733" +dependencies = [ + "atomic-waker", + "bytes", + "fnv", + "futures-core", + "futures-sink", + "http", + "indexmap", + "slab", + "tokio", + "tokio-util", + "tracing", +] + +[[package]] +name = "half" +version = "2.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" +dependencies = [ + "cfg-if", + "crunchy", + "num-traits", + "zerocopy", +] + +[[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 = [ + "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 = "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 = "http" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3ba2a386d7f85a81f119ad7498ebe444d2e22c2af0b86b069416ace48b3311a" +dependencies = [ + "bytes", + "itoa", +] + +[[package]] +name = "http-body" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1efedce1fb8e6913f23e0c92de8e62cd5b772a67e7b3946df930a62566c93184" +dependencies = [ + "bytes", + "http", +] + +[[package]] +name = "http-body-util" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b021d93e26becf5dc7e1b75b1bed1fd93124b374ceb73f43d4d4eafec896a64a" +dependencies = [ + "bytes", + "futures-core", + "http", + "http-body", + "pin-project-lite", +] + +[[package]] +name = "httparse" +version = "1.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6dbf3de79e51f3d586ab4cb9d5c3e2c14aa28ed23d180cf89b4df0454a69cc87" + +[[package]] +name = "humantime" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "135b12329e5e3ce057a9f972339ea52bc954fe1e9358ef27f95e89716fbc5424" + +[[package]] +name = "hyper" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6299f016b246a94207e63da54dbe807655bf9e00044f73ded42c3ac5305fbcca" +dependencies = [ + "atomic-waker", + "bytes", + "futures-channel", + "futures-core", + "h2", + "http", + "http-body", + "httparse", + "itoa", + "pin-project-lite", + "smallvec", + "tokio", + "want", +] + +[[package]] +name = "hyper-rustls" +version = "0.27.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33ca68d021ef39cf6463ab54c1d0f5daf03377b70561305bb89a8f83aab66e0f" +dependencies = [ + "http", + "hyper", + "hyper-util", + "rustls", + "rustls-native-certs", + "tokio", + "tokio-rustls", + "tower-service", +] + +[[package]] +name = "hyper-util" +version = "0.1.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96547c2556ec9d12fb1578c4eaf448b04993e7fb79cbaad930a656880a6bdfa0" +dependencies = [ + "base64", + "bytes", + "futures-channel", + "futures-util", + "http", + "http-body", + "hyper", + "ipnet", + "libc", + "percent-encoding", + "pin-project-lite", + "socket2", + "tokio", + "tower-service", + "tracing", +] + +[[package]] +name = "iana-time-zone" +version = "0.1.65" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e31bc9ad994ba00e440a8aa5c9ef0ec67d5cb5e5cb0cc7f8b744a35b389cc470" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "log", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[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 = "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 = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "ipnet" +version = "2.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d98f6fed1fde3f8c21bc40a1abb88dd75e67924f9cffc3ef95607bad8017f8e2" + +[[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 = "jni" +version = "0.22.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5efd9a482cf3a427f00d6b35f14332adc7902ce91efb778580e180ff90fa3498" +dependencies = [ + "cfg-if", + "combine", + "jni-macros", + "jni-sys", + "log", + "simd_cesu8", + "thiserror", + "walkdir", + "windows-link", +] + +[[package]] +name = "jni-macros" +version = "0.22.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a00109accc170f0bdb141fed3e393c565b6f5e072365c3bd58f5b062591560a3" +dependencies = [ + "proc-macro2", + "quote", + "rustc_version", + "simd_cesu8", + "syn", +] + +[[package]] +name = "jni-sys" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6377a88cb3910bee9b0fa88d4f42e1d2da8e79915598f65fb0c7ee14c878af2" +dependencies = [ + "jni-sys-macros", +] + +[[package]] +name = "jni-sys-macros" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38c0b942f458fe50cdac086d2f946512305e5631e720728f2a61aabcd47a6264" +dependencies = [ + "quote", + "syn", +] + +[[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.98" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67df7112613f8bfd9150013a0314e196f4800d3201ae742489d999db2f979f08" +dependencies = [ + "cfg-if", + "futures-util", + "once_cell", + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" + +[[package]] +name = "leb128fmt" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09edd9e8b54e49e587e4f6295a7d29c3ea94d469cb40ab8ca70b288248a81db2" + +[[package]] +name = "lexical-core" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7d8d125a277f807e55a77304455eb7b1cb52f2b18c143b60e766c120bd64a594" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "52a9f232fbd6f550bc0137dcb5f99ab674071ac2d690ac69704593cb4abbea56" +dependencies = [ + "lexical-parse-integer", + "lexical-util", +] + +[[package]] +name = "lexical-parse-integer" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a7a039f8fb9c19c996cd7b2fcce303c1b2874fe1aca544edc85c4a5f8489b34" +dependencies = [ + "lexical-util", +] + +[[package]] +name = "lexical-util" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2604dd126bb14f13fb5d1bd6a66155079cb9fa655b37f875b3a742c705dbed17" + +[[package]] +name = "lexical-write-float" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50c438c87c013188d415fbabbb1dceb44249ab81664efbd31b14ae55dabb6361" +dependencies = [ + "lexical-util", + "lexical-write-integer", +] + +[[package]] +name = "lexical-write-integer" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "409851a618475d2d5796377cad353802345cba92c867d9fbcde9cf4eac4e14df" +dependencies = [ + "lexical-util", +] + +[[package]] +name = "libc" +version = "0.2.186" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68ab91017fe16c622486840e4c83c9a37afeff978bd239b5293d61ece587de66" + +[[package]] +name = "libm" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6d2cec3eae94f9f509c767b45932f1ada8350c4bdb85af2fcab4a3c14807981" + +[[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 = "litrs" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "11d3d7f243d5c5a8b9bb5d6dd2b1602c0cb0b9db1621bafc7ed66e35ff9fe092" + +[[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.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e5032e24019045c762d3c0f28f5b6b8bbf38563a65908389bf7978758920897" + +[[package]] +name = "lru-slab" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" + +[[package]] +name = "lz4_flex" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90071f8077f8e40adfc4b7fe9cd495ce316263f19e75c2211eeff3fdf475a3d9" +dependencies = [ + "twox-hash", +] + +[[package]] +name = "lz4_flex" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7ef0d4ed8669f8f8826eb00dc878084aa8f253506c4fd5e8f58f5bce72ddb97e" +dependencies = [ + "twox-hash", +] + +[[package]] +name = "md-5" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" +dependencies = [ + "cfg-if", + "digest", +] + +[[package]] +name = "memchr" +version = "2.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8ca58f447f06ed17d5fc4043ce1b10dd205e060fb3ce5b979b8ed8e59ff3f79" + +[[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 = "mio" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "50b7e5b27aa02a74bac8c3f23f448f8d87ff11f92d3aac1a6ed369ee08cc56c1" +dependencies = [ + "libc", + "wasi", + "windows-sys 0.61.2", +] + +[[package]] +name = "multimap" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" + +[[package]] +name = "num-bigint" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" +dependencies = [ + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-integer" +version = "0.1.46" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "object_store" +version = "0.12.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbfbfff40aeccab00ec8a910b57ca8ecf4319b335c542f2edcd19dd25a1e2a00" +dependencies = [ + "async-trait", + "base64", + "bytes", + "chrono", + "form_urlencoded", + "futures", + "http", + "http-body-util", + "httparse", + "humantime", + "hyper", + "itertools", + "md-5", + "parking_lot", + "percent-encoding", + "quick-xml", + "rand 0.9.4", + "reqwest", + "ring", + "rustls-pemfile", + "serde", + "serde_json", + "serde_urlencoded", + "thiserror", + "tokio", + "tracing", + "url", + "walkdir", + "wasm-bindgen-futures", + "web-time", +] + +[[package]] +name = "object_store" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "622acbc9100d3c10e2ee15804b0caa40e55c933d5aa53814cd520805b7958a49" +dependencies = [ + "async-trait", + "bytes", + "chrono", + "futures-channel", + "futures-core", + "futures-util", + "http", + "humantime", + "itertools", + "parking_lot", + "percent-encoding", + "thiserror", + "tokio", + "tracing", + "url", + "walkdir", + "wasm-bindgen-futures", + "web-time", +] + +[[package]] +name = "once_cell" +version = "1.21.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9f7c3e4beb33f85d45ae3e3a1792185706c8e16d043238c593331cc7cd313b50" + +[[package]] +name = "openssl-probe" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c87def4c32ab89d880effc9e097653c8da5d6ef28e6b539d313baaacfbafcbe" + +[[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 = "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 = "parquet" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e832c6aa20310fc6de7ea5a3f4e20d34fd83e3b43229d32b81ffe5c14d74692" +dependencies = [ + "ahash", + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-cast 57.3.1", + "arrow-data 57.3.1", + "arrow-ipc 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "base64", + "brotli", + "bytes", + "chrono", + "flate2", + "futures", + "half", + "hashbrown 0.16.1", + "lz4_flex 0.12.2", + "num-bigint", + "num-integer", + "num-traits", + "object_store 0.12.5", + "paste", + "seq-macro", + "simdutf8", + "snap", + "thrift", + "tokio", + "twox-hash", + "zstd", +] + +[[package]] +name = "parquet" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5dafa7d01085b62a47dd0c1829550a0a36710ea9c4fe358a05a85477cec8a908" +dependencies = [ + "ahash", + "arrow-array 58.3.0", + "arrow-buffer 58.3.0", + "arrow-data 58.3.0", + "arrow-ipc 58.3.0", + "arrow-schema 58.3.0", + "arrow-select 58.3.0", + "base64", + "brotli", + "bytes", + "chrono", + "flate2", + "futures", + "half", + "hashbrown 0.17.1", + "lz4_flex 0.13.1", + "num-bigint", + "num-integer", + "num-traits", + "object_store 0.13.2", + "parquet-variant", + "parquet-variant-compute", + "parquet-variant-json", + "paste", + "seq-macro", + "simdutf8", + "snap", + "thrift", + "tokio", + "twox-hash", + "zstd", +] + +[[package]] +name = "parquet-variant" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74c8db065291f088a2aad8ab831853eae1871c0d311c8d0b83bbc3b7e735d0fc" +dependencies = [ + "arrow 58.3.0", + "arrow-schema 58.3.0", + "chrono", + "half", + "indexmap", + "num-traits", + "simdutf8", + "uuid", +] + +[[package]] +name = "parquet-variant-compute" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a530e8d5b5e14efcb39c9a6ec55432ad11f6afb7dc4455a79be0dc615fe3cc31" +dependencies = [ + "arrow 58.3.0", + "arrow-schema 58.3.0", + "chrono", + "half", + "indexmap", + "parquet-variant", + "parquet-variant-json", + "serde_json", + "uuid", +] + +[[package]] +name = "parquet-variant-json" +version = "58.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00ed89908289f67caa2ca078f9ff9aacd6229a313ec92b12bf4f48f613dc2b97" +dependencies = [ + "arrow-schema 58.3.0", + "base64", + "chrono", + "parquet-variant", + "serde_json", + "uuid", +] + +[[package]] +name = "paste" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" + +[[package]] +name = "percent-encoding" +version = "2.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b4f627cb1b25917193a259e49bdad08f671f8d9708acfd5fe0a8c1455d87220" + +[[package]] +name = "petgraph" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" +dependencies = [ + "fixedbitset", + "hashbrown 0.15.5", + "indexmap", + "serde", +] + +[[package]] +name = "phf" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "913273894cec178f401a31ec4b656318d95473527be05c0752cc41cdc32be8b7" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_shared" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06005508882fb681fd97892ecff4b7fd0fee13ef1aa569f8695dae7ab9099981" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a89322df9ebe1c1578d689c92318e070967d1042b512afbe49518723f4e6d5cd" + +[[package]] +name = "pkg-config" +version = "0.3.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19f132c84eca552bf34cab8ec81f1c1dcc229b811638f9d283dceabe58c5569e" + +[[package]] +name = "potential_utf" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0103b1cef7ec0cf76490e969665504990193874ea05c85ff9bab8b911d0a0564" +dependencies = [ + "zerovec", +] + +[[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 = "proc-macro2" +version = "1.0.106" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fd00f0bb2e90d81d1044c2b32617f68fcb9fa3bb7640c23e9c748e53fb30934" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2ea70524a2f82d518bce41317d0fae74151505651af45faf1ffbd6fd33f0568" +dependencies = [ + "bytes", + "prost-derive", +] + +[[package]] +name = "prost-build" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "343d3bd7056eda839b03204e68deff7d1b13aba7af2b2fd16890697274262ee7" +dependencies = [ + "heck", + "itertools", + "log", + "multimap", + "petgraph", + "prettyplease", + "prost", + "prost-types", + "regex", + "syn", + "tempfile", +] + +[[package]] +name = "prost-derive" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "27c6023962132f4b30eb4c172c91ce92d933da334c59c23cddee82358ddafb0b" +dependencies = [ + "anyhow", + "itertools", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "prost-types" +version = "0.14.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8991c4cbdb8bc5b11f0b074ffe286c30e523de90fee5ba8132f1399f23cb3dd7" +dependencies = [ + "prost", +] + +[[package]] +name = "quick-xml" +version = "0.38.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b66c2058c55a409d601666cffe35f04333cf1013010882cec174a7467cd4e21c" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quinn" +version = "0.11.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e20a958963c291dc322d98411f541009df2ced7b5a4f2bd52337638cfccf20" +dependencies = [ + "bytes", + "cfg_aliases", + "pin-project-lite", + "quinn-proto", + "quinn-udp", + "rustc-hash", + "rustls", + "socket2", + "thiserror", + "tokio", + "tracing", + "web-time", +] + +[[package]] +name = "quinn-proto" +version = "0.11.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "434b42fec591c96ef50e21e886936e66d3cc3f737104fdb9b737c40ffb94c098" +dependencies = [ + "bytes", + "getrandom 0.3.4", + "lru-slab", + "rand 0.9.4", + "ring", + "rustc-hash", + "rustls", + "rustls-pki-types", + "slab", + "thiserror", + "tinyvec", + "tracing", + "web-time", +] + +[[package]] +name = "quinn-udp" +version = "0.5.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "addec6a0dcad8a8d96a771f815f0eaf55f9d1805756410b39f5fa81332574cbd" +dependencies = [ + "cfg_aliases", + "libc", + "once_cell", + "socket2", + "tracing", + "windows-sys 0.60.2", +] + +[[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" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2e8e8bcc7961af1fdac401278c6a831614941f6164ee3bf4ce61b7edb162207" +dependencies = [ + "chacha20", + "getrandom 0.4.2", + "rand_core 0.10.1", +] + +[[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.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76afc826de14238e6e8c374ddcc1fa19e374fd8dd986b0d2af0d02377261d83c" +dependencies = [ + "getrandom 0.3.4", +] + +[[package]] +name = "rand_core" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63b8176103e19a2643978565ca18b50549f6101881c443590420e4dc998a3c69" + +[[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.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e10754a14b9137dd7b1e3e5b0493cc9171fdd105e0ab477f51b72e7f3ac0e276" +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.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc897dd8d9e8bd1ed8cdad82b5966c3e0ecae09fb1907d58efaa013543185d0a" + +[[package]] +name = "reqwest" +version = "0.12.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eddd3ca559203180a307f12d114c268abf583f59b03cb906fd0b3ff8646c1147" +dependencies = [ + "base64", + "bytes", + "futures-core", + "futures-util", + "h2", + "http", + "http-body", + "http-body-util", + "hyper", + "hyper-rustls", + "hyper-util", + "js-sys", + "log", + "percent-encoding", + "pin-project-lite", + "quinn", + "rustls", + "rustls-native-certs", + "rustls-pki-types", + "serde", + "serde_json", + "serde_urlencoded", + "sync_wrapper", + "tokio", + "tokio-rustls", + "tokio-util", + "tower", + "tower-http", + "tower-service", + "url", + "wasm-bindgen", + "wasm-bindgen-futures", + "wasm-streams", + "web-sys", +] + +[[package]] +name = "ring" +version = "0.17.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4689e6c2294d81e88dc6261c768b63bc4fcdb852be6d1352498b114f61383b7" +dependencies = [ + "cc", + "cfg-if", + "getrandom 0.2.17", + "libc", + "untrusted", + "windows-sys 0.52.0", +] + +[[package]] +name = "roaring" +version = "0.10.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19e8d2cfa184d94d0726d650a9f4a1be7f9b76ac9fdb954219878dc00c1c1e7b" +dependencies = [ + "bytemuck", + "byteorder", +] + +[[package]] +name = "roaring" +version = "0.11.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1dedc5658c6ecb3bdb5ef5f3295bb9253f42dcf3fd1402c03f6b1f7659c3c4a9" +dependencies = [ + "bytemuck", + "byteorder", +] + +[[package]] +name = "rustc-hash" +version = "2.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94300abf3f1ae2e2b8ffb7b58043de3d399c73fa6f4b73826402a5c457614dbe" + +[[package]] +name = "rustc_version" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" +dependencies = [ + "semver", +] + +[[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 = "rustls" +version = "0.23.40" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef86cd5876211988985292b91c96a8f2d298df24e75989a43a3c73f2d4d8168b" +dependencies = [ + "once_cell", + "ring", + "rustls-pki-types", + "rustls-webpki", + "subtle", + "zeroize", +] + +[[package]] +name = "rustls-native-certs" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "612460d5f7bea540c490b2b6395d8e34a953e52b491accd6c86c8164c5932a63" +dependencies = [ + "openssl-probe", + "rustls-pki-types", + "schannel", + "security-framework", +] + +[[package]] +name = "rustls-pemfile" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dce314e5fee3f39953d46bb63bb8a46d40c2f8fb7cc5a3b6cab2bde9721d6e50" +dependencies = [ + "rustls-pki-types", +] + +[[package]] +name = "rustls-pki-types" +version = "1.14.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30a7197ae7eb376e574fe940d068c30fe0462554a3ddbe4eca7838e049c937a9" +dependencies = [ + "web-time", + "zeroize", +] + +[[package]] +name = "rustls-webpki" +version = "0.103.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61c429a8649f110dddef65e2a5ad240f747e85f7758a6bccc7e5777bd33f756e" +dependencies = [ + "ring", + "rustls-pki-types", + "untrusted", +] + +[[package]] +name = "rustversion" +version = "1.0.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" + +[[package]] +name = "ryu" +version = "1.0.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9774ba4a74de5f7b1c1451ed6cd5285a32eddb5cccb8cc655a4e50009e06477f" + +[[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 = "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" + +[[package]] +name = "seq-macro" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" + +[[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_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_json" +version = "1.0.149" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "83fc039473c5595ace860d8c4fafa220ff474b3fc6bfdb4293327f1a37e94d86" +dependencies = [ + "itoa", + "memchr", + "serde", + "serde_core", + "zmij", +] + +[[package]] +name = "serde_urlencoded" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3491c14715ca2294c4d6a88f15e84739788c1d030eed8c110436aafdaa2f3fd" +dependencies = [ + "form_urlencoded", + "itoa", + "ryu", + "serde", +] + +[[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 = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + +[[package]] +name = "simd-adler32" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "703d5c7ef118737c72f1af64ad2f6f8c5e1921f818cdcb97b8fe6fc69bf66214" + +[[package]] +name = "simd_cesu8" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94f90157bb87cddf702797c5dadfa0be7d266cdf49e22da2fcaa32eff75b2c33" +dependencies = [ + "rustc_version", + "simdutf8", +] + +[[package]] +name = "simdutf8" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3a9fe34e3e7a50316060351f37187a3f546bce95496156754b601a5fa71b76e" + +[[package]] +name = "siphasher" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ee5873ec9cce0195efcb7a4e9507a04cd49aec9c83d0389df45b1ef7ba2e649" + +[[package]] +name = "slab" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c790de23124f9ab44544d7ac05d60440adc586479ce501c1d6d7da3cd8c9cf5" + +[[package]] +name = "smallvec" +version = "1.15.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67b1b7a3b5fe4f1376887184045fcf45c69e92af734b7aaddc05fb777b6fbd03" + +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + +[[package]] +name = "socket2" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3a766e1110788c36f4fa1c2b71b387a7815aa65f88ce0229841826633d93723e" +dependencies = [ + "libc", + "windows-sys 0.61.2", +] + +[[package]] +name = "sqlparser" +version = "0.61.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbf5ea8d4d7c808e1af1cbabebca9a2abe603bcefc22294c5b95018d53200cb7" +dependencies = [ + "log", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6dd45d8fc1c79299bfbb7190e42ccbbdf6a5f52e4a6ad98d92357ea965bd289" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "stable_deref_trait" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" + +[[package]] +name = "strum" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" +dependencies = [ + "strum_macros", +] + +[[package]] +name = "strum_macros" +version = "0.27.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7695ce3845ea4b33927c055a39dc438a45b059f7c1b3d91d38d10355fb8cbca7" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "subtle" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" + +[[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 = "sync_wrapper" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bf256ce5efdfa370213c1dabab5935a12e49f2c58d15e9eac2870d3b4f27263" +dependencies = [ + "futures-core", +] + +[[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 = "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 = "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 = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding", + "ordered-float", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[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 = "tokio" +version = "1.52.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fc7f01b389ac15039e4dc9531aa973a135d7a4135281b12d7c1bc79fd57fffe" +dependencies = [ + "bytes", + "libc", + "mio", + "pin-project-lite", + "socket2", + "tokio-macros", + "windows-sys 0.61.2", +] + +[[package]] +name = "tokio-macros" +version = "2.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "385a6cb71ab9ab790c5fe8d67f1645e6c450a7ce006a33de03daa956cf70a496" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "tokio-rustls" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" +dependencies = [ + "rustls", + "tokio", +] + +[[package]] +name = "tokio-stream" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32da49809aab5c3bc678af03902d4ccddea2a87d028d86392a4b1560c6906c70" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", + "tokio-util", +] + +[[package]] +name = "tokio-util" +version = "0.7.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ae9cec805b01e8fc3fd2fe289f89149a9b66dd16786abd8b19cfa7b48cb0098" +dependencies = [ + "bytes", + "futures-core", + "futures-sink", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tower" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebe5ef63511595f1344e2d5cfa636d973292adc0eec1f0ad45fae9f0851ab1d4" +dependencies = [ + "futures-core", + "futures-util", + "pin-project-lite", + "sync_wrapper", + "tokio", + "tower-layer", + "tower-service", +] + +[[package]] +name = "tower-http" +version = "0.6.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cfcf7e2740e6fc6d4d688b4ef00650406bb94adf4731e43c096c3a19fe40840" +dependencies = [ + "bitflags", + "bytes", + "futures-util", + "http", + "http-body", + "pin-project-lite", + "tower", + "tower-layer", + "tower-service", + "url", +] + +[[package]] +name = "tower-layer" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "121c2a6cda46980bb0fcd1647ffaf6cd3fc79a013de288782836f6df9c48780e" + +[[package]] +name = "tower-service" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" + +[[package]] +name = "tracing" +version = "0.1.44" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "63e71662fa4b2a2c3a26f570f037eb95bb1f85397f3cd8076caed2f026a6d100" +dependencies = [ + "log", + "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-core" +version = "0.1.36" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db97caf9d906fbde555dd62fa95ddba9eecfd14cb388e4f491a66d74cd5fb79a" +dependencies = [ + "once_cell", +] + +[[package]] +name = "try-lock" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" + +[[package]] +name = "twox-hash" +version = "2.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ea3136b675547379c4bd395ca6b938e5ad3c3d20fad76e7fe85f9e0d011419c" + +[[package]] +name = "typenum" +version = "1.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40ce102ab67701b8526c123c1bab5cbe42d7040ccfd0f64af1a385808d2f43de" + +[[package]] +name = "unicode-ident" +version = "1.0.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6e4313cd5fcd3dad5cafa179702e2b244f760991f45397d14d4ebf38247da75" + +[[package]] +name = "unicode-segmentation" +version = "1.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9629274872b2bfaf8d66f5f15725007f635594914870f65218920345aa11aa8c" + +[[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 = "untrusted" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" + +[[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 = "uuid" +version = "1.23.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ddd74a9687298c6858e9b88ec8935ec45d22e8fd5e6394fa1bd4e99a87789c76" +dependencies = [ + "getrandom 0.4.2", + "js-sys", + "rand 0.10.1", + "wasm-bindgen", +] + +[[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 = "want" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bfa7760aed19e106de2c7c0b581b509f2f25d3dacaf737cb82ac61bc6d760b0e" +dependencies = [ + "try-lock", +] + +[[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.3+wasi-0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20064672db26d7cdc89c7798c48a0fdfac8213434a1186e5ef29fd560ae223d6" +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.121" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "49ace1d07c165b0864824eee619580c4689389afa9dc9ed3a4c75040d82e6790" +dependencies = [ + "cfg-if", + "once_cell", + "rustversion", + "wasm-bindgen-macro", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-futures" +version = "0.4.71" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96492d0d3ffba25305a7dc88720d250b1401d7edca02cc3bcd50633b424673b8" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.121" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e68e6f4afd367a562002c05637acb8578ff2dea1943df76afb9e83d177c8578" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.121" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d95a9ec35c64b2a7cb35d3fead40c4238d0940c86d107136999567a4703259f2" +dependencies = [ + "bumpalo", + "proc-macro2", + "quote", + "syn", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.121" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4e0100b01e9f0d03189a92b96772a1fb998639d981193d7dbab487302513441" +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 = "wasm-streams" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "15053d8d85c7eccdbefef60f06769760a563c7f0a9d6902a13d35c7800b0ad65" +dependencies = [ + "futures-util", + "js-sys", + "wasm-bindgen", + "wasm-bindgen-futures", + "web-sys", +] + +[[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 = "web-sys" +version = "0.3.98" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b572dff8bcf38bad0fa19729c89bb5748b2b9b1d8be70cf90df697e3a8f32aa" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "web-time" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6580f308b1fad9207618087a65c04e7a10bc77e02c8e84e9b00dd4b12fa0bb" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[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 = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-core" +version = "0.62.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8e83a14d34d0623b51dce9581199302a221863196a1dde71a7663a4c2be9deb" +dependencies = [ + "windows-implement", + "windows-interface", + "windows-link", + "windows-result", + "windows-strings", +] + +[[package]] +name = "windows-implement" +version = "0.60.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "windows-interface" +version = "0.59.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] + +[[package]] +name = "windows-link" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" + +[[package]] +name = "windows-result" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7781fa89eaf60850ac3d2da7af8e5242a5ea78d1a11c49bf2910bb5a73853eb5" +dependencies = [ + "windows-link", +] + +[[package]] +name = "windows-strings" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7837d08f69c77cf6b07689544538e017c1bfcf57e34b4c0ff58e6c2cd3b37091" +dependencies = [ + "windows-link", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.60.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" +dependencies = [ + "windows-targets 0.53.5", +] + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" +dependencies = [ + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", + "windows_i686_gnullvm 0.52.6", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", +] + +[[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 0.53.1", + "windows_aarch64_msvc 0.53.1", + "windows_i686_gnu 0.53.1", + "windows_i686_gnullvm 0.53.1", + "windows_i686_msvc 0.53.1", + "windows_x86_64_gnu 0.53.1", + "windows_x86_64_gnullvm 0.53.1", + "windows_x86_64_msvc 0.53.1", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" + +[[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.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" + +[[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.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "abe8c5fda708d9ca3df187cae8bfb9ceda00dd96231bed36e445a1a48e66f9ca" +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 = "z85" +version = "3.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6e61e59a957b7ccee15d2049f86e8bfd6f66968fcd88f018950662d9b86e675" + +[[package]] +name = "zerocopy" +version = "0.8.48" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eed437bf9d6692032087e337407a86f04cd8d6a16a37199ed57949d415bd68e9" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.8.48" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "70e3cd084b1788766f53af483dd21f93881ff30d7320490ec3ef7526d203bad4" +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" + +[[package]] +name = "zstd" +version = "0.13.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e91ee311a569c327171651566e07972200e76fcfe2242a4fa446149a3881c08a" +dependencies = [ + "zstd-safe", +] + +[[package]] +name = "zstd-safe" +version = "7.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f49c4d5f0abb602a93fb8736af2a4f4dd9512e36f7f570d66e65ff867ed3b9d" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.16+zstd.1.5.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91e19ebc2adc8f83e43039e79776e3fda8ca919132d68a1fed6a5faca2683748" +dependencies = [ + "cc", + "pkg-config", +] diff --git a/contrib/delta/native/Cargo.toml b/contrib/delta/native/Cargo.toml new file mode 100644 index 0000000000..54288a2424 --- /dev/null +++ b/contrib/delta/native/Cargo.toml @@ -0,0 +1,74 @@ +# 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. + +# Standalone Cargo.toml -- this crate is outside the `native/` workspace root, so it +# cannot use `{ workspace = true }` inheritance. Versions are kept in sync with the +# rest of the repo by convention; the path deps anchor against the same on-disk crates. + +[package] +name = "comet-contrib-delta" +description = "delta-kernel-rs integration for Comet. Reads Delta tables via kernel-rs's log replay and DataFusion's parquet scan, with DV / column-mapping / row-tracking support. Linked into libcomet via core's `contrib-delta` Cargo feature flag." +version = "0.17.0" +edition = "2021" +rust-version = "1.86.0" +publish = false +license = "Apache-2.0" + +[lib] +# rlib: linked INTO `libcomet` when `contrib-delta` is enabled on core. Never a cdylib +# on its own — there's no separate Delta library to ship. +crate-type = ["rlib"] + +[dependencies] +# Typed Delta proto messages live in core's proto crate (alongside IcebergScan, ...) +# so the dispatcher arm has direct access. We re-export them as `crate::proto::*`. +datafusion-comet-proto = { path = "../../../native/proto" } +# JNI helpers (CometError, CometResult, try_unwrap_or_throw). jni-bridge is a leaf +# crate -- depending on it doesn't drag any Comet logic into the contrib. +datafusion-comet-jni-bridge = { path = "../../../native/jni-bridge" } +# Heavy Delta deps -- intentionally live ONLY in this contrib, never in core. delta_kernel +# 0.19 pins arrow-57 / object_store-0.12 internally; that subtree never exchanges typed +# values with Comet's arrow-58 / object_store-0.13 -- only plain Rust types cross the +# boundary (ScanFile, HashMap, etc.). +delta_kernel = { version = "0.19", default-features = false, features = ["default-engine-rustls", "arrow"] } +# Second object_store version required by delta_kernel 0.19 (kernel's engine uses 0.12). +# Renamed so it doesn't collide with Comet's `object_store = "0.13.1"`. +object_store_kernel = { package = "object_store", version = "0.12", features = ["aws", "azure"] } +# Roaring bitmap decoder for Delta deletion vectors (inline + on-disk). +roaring = "0.10" + +# DataFusion / Arrow / parquet versions chosen to match core's pinned values. +datafusion = { version = "53.1.0", default-features = false, features = ["parquet"] } +arrow = { version = "58.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } +object_store = { version = "0.13.1" } +url = "2.5.4" +parquet = { version = "58.1.0", default-features = false, features = ["experimental"] } +futures = "0.3" +thiserror = "2" +prost = "0.14.3" +jni = "0.22.4" +# Used by parse_delta_partition_scalar for timestamp parsing across the JVM's TZ shapes +# (IANA names, GMT+/-HH:MM, etc). +chrono = "0.4" +chrono-tz = "0.10" +log = "0.4" + +[dev-dependencies] +# Used by unit tests under #[cfg(test)] in scan.rs to materialise a Delta table +# in a tempdir without polluting the real filesystem. +tempfile = "3" +tokio = { version = "1.39.0", features = ["macros", "rt-multi-thread"] } diff --git a/contrib/delta/native/src/dv_filter.rs b/contrib/delta/native/src/dv_filter.rs new file mode 100644 index 0000000000..75f9e012d2 --- /dev/null +++ b/contrib/delta/native/src/dv_filter.rs @@ -0,0 +1,311 @@ +// 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. + +//! Delta Lake deletion-vector filter operator. +//! +//! Wraps a child `ExecutionPlan` (produced by `init_datasource_exec` over the +//! list of Delta parquet files) and applies Delta deletion vectors at the +//! batch level. One `Vec` of deleted row indexes per partition drives +//! the filter. +//! +//! Design notes: +//! +//! - **One file per partition.** The planner match arm places each DV'd +//! file in its own `FileGroup`, so when this operator sees partition +//! `i`, it knows the full set of rows that `ParquetSource` is going to +//! emit for that partition is exactly the physical rows of one file +//! in physical order. That's the only assumption we rely on for the +//! "subtract deleted indexes by tracking a running row offset" strategy +//! to be correct. +//! +//! - **Indexes are pre-materialized.** Kernel already turned the DV +//! (inline bitmap / on-disk file / UUID reference) into a sorted +//! `Vec` on the driver via `DvInfo::get_row_indexes`. That's what +//! `plan_delta_scan` returns. We don't touch DV bytes on the executor +//! side at all. +//! +//! - **Filter uses arrow `filter_record_batch`.** Builds a per-batch +//! `BooleanArray` mask where `true` means "keep". One mask per batch, +//! allocated fresh — the batch sizes are small and allocation overhead +//! is negligible compared with decoding parquet. + +use std::any::Any; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use arrow::array::{BooleanArray, RecordBatch}; +use arrow::compute::filter_record_batch; +use arrow::datatypes::SchemaRef; +use datafusion::common::{DataFusionError, Result as DFResult}; +use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; +use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::metrics::{ + BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, +}; +use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use futures::{Stream, StreamExt}; + +/// Execution-plan wrapper that applies per-partition deletion-vector filters +/// to the output of a child parquet scan. +/// +/// `deleted_row_indexes_by_partition[i]` is the sorted list of physical row +/// indexes to drop from partition `i`'s output. An empty vec means "no DV +/// for this partition — pass through untouched". +#[derive(Debug)] +pub struct DeltaDvFilterExec { + input: Arc, + /// One entry per output partition. Length must match the input's + /// partition count. + deleted_row_indexes_by_partition: Vec>, + plan_properties: Arc, + metrics: ExecutionPlanMetricsSet, +} + +impl DeltaDvFilterExec { + pub fn new( + input: Arc, + deleted_row_indexes_by_partition: Vec>, + ) -> DFResult { + let input_props = input.properties(); + let num_partitions = input_props.output_partitioning().partition_count(); + if deleted_row_indexes_by_partition.len() != num_partitions { + return Err(DataFusionError::Internal(format!( + "DeltaDvFilterExec: got {} DV entries for {} partitions", + deleted_row_indexes_by_partition.len(), + num_partitions + ))); + } + let plan_properties = Arc::new(PlanProperties::new( + EquivalenceProperties::new(input.schema()), + input_props.output_partitioning().clone(), + EmissionType::Incremental, + Boundedness::Bounded, + )); + Ok(Self { + input, + deleted_row_indexes_by_partition, + plan_properties, + metrics: ExecutionPlanMetricsSet::new(), + }) + } +} + +impl DisplayAs for DeltaDvFilterExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + let total_dv: usize = self + .deleted_row_indexes_by_partition + .iter() + .map(|v| v.len()) + .sum(); + let dv_partitions = self + .deleted_row_indexes_by_partition + .iter() + .filter(|v| !v.is_empty()) + .count(); + write!( + f, + "DeltaDvFilterExec: {dv_partitions} partitions with DVs, \ + {total_dv} total deleted rows" + ) + } +} + +impl ExecutionPlan for DeltaDvFilterExec { + fn name(&self) -> &str { + "DeltaDvFilterExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &Arc { + &self.plan_properties + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + // DV filtering relies on `current_row_offset` matching the child's physical row + // index. That invariant only holds if (a) the child preserves its input order and + // (b) DataFusion doesn't slip in a RepartitionExec / SortPreservingMergeExec that + // interleaves rows between the parquet scan and this exec. Override both to pin + // the contract: if either ever stops being true the optimizer is forced to bail + // rather than silently re-order rows. + fn maintains_input_order(&self) -> Vec { + vec![true] + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> DFResult> { + if children.len() != 1 { + return Err(DataFusionError::Internal(format!( + "DeltaDvFilterExec takes exactly one child, got {}", + children.len() + ))); + } + Ok(Arc::new(Self::new( + Arc::clone(&children[0]), + self.deleted_row_indexes_by_partition.clone(), + )?)) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> DFResult { + let child_stream = self.input.execute(partition, context)?; + let deleted = self + .deleted_row_indexes_by_partition + .get(partition) + .cloned() + .unwrap_or_default(); + let metrics = DeltaDvFilterMetrics::new(&self.metrics, partition); + metrics.num_deleted.add(deleted.len()); + Ok(Box::pin(DeltaDvFilterStream { + inner: child_stream, + deleted, + current_row_offset: 0, + next_delete_idx: 0, + schema: self.input.schema(), + baseline_metrics: metrics.baseline, + rows_dropped_metric: metrics.rows_dropped, + })) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } +} + +struct DeltaDvFilterMetrics { + baseline: BaselineMetrics, + num_deleted: Count, + rows_dropped: Count, +} + +impl DeltaDvFilterMetrics { + fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { + Self { + baseline: BaselineMetrics::new(metrics, partition), + num_deleted: MetricBuilder::new(metrics).counter("dv_rows_scheduled_delete", partition), + rows_dropped: MetricBuilder::new(metrics).counter("dv_rows_dropped", partition), + } + } +} + +struct DeltaDvFilterStream { + inner: SendableRecordBatchStream, + /// Sorted deleted row indexes for this partition. + deleted: Vec, + /// Physical row offset into the file that the NEXT batch starts at. + current_row_offset: u64, + /// Index into `deleted` of the first entry that hasn't been applied yet. + /// `deleted[..next_delete_idx]` are all strictly less than + /// `current_row_offset`. + next_delete_idx: usize, + schema: SchemaRef, + baseline_metrics: BaselineMetrics, + rows_dropped_metric: Count, +} + +impl DeltaDvFilterStream { + /// Drop rows from `batch` whose physical row index is in the DV. Returns + /// the filtered batch (possibly empty) and advances `current_row_offset`. + fn apply(&mut self, batch: RecordBatch) -> DFResult { + let batch_rows = batch.num_rows() as u64; + if batch_rows == 0 || self.deleted.is_empty() { + self.current_row_offset += batch_rows; + return Ok(batch); + } + + let batch_start = self.current_row_offset; + let batch_end = batch_start + batch_rows; + + // Fast-path: if no remaining deletes fall into this batch's row + // range, pass it through untouched. + if self.next_delete_idx >= self.deleted.len() + || self.deleted[self.next_delete_idx] >= batch_end + { + self.current_row_offset = batch_end; + return Ok(batch); + } + + // Build the keep-mask. Walk forward through `deleted` popping entries + // that fall inside [batch_start, batch_end). + let mut mask_buf: Vec = vec![true; batch_rows as usize]; + let mut dropped: usize = 0; + // Loop is safe: next_delete_idx < deleted.len() is checked by the while + // condition, and deleted is sorted ascending by the kernel contract. + while self.next_delete_idx < self.deleted.len() { + let d = self.deleted[self.next_delete_idx]; + if d >= batch_end { + break; + } + if d < batch_start { + return Err(DataFusionError::Internal(format!( + "DV index {d} predates batch start {batch_start}" + ))); + } + let local = (d - batch_start) as usize; + if local < mask_buf.len() && mask_buf[local] { + mask_buf[local] = false; + dropped += 1; + } + self.next_delete_idx += 1; + } + + self.current_row_offset = batch_end; + self.rows_dropped_metric.add(dropped); + + if dropped == 0 { + return Ok(batch); + } + let mask = BooleanArray::from(mask_buf); + filter_record_batch(&batch, &mask).map_err(DataFusionError::from) + } +} + +impl Stream for DeltaDvFilterStream { + type Item = DFResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let poll = self.inner.poll_next_unpin(cx); + let result = match poll { + Poll::Ready(Some(Ok(batch))) => Poll::Ready(Some(self.apply(batch))), + other => other, + }; + self.baseline_metrics.record_poll(result) + } +} + +impl RecordBatchStream for DeltaDvFilterStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} diff --git a/contrib/delta/native/src/engine.rs b/contrib/delta/native/src/engine.rs new file mode 100644 index 0000000000..2e0945003d --- /dev/null +++ b/contrib/delta/native/src/engine.rs @@ -0,0 +1,196 @@ +// 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. + +//! Construction of a delta-kernel-rs `DefaultEngine` backed by `object_store`. +//! +//! Ported from tantivy4java's `delta_reader/engine.rs` (Apache-2.0) with +//! minor changes: uses Comet's error type instead of `anyhow`, and uses the +//! renamed `object_store_kernel` (object_store 0.12) dependency that kernel +//! requires. Comet's main `object_store = "0.13"` tree is untouched. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex, OnceLock}; +use url::Url; + +use delta_kernel::engine::default::executor::tokio::TokioBackgroundExecutor; +use delta_kernel::engine::default::DefaultEngine; +use object_store_kernel::aws::AmazonS3Builder; +use object_store_kernel::azure::MicrosoftAzureBuilder; +use object_store_kernel::local::LocalFileSystem; +use object_store_kernel::ObjectStore; + +use super::error::{DeltaError, DeltaResult}; + +/// Concrete engine type returned by [`get_or_create_engine`]. +pub type DeltaEngine = DefaultEngine; + +/// Storage credentials used to construct kernel's engine. +/// +/// Mirrors tantivy4java's `DeltaStorageConfig`. Field-per-knob rather than a +/// generic map so we can validate at the boundary; the Scala side will +/// populate this from a Spark options map. +#[derive(Debug, Clone, Default, Hash, PartialEq, Eq)] +pub struct DeltaStorageConfig { + pub aws_access_key: Option, + pub aws_secret_key: Option, + pub aws_session_token: Option, + pub aws_region: Option, + pub aws_endpoint: Option, + pub aws_force_path_style: bool, + + pub azure_account_name: Option, + pub azure_access_key: Option, + pub azure_bearer_token: Option, +} + +/// Build an `ObjectStore` for the given URL and credentials. +/// +/// Supports `s3://` / `s3a://`, `az://` / `azure://` / `abfs://` / `abfss://`, +/// and `file://`. Any other scheme is rejected with +/// [`DeltaError::UnsupportedScheme`]. +pub fn create_object_store( + url: &Url, + config: &DeltaStorageConfig, +) -> DeltaResult> { + let scheme = url.scheme(); + + let store: Arc = match scheme { + "s3" | "s3a" => { + let bucket = url.host_str().ok_or_else(|| DeltaError::MissingBucket { + url: url.to_string(), + })?; + let mut builder = AmazonS3Builder::new().with_bucket_name(bucket); + + if let Some(ref key) = config.aws_access_key { + builder = builder.with_access_key_id(key); + } + if let Some(ref secret) = config.aws_secret_key { + builder = builder.with_secret_access_key(secret); + } + if let Some(ref token) = config.aws_session_token { + builder = builder.with_token(token); + } + if let Some(ref region) = config.aws_region { + builder = builder.with_region(region); + } + if let Some(ref endpoint) = config.aws_endpoint { + builder = builder.with_endpoint(endpoint); + } + if config.aws_force_path_style { + builder = builder.with_virtual_hosted_style_request(false); + } + // Allow HTTP endpoints (MinIO, LocalStack, custom S3-compat) + if config + .aws_endpoint + .as_ref() + .is_some_and(|e| e.starts_with("http://")) + { + builder = builder.with_allow_http(true); + } + + Arc::new(builder.build()?) + } + "az" | "azure" | "abfs" | "abfss" => { + let container = url.host_str().ok_or_else(|| DeltaError::MissingBucket { + url: url.to_string(), + })?; + let mut builder = MicrosoftAzureBuilder::new().with_container_name(container); + + if let Some(ref account) = config.azure_account_name { + builder = builder.with_account(account); + } + if let Some(ref key) = config.azure_access_key { + builder = builder.with_access_key(key); + } + if let Some(ref token) = config.azure_bearer_token { + builder = builder.with_bearer_token_authorization(token); + } + + Arc::new(builder.build()?) + } + "file" | "" => Arc::new(LocalFileSystem::new()), + other => { + return Err(DeltaError::UnsupportedScheme { + scheme: other.to_string(), + url: url.to_string(), + }); + } + }; + + Ok(store) +} + +/// Process-wide cache of constructed engines, keyed by (scheme, authority, config). +/// +/// Each `DefaultEngine` owns a `TokioBackgroundExecutor` which spawns one std::thread +/// running a current_thread tokio runtime; the runtime's blocking pool (used by +/// kernel for parquet/object_store IO) holds spawned threads for `thread_keep_alive` +/// (~10s) after each spawn_blocking call. Constructing a fresh engine per JNI +/// `planDeltaScan` call therefore accumulates OS threads during regression runs that +/// hit kernel hundreds of times per minute, eventually tripping the per-process +/// thread cap (e.g. `pthread_create EAGAIN` aborts on macOS where `ulimit -u` +/// defaults to ~1300). Sharing one engine per (scheme, authority, config) bounds the +/// thread count by table-storage diversity instead of by request count. +/// +/// `Arc` is handed out so callers don't hold the mutex while using the +/// engine. We never evict — entries are cheap (one Arc per distinct storage target), +/// and dropping the cache at JVM teardown is acceptable. +type EngineKey = (String, String, DeltaStorageConfig); +fn engine_cache() -> &'static Mutex>> { + static CACHE: OnceLock>>> = OnceLock::new(); + CACHE.get_or_init(|| Mutex::new(HashMap::new())) +} + +fn engine_key(url: &Url, config: &DeltaStorageConfig) -> EngineKey { + let scheme = url.scheme().to_string(); + // host+port form the storage target (e.g. S3 bucket, ABFS account); for file:// + // the authority is empty which collapses every local table to a single entry. + let authority = match (url.host_str(), url.port()) { + (Some(h), Some(p)) => format!("{h}:{p}"), + (Some(h), None) => h.to_string(), + _ => String::new(), + }; + (scheme, authority, config.clone()) +} + +// Suppress dead_code: the standalone constructor stays useful for tests that want +// to exercise a fresh engine without polluting the cache. +#[allow(dead_code)] +pub fn create_engine(table_url: &Url, config: &DeltaStorageConfig) -> DeltaResult { + let store = create_object_store(table_url, config)?; + Ok(DefaultEngine::new(store)) +} + +/// Return a shared `DeltaEngine` for the given URL+config, building one on first use. +pub fn get_or_create_engine( + table_url: &Url, + config: &DeltaStorageConfig, +) -> DeltaResult> { + let key = engine_key(table_url, config); + // Mutex is held only across the (cheap) HashMap lookup and, on miss, the engine + // construction. Multi-threaded JNI callers serialize here on first miss per key + // but proceed lock-free on subsequent hits via the returned Arc clone. + let mut cache = engine_cache().lock().unwrap_or_else(|e| e.into_inner()); + if let Some(existing) = cache.get(&key) { + return Ok(Arc::clone(existing)); + } + let store = create_object_store(table_url, config)?; + let engine = Arc::new(DefaultEngine::new(store)); + cache.insert(key, Arc::clone(&engine)); + Ok(engine) +} + diff --git a/contrib/delta/native/src/error.rs b/contrib/delta/native/src/error.rs new file mode 100644 index 0000000000..fd61e493ad --- /dev/null +++ b/contrib/delta/native/src/error.rs @@ -0,0 +1,62 @@ +// 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. + +//! Error types for the delta module. +//! +//! Kept local rather than folded into `CometError` because `delta_kernel` +//! lives in an isolated dep subtree — we don't want kernel's error type +//! leaking into `errors.rs` where it could pull kernel's arrow-57 into the +//! main error path. + +use thiserror::Error; + +#[derive(Debug, Error)] +pub enum DeltaError { + #[error("invalid delta table URL '{url}': {source}")] + InvalidUrl { + url: String, + #[source] + source: url::ParseError, + }, + + #[error("cannot resolve local path '{path}': {source}")] + PathResolution { + path: String, + #[source] + source: std::io::Error, + }, + + #[error("cannot convert path to URL: {path}")] + PathToUrl { path: String }, + + #[error("unsupported URL scheme '{scheme}' for delta table: {url}")] + UnsupportedScheme { scheme: String, url: String }, + + #[error("missing bucket/container in URL: {url}")] + MissingBucket { url: String }, + + #[error("object store construction failed: {0}")] + ObjectStore(#[from] object_store_kernel::Error), + + #[error("delta kernel error: {0}")] + Kernel(#[from] delta_kernel::Error), + + #[error("{0}")] + Internal(String), +} + +pub type DeltaResult = std::result::Result; diff --git a/contrib/delta/native/src/jni.rs b/contrib/delta/native/src/jni.rs new file mode 100644 index 0000000000..a26dcab591 --- /dev/null +++ b/contrib/delta/native/src/jni.rs @@ -0,0 +1,403 @@ +// 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. + +//! Driver-side JNI entry point for Delta log replay. +//! +//! Exposes `Java_org_apache_comet_contrib_delta_Native_planDeltaScan`. The Scala driver +//! calls this once per query to ask kernel for the active file list at a +//! given snapshot version, then distributes the returned tasks across +//! Spark executors via Comet's usual split-mode serialization. + +use jni::{ + objects::{JByteArray, JClass, JMap, JObject, JString}, + sys::{jbyteArray, jlong}, + Env, EnvUnowned, +}; +use prost::Message; + +use crate::scan::plan_delta_scan_with_predicate; +use crate::DeltaStorageConfig; +use datafusion_comet_jni_bridge::errors::{try_unwrap_or_throw, CometError, CometResult}; +// Proto types now live in this contrib's own proto module (was core's +// datafusion_comet_proto::spark_operator). +use crate::proto::{DeltaPartitionValue, DeltaScanTask, DeltaScanTaskList}; + +/// `Java_org_apache_comet_contrib_delta_Native_planDeltaScan`. +/// +/// # Arguments (JNI wire order) +/// 1. `table_url` — absolute URL or bare path of the Delta table root +/// 2. `snapshot_version` — `-1` for latest, otherwise the exact version +/// 3. `storage_options` — a `java.util.Map` of cloud +/// credentials. **Phase 1 currently only consumes a small subset** (the +/// AWS / Azure keys listed in `DeltaStorageConfig`); unknown keys are +/// silently ignored. Full options-map plumbing lands with Phase 2. +/// +/// # Returns +/// A Java `byte[]` containing a prost-encoded [`DeltaScanTaskList`] +/// message, or `null` on error (with a `CometNativeException` thrown on +/// the JVM side via `try_unwrap_or_throw`). +/// +/// # Safety +/// Inherently unsafe because it dereferences raw JNI pointers. +#[no_mangle] +pub unsafe extern "system" fn Java_org_apache_comet_contrib_delta_Native_planDeltaScan( + e: EnvUnowned, + _class: JClass, + table_url: JString, + snapshot_version: jlong, + storage_options: JObject, + predicate_bytes: JByteArray, + column_names: jni::objects::JObjectArray, +) -> jbyteArray { + try_unwrap_or_throw(&e, |env| { + let url_str: String = table_url.try_to_string(env)?; + let version = if snapshot_version < 0 { + None + } else { + Some(snapshot_version as u64) + }; + let config = if storage_options.is_null() { + DeltaStorageConfig::default() + } else { + let jmap: JMap<'_> = env.cast_local::(storage_options)?; + // TODO(contrib-delta): the rich Hadoop credential-provider chain (PR1 on + // delta-kernel-phase-1, commit 461fa4f4) called into + // `core::parquet::objectstore::s3::resolve_static_credentials` to walk + // SimpleAWSCredentialsProvider / TemporaryAWSCredentialsProvider / + // AssumedRoleCredentialProvider / IAMInstanceCredentialsProvider. That + // helper lives in core and is not exposed through `comet-contrib-spi`. + // For PR2 we'll either (a) move the helper into contrib-spi or a shared + // leaf crate, or (b) re-implement a Delta-local credential resolver. The + // local-fs regression doesn't hit this path so we defer for the validation + // build; cloud-storage Delta tables will need this re-enabled before ship. + extract_storage_config(env, &jmap)? + }; + + // Phase 2: read column names for BoundReference resolution. + // storageOptions map carries Hadoop-style keys (fs.s3a.access.key, + // fs.s3a.secret.key, fs.s3a.endpoint, fs.s3a.path.style.access, + // fs.s3a.endpoint.region, fs.s3a.session.token) extracted by + // NativeConfig.extractObjectStoreOptions on the Scala side. + // extract_storage_config below maps these to kernel's DeltaStorageConfig. + let col_names = read_string_array(env, &column_names)?; + + // Phase 2: deserialize the Catalyst predicate (if provided) for + // kernel's stats-based file pruning. Empty bytes = no predicate. + let _predicate_proto: Option> = if predicate_bytes.is_null() { + None + } else { + let bytes = env.convert_byte_array(predicate_bytes)?; + if bytes.is_empty() { + None + } else { + Some(bytes) + } + }; + + // Phase 2: translate Catalyst predicate proto to kernel Predicate for + // stats-based file pruning during log replay. Pass column names for + // BoundReference index-to-name resolution. + let kernel_predicate = _predicate_proto.and_then(|bytes| { + use prost::Message; + match datafusion_comet_proto::spark_expression::Expr::decode(bytes.as_slice()) { + Ok(expr) => Some( + crate::predicate::catalyst_to_kernel_predicate_with_names( + &expr, &col_names, + ), + ), + Err(e) => { + log::warn!( + "Failed to decode predicate for Delta file pruning: {e}; \ + scanning all files" + ); + None + } + } + }); + + let plan = plan_delta_scan_with_predicate(&url_str, &config, version, kernel_predicate) + .map_err(|e| CometError::Internal(format!("delta_kernel log replay failed: {e}")))?; + + // Under column mapping, kernel returns partition_values keyed by the + // PHYSICAL column name (e.g. `col-`), but `partition_schema` + // (and therefore `build_delta_partitioned_files`'s lookup) uses the + // LOGICAL name. Build the inverse lookup so we can translate keys + // back to logical names on the wire. + let physical_to_logical: std::collections::HashMap = plan + .column_mappings + .iter() + .map(|(logical, physical)| (physical.clone(), logical.clone())) + .collect(); + + let tasks: Vec = plan + .entries + .into_iter() + .map(|entry| DeltaScanTask { + file_path: resolve_file_path(&url_str, &entry.path), + file_size: entry.size as u64, + record_count: entry.num_records, + // Partition values are produced by kernel as an + // unordered `HashMap` per file. Translate + // physical -> logical when a column mapping is present so + // `build_delta_partitioned_files` can match by logical name. + partition_values: entry + .partition_values + .into_iter() + .map(|(name, value)| { + let logical_name = physical_to_logical + .get(&name) + .cloned() + .unwrap_or(name); + DeltaPartitionValue { + name: logical_name, + value: Some(value), + } + }) + .collect(), + // Phase 3: the DV is already materialized into a sorted + // `Vec` of deleted row indexes by `plan_delta_scan` + // (which calls `DvInfo::get_row_indexes` on the driver). + deleted_row_indexes: entry.deleted_row_indexes, + // Row tracking: kernel 0.19.x doesn't yet surface baseRowId / + // defaultRowCommitVersion on the ScanFile path (it's read during + // log replay but consumed internally for TransformSpec). Leave + // unset on the kernel plan path; the pre-materialised-index + // path on the Scala side fills these in from AddFile when + // rowTracking is enabled. + base_row_id: None, + default_row_commit_version: None, + // Splitting is done on the Scala side just before serialization, + // not here on the kernel-driver path. Leave unset. + byte_range_start: None, + byte_range_end: None, + }) + .collect(); + + let column_mappings: Vec = plan + .column_mappings + .into_iter() + .map( + |(logical, physical)| crate::proto::DeltaColumnMapping { + logical_name: logical, + physical_name: physical, + }, + ) + .collect(); + + let msg = DeltaScanTaskList { + snapshot_version: plan.version, + table_root: url_str, + tasks, + unsupported_features: plan.unsupported_features, + column_mappings, + }; + + let bytes = msg.encode_to_vec(); + let result = env.byte_array_from_slice(&bytes)?; + Ok(result.into_raw()) + }) +} + +/// Join `entry.path` (Delta add-action path, usually relative to the +/// table root) with `table_root` to yield an absolute URL the native-side +/// `build_delta_partitioned_files` can feed straight into +/// `object_store::path::Path::from_url_path`. +fn resolve_file_path(table_root: &str, relative: &str) -> String { + // Fully-qualified paths (kernel surfaces these for some tables, e.g. after + // MERGE, REPLACE, or SHALLOW CLONE) pass through untouched. Accept both + // `file:///abs` (authority form) and `file:/abs` (Hadoop `Path.toUri` form, + // which SHALLOW CLONE uses when it stores absolute paths in AddFile.path). + if has_uri_scheme(relative) { + return relative.to_string(); + } + + if table_root.ends_with('/') { + format!("{table_root}{relative}") + } else { + format!("{table_root}/{relative}") + } +} + +/// True if `s` starts with a URI scheme — `^[A-Za-z][A-Za-z0-9+.-]*:` per RFC 3986. +/// We check the scheme only (not whether a `//` authority follows) because Hadoop's +/// `Path.toUri.toString` emits `file:/abs` (single slash) for local absolute paths +/// and Delta stores that form verbatim in AddFile.path for SHALLOW CLONE tables. +fn has_uri_scheme(s: &str) -> bool { + let bytes = s.as_bytes(); + if bytes.is_empty() || !bytes[0].is_ascii_alphabetic() { + return false; + } + for (i, &b) in bytes.iter().enumerate().skip(1) { + if b == b':' { + return i >= 1; + } + if !(b.is_ascii_alphanumeric() || b == b'+' || b == b'-' || b == b'.') { + return false; + } + } + false +} + +/// Walk a `java.util.Map` of storage options into a +/// [`DeltaStorageConfig`]. Checks both kernel-style keys (`aws_access_key_id`) +/// and Hadoop-style keys (`fs.s3a.access.key`) since Comet's +/// `NativeConfig.extractObjectStoreOptions` passes the latter. +fn extract_storage_config(env: &mut Env, jmap: &JMap<'_>) -> CometResult { + // Helper: try kernel key first, fall back to Hadoop key. + let get = |env: &mut Env, k1: &str, k2: &str| -> CometResult> { + let v = map_get_string(env, jmap, k1)?; + if v.is_some() { + return Ok(v); + } + map_get_string(env, jmap, k2) + }; + + Ok(DeltaStorageConfig { + aws_access_key: get(env, "aws_access_key_id", "fs.s3a.access.key")?, + aws_secret_key: get(env, "aws_secret_access_key", "fs.s3a.secret.key")?, + aws_session_token: get(env, "aws_session_token", "fs.s3a.session.token")?, + aws_region: get(env, "aws_region", "fs.s3a.endpoint.region")?.or(map_get_string( + env, + jmap, + "fs.s3a.region", + )?), + aws_endpoint: get(env, "aws_endpoint", "fs.s3a.endpoint")?, + aws_force_path_style: get(env, "aws_force_path_style", "fs.s3a.path.style.access")? + .map(|s| s == "true") + .unwrap_or(false), + azure_account_name: map_get_string(env, jmap, "azure_account_name")?, + azure_access_key: map_get_string(env, jmap, "azure_access_key")?, + azure_bearer_token: map_get_string(env, jmap, "azure_bearer_token")?, + }) +} + +/// Read a Java `String[]` into a `Vec`. Returns empty vec for null arrays. +fn read_string_array(env: &mut Env, arr: &jni::objects::JObjectArray) -> CometResult> { + if arr.is_null() { + return Ok(Vec::new()); + } + let len = arr.len(env)?; + let mut result = Vec::with_capacity(len); + for i in 0..len { + let obj = arr.get_element(env, i)?; + // SAFETY: get_element returns a valid local JObject reference that we + // immediately convert to JString. The array is String[], so the cast + // is valid. The env lifetime outlives this scope. + let jstr = unsafe { JString::from_raw(env, obj.into_raw()) }; + result.push(jstr.try_to_string(env)?); + } + Ok(result) +} + +/// Iterate a `java.util.Map` into a Rust `HashMap`. Used when we need to +/// pass the full Hadoop config map to a downstream consumer (e.g., +/// `s3::resolve_static_credentials`) that walks its own provider chain. +/// +/// Uses `env.cast_local::(...)` to safely downcast each key/value entry rather +/// than the `unsafe { JString::from_raw(..., into_raw()) }` shortcut used elsewhere in +/// this file -- the runtime cast performs the same JNI-side type check the JLS implies +/// for `Map` but without the unchecked transmute. +fn jmap_to_hashmap( + env: &mut Env, + jmap: &JMap<'_>, +) -> CometResult> { + let mut out = std::collections::HashMap::new(); + jmap.iter(env).and_then(|mut iter| { + while let Some(entry) = iter.next(env)? { + let k = entry.key(env)?; + let v = entry.value(env)?; + let kstr: JString = env.cast_local::(k)?; + let key = kstr.try_to_string(env)?; + if !v.is_null() { + let vstr: JString = env.cast_local::(v)?; + let value = vstr.try_to_string(env)?; + out.insert(key, value); + } + } + Ok(()) + })?; + Ok(out) +} + +/// `map.get(key)` for a `java.util.Map` surfaced as a +/// `JMap`. Returns `None` if the key is absent or the value is `null`. +fn map_get_string(env: &mut Env, jmap: &JMap<'_>, key: &str) -> CometResult> { + let key_obj = env.new_string(key)?; + let key_jobj: JObject = key_obj.into(); + match jmap.get(env, &key_jobj)? { + None => Ok(None), + Some(value) => { + // SAFETY: Map::get always returns a String. The + // JObject reference is valid because JMap::get returned it from the + // current env frame. We consume the local ref via into_raw(). + let jstr = unsafe { JString::from_raw(env, value.into_raw()) }; + Ok(Some(jstr.try_to_string(env)?)) + } + } +} + +// Re-export the test helpers so the integration_tests module can verify +// `resolve_file_path` without exposing it in the public API surface. +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn resolve_file_path_joins_with_slash() { + assert_eq!( + resolve_file_path("file:///tmp/t/", "part-0.parquet"), + "file:///tmp/t/part-0.parquet" + ); + assert_eq!( + resolve_file_path("file:///tmp/t", "part-0.parquet"), + "file:///tmp/t/part-0.parquet" + ); + } + + #[test] + fn resolve_file_path_passes_through_absolute() { + assert_eq!( + resolve_file_path("file:///tmp/t/", "s3://bucket/data/part-0.parquet"), + "s3://bucket/data/part-0.parquet" + ); + } + + #[test] + fn resolve_file_path_passes_through_single_slash_file_uri() { + // SHALLOW CLONE stores paths as Hadoop `Path.toUri.toString` which uses + // single-slash form `file:/abs/...`. Must not be concat'd onto the clone root. + assert_eq!( + resolve_file_path( + "file:/tmp/clonetable/", + "file:/tmp/parquet_table/part-0.parquet" + ), + "file:/tmp/parquet_table/part-0.parquet" + ); + } + + #[test] + fn has_uri_scheme_matches_schemes() { + assert!(has_uri_scheme("file:/abs")); + assert!(has_uri_scheme("file:///abs")); + assert!(has_uri_scheme("s3://bucket/k")); + assert!(has_uri_scheme("hdfs://nn/path")); + assert!(!has_uri_scheme("part-0.parquet")); + assert!(!has_uri_scheme("/abs/path")); + assert!(!has_uri_scheme("1bad:/scheme")); // must start with letter + assert!(!has_uri_scheme("")); + } +} diff --git a/contrib/delta/native/src/lib.rs b/contrib/delta/native/src/lib.rs new file mode 100644 index 0000000000..aca4be42ad --- /dev/null +++ b/contrib/delta/native/src/lib.rs @@ -0,0 +1,58 @@ +// 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. + +//! Delta Lake integration for Apache DataFusion Comet. +//! +//! Enabled in core via `--features contrib-delta`. Default builds carry zero +//! Delta surface; this crate is not linked unless the feature is on. +//! +//! Surfaces: +//! - JNI: `Java_org_apache_comet_contrib_delta_Native_planDeltaScan` (driver-side +//! log replay via delta-kernel-rs; returns a `DeltaScanTaskList` proto) +//! - [`DeltaDvFilterExec`]: deletion-vector filter exec wrapper, constructed by +//! core's planner dispatcher when any task in the scan carries a DV +//! - [`plan_delta_scan`]: helpers core's planner dispatcher invokes to assemble +//! a Delta scan's `DataSourceExec` (kernel-rs is JVM-side, so the per-scan +//! planning the JVM doesn't pre-resolve happens here) +//! +//! No `#[ctor]` registration, no contrib-private operator-planner registry; this +//! crate exposes plain Rust functions that core calls directly under +//! `#[cfg(feature = "contrib-delta")]`. + +pub mod dv_filter; +pub mod engine; +pub mod error; +pub mod jni; +pub mod planner; +pub mod predicate; +pub mod scan; + +/// Re-export of the Delta proto messages, named so module paths inside this crate +/// can keep their original `use crate::proto::Delta...` form. The messages +/// themselves live in core's proto crate (so the dispatcher arm in core has direct +/// access to the typed variants). +pub mod proto { + pub use datafusion_comet_proto::spark_operator::{ + DeltaColumnMapping, DeltaPartitionValue, DeltaScan, DeltaScanCommon, DeltaScanTask, + DeltaScanTaskList, + }; +} + +pub use dv_filter::DeltaDvFilterExec; +pub use engine::{create_engine, DeltaStorageConfig}; +pub use error::{DeltaError, DeltaResult}; +pub use scan::{list_delta_files, plan_delta_scan, DeltaFileEntry, DeltaScanPlan}; diff --git a/contrib/delta/native/src/planner.rs b/contrib/delta/native/src/planner.rs new file mode 100644 index 0000000000..eb3d9d4c8a --- /dev/null +++ b/contrib/delta/native/src/planner.rs @@ -0,0 +1,296 @@ +// 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. + +//! Delta-specific helpers core's `OpStruct::DeltaScan` dispatcher arm composes onto +//! the standard parquet datasource path: +//! +//! - [`build_delta_partitioned_files`] -- convert a `DeltaScanTask` list into a +//! `Vec` (Delta's add.path is already absolute on the driver; +//! partition values arrive as strings, parsed here) +//! - [`parse_delta_partition_scalar`] -- string -> `ScalarValue` with Delta's TZ +//! semantics and the DATE -> TIMESTAMP_NTZ widening fallback +//! - [`ColumnMappingFilterRewriter`] -- rewrites pushed-down data filters from +//! logical to physical column names when column mapping is active +//! +//! All take pure DataFusion / arrow types so this crate stays free of any +//! datafusion-comet dependency (no cycle: core can call us, we can't call core). + +use std::collections::HashMap; +use std::sync::Arc; + +use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; +use datafusion::common::tree_node::{Transformed, TreeNodeRewriter}; +use datafusion::common::ScalarValue; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::expressions::Column; +use object_store::path::Path; +use url::Url; + +use crate::proto::DeltaScanTask; + +/// Pre-parsed session timezone, computed once per scan and reused across every partition +/// value parse. Avoids the per-row `chrono_tz::Tz::from_str` lookup +/// `parse_delta_partition_scalar` would otherwise do for every TIMESTAMP partition value. +pub enum SessionTimezone { + Tz(chrono_tz::Tz), + Offset(chrono::FixedOffset), + /// `session_tz` didn't parse as either a named TZ or a fixed offset. We defer the + /// "invalid session TZ" error to the per-row parse path so callers that don't have any + /// TIMESTAMP partitions never see it. + Invalid, +} + +impl SessionTimezone { + pub fn parse(session_tz: &str) -> Self { + if let Ok(tz) = session_tz.parse::() { + return Self::Tz(tz); + } + if let Some(off) = parse_fixed_offset(session_tz) { + return Self::Offset(off); + } + Self::Invalid + } +} + +fn parse_fixed_offset(s: &str) -> Option { + let trimmed = s.trim(); + let body = trimmed + .strip_prefix("GMT") + .or_else(|| trimmed.strip_prefix("UTC")) + .unwrap_or(trimmed); + if body.is_empty() || body.eq_ignore_ascii_case("Z") { + return Some(chrono::FixedOffset::east_opt(0).unwrap()); + } + let (sign, rest) = match body.chars().next()? { + '+' => (1, &body[1..]), + '-' => (-1, &body[1..]), + _ => return None, + }; + let secs = if rest.contains(':') { + let mut parts = rest.splitn(2, ':'); + let h: i32 = parts.next()?.parse().ok()?; + let m: i32 = parts.next()?.parse().ok()?; + h * 3600 + m * 60 + } else if rest.len() == 4 { + let h: i32 = rest[..2].parse().ok()?; + let m: i32 = rest[2..].parse().ok()?; + h * 3600 + m * 60 + } else { + let h: i32 = rest.parse().ok()?; + h * 3600 + }; + chrono::FixedOffset::east_opt(sign * secs) +} + +/// Convert `DeltaScanTask`s into DataFusion `PartitionedFile`s. Delta's add.path is +/// already an absolute URL once kernel has resolved it on the driver. +pub fn build_delta_partitioned_files( + tasks: &[DeltaScanTask], + partition_schema: &Schema, + session_tz: &str, +) -> Result, String> { + let parsed_tz = SessionTimezone::parse(session_tz); + let mut files = Vec::with_capacity(tasks.len()); + // Reused scratch map for per-task partition-value lookup. Without it, the inner + // `partition_schema.fields()` loop walks `task.partition_values` with `.iter().find()` + // for every field -- O(width × values) per task. With it, build the map once per task + // and do O(1) gets. `clear()` keeps the allocation across tasks. + let mut partition_values_by_name: std::collections::HashMap<&str, &str> = + std::collections::HashMap::new(); + for task in tasks { + let url = Url::parse(task.file_path.as_ref()) + .map_err(|e| format!("Invalid Delta file URL: {e}"))?; + let path = Path::from_url_path(url.path()) + .map_err(|e| format!("from_url_path: {e}"))?; + + let mut partitioned_file = match (task.byte_range_start, task.byte_range_end) { + (Some(start), Some(end)) => PartitionedFile::new_with_range( + String::new(), + task.file_size, + start as i64, + end as i64, + ), + _ => PartitionedFile::new(String::new(), task.file_size), + }; + partitioned_file.object_meta.location = path; + + let mut partition_values: Vec = + Vec::with_capacity(partition_schema.fields().len()); + partition_values_by_name.clear(); + for pv in &task.partition_values { + if let Some(v) = pv.value.as_deref() { + partition_values_by_name.insert(pv.name.as_str(), v); + } + } + for field in partition_schema.fields() { + let scalar = match partition_values_by_name.get(field.name().as_str()).copied() { + Some(s) => parse_delta_partition_scalar(s, field.data_type(), &parsed_tz, session_tz) + .map_err(|e| { + format!( + "Failed to parse Delta partition value for column '{}': {e}", + field.name() + ) + })?, + None => ScalarValue::try_from(field.data_type()).map_err(|e| { + format!( + "Failed to build null partition value for column '{}': {e}", + field.name() + ) + })?, + }; + partition_values.push(scalar); + } + partitioned_file.partition_values = partition_values; + files.push(partitioned_file); + } + Ok(files) +} + +/// Parse a Delta partition value string into a `ScalarValue`. Honours session TZ for +/// TIMESTAMP columns. Delta writes TIMESTAMP partition values in the JVM default TZ +/// (`yyyy-MM-dd HH:mm:ss[.S]`); DataFusion's default parser interprets them as UTC +/// which would be off by the session offset. +/// +/// Includes the DATE -> TIMESTAMP_NTZ widening fallback: Delta's TypeWidening leaves +/// the original "YYYY-MM-DD" partition strings in place when the column changes from +/// DATE to TIMESTAMP_NTZ, so we accept the date-only form by promoting to midnight +/// (matches Spark's `cast(DATE as TIMESTAMP)` semantics). +pub fn parse_delta_partition_scalar( + s: &str, + dt: &DataType, + parsed_tz: &SessionTimezone, + session_tz: &str, +) -> Result { + match dt { + DataType::Timestamp(unit, tz_opt) => { + use chrono::{DateTime, NaiveDateTime, TimeZone}; + if tz_opt.is_none() { + let naive = NaiveDateTime::parse_from_str(s, "%Y-%m-%d %H:%M:%S%.f") + .or_else(|_| NaiveDateTime::parse_from_str(s, "%Y-%m-%d %H:%M:%S")) + .or_else(|_| { + chrono::NaiveDate::parse_from_str(s, "%Y-%m-%d") + .map(|d| d.and_hms_opt(0, 0, 0).unwrap()) + }) + .map_err(|e| format!("cannot parse TIMESTAMP_NTZ '{s}': {e}"))?; + let micros = chrono::Utc.from_utc_datetime(&naive).timestamp_micros(); + return Ok(match unit { + datafusion::arrow::datatypes::TimeUnit::Microsecond => { + ScalarValue::TimestampMicrosecond(Some(micros), None) + } + datafusion::arrow::datatypes::TimeUnit::Millisecond => { + ScalarValue::TimestampMillisecond(Some(micros / 1_000), None) + } + datafusion::arrow::datatypes::TimeUnit::Nanosecond => { + ScalarValue::TimestampNanosecond(Some(micros.saturating_mul(1_000)), None) + } + datafusion::arrow::datatypes::TimeUnit::Second => { + ScalarValue::TimestampSecond(Some(micros / 1_000_000), None) + } + }); + } + let micros = if let Ok(dt_with_tz) = DateTime::parse_from_rfc3339(s) { + dt_with_tz.timestamp_micros() + } else if let Ok(dt_with_tz) = + DateTime::parse_from_str(s, "%Y-%m-%d %H:%M:%S%.f %z") + .or_else(|_| DateTime::parse_from_str(s, "%Y-%m-%d %H:%M:%S %z")) + { + dt_with_tz.timestamp_micros() + } else { + let naive = NaiveDateTime::parse_from_str(s, "%Y-%m-%d %H:%M:%S%.f") + .or_else(|_| NaiveDateTime::parse_from_str(s, "%Y-%m-%d %H:%M:%S")) + .or_else(|_| { + chrono::NaiveDate::parse_from_str(s, "%Y-%m-%d") + .map(|d| d.and_hms_opt(0, 0, 0).unwrap()) + }) + .map_err(|e| format!("cannot parse timestamp '{s}': {e}"))?; + use chrono::LocalResult; + match parsed_tz { + SessionTimezone::Tz(tz) => match tz.from_local_datetime(&naive) { + LocalResult::Single(dt) => dt.timestamp_micros(), + LocalResult::Ambiguous(earlier, _later) => earlier.timestamp_micros(), + LocalResult::None => { + chrono::Utc.from_utc_datetime(&naive).timestamp_micros() + } + }, + SessionTimezone::Offset(off) => match off.from_local_datetime(&naive) { + LocalResult::Single(dt) => dt.timestamp_micros(), + _ => chrono::Utc.from_utc_datetime(&naive).timestamp_micros(), + }, + SessionTimezone::Invalid => { + return Err(format!("invalid session TZ '{session_tz}'")); + } + } + }; + match unit { + datafusion::arrow::datatypes::TimeUnit::Microsecond => Ok( + ScalarValue::TimestampMicrosecond(Some(micros), tz_opt.clone()), + ), + datafusion::arrow::datatypes::TimeUnit::Millisecond => Ok( + ScalarValue::TimestampMillisecond(Some(micros / 1000), tz_opt.clone()), + ), + datafusion::arrow::datatypes::TimeUnit::Nanosecond => Ok( + ScalarValue::TimestampNanosecond(Some(micros * 1000), tz_opt.clone()), + ), + datafusion::arrow::datatypes::TimeUnit::Second => Ok( + ScalarValue::TimestampSecond(Some(micros / 1_000_000), tz_opt.clone()), + ), + } + } + _ => ScalarValue::try_from_string(s.to_string(), dt).map_err(|e| format!("{e}")), + } +} + +/// Rewrites Column references in a PhysicalExpr from logical names/indices (in +/// required_schema) to physical names/indices (in data_schema). Used when Delta column +/// mapping is active so pushed-down data filters match the DataSourceExec's physical +/// names. +pub struct ColumnMappingFilterRewriter<'a> { + pub logical_to_physical: &'a HashMap, + pub data_schema: &'a SchemaRef, +} + +impl TreeNodeRewriter for ColumnMappingFilterRewriter<'_> { + type Node = Arc; + + fn f_down( + &mut self, + node: Self::Node, + ) -> datafusion::common::Result> { + if let Some(column) = node.as_any().downcast_ref::() { + if let Some(physical_name) = self.logical_to_physical.get(column.name()) { + if let Some(idx) = self + .data_schema + .fields() + .iter() + .position(|f| f.name() == physical_name) + { + return Ok(Transformed::yes(Arc::new(Column::new(physical_name, idx)))); + } + log::warn!( + "Column mapping: physical name '{}' for logical '{}' not found in \ + data_schema; filter may fail at execution time", + physical_name, + column.name() + ); + } + Ok(Transformed::no(node)) + } else { + Ok(Transformed::no(node)) + } + } +} diff --git a/contrib/delta/native/src/predicate.rs b/contrib/delta/native/src/predicate.rs new file mode 100644 index 0000000000..afe2ea7384 --- /dev/null +++ b/contrib/delta/native/src/predicate.rs @@ -0,0 +1,232 @@ +// 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. + +//! Translates Catalyst-proto `Expr` to delta-kernel `Predicate` for +//! stats-based file pruning. +//! +//! Supported operators: =, !=, <, <=, >, >=, AND, OR, NOT, IS NULL, +//! IS NOT NULL, IN (including NOT IN). Cast wrappers are unwrapped +//! (kernel stats don't need type coercion). Anything else becomes +//! `Predicate::unknown()`, which disables data skipping for that +//! subtree but is never incorrect. + +use datafusion_comet_proto::spark_expression::{self, expr::ExprStruct, literal, Expr}; +use delta_kernel::expressions::{ArrayData, BinaryPredicateOp, Expression, Predicate, Scalar}; +use delta_kernel::schema::{ArrayType, DataType}; + +/// Translate with column name resolution for BoundReferences. +pub fn catalyst_to_kernel_predicate_with_names(expr: &Expr, column_names: &[String]) -> Predicate { + translate_predicate(expr, column_names) +} + +/// Try to translate a Catalyst-proto `Expr` into a kernel `Predicate` +/// (without column name resolution — BoundReferences become Unknown). +pub fn catalyst_to_kernel_predicate(expr: &Expr) -> Predicate { + translate_predicate(expr, &[]) +} + +fn translate_predicate(expr: &Expr, names: &[String]) -> Predicate { + let to_expr = |e: &Expr| catalyst_to_kernel_expression_with_names(e, names); + match expr.expr_struct.as_ref() { + Some(ExprStruct::IsNull(unary)) => match unary.child.as_deref() { + Some(child) => Predicate::is_null(to_expr(child)), + None => Predicate::unknown("missing_child"), + }, + Some(ExprStruct::IsNotNull(unary)) => match unary.child.as_deref() { + Some(child) => Predicate::is_not_null(to_expr(child)), + None => Predicate::unknown("missing_child"), + }, + Some(ExprStruct::Eq(binary)) => binary_pred_n( + Predicate::eq, + binary.left.as_deref(), + binary.right.as_deref(), + names, + ), + Some(ExprStruct::Neq(binary)) => binary_pred_n( + Predicate::ne, + binary.left.as_deref(), + binary.right.as_deref(), + names, + ), + Some(ExprStruct::Lt(binary)) => binary_pred_n( + Predicate::lt, + binary.left.as_deref(), + binary.right.as_deref(), + names, + ), + Some(ExprStruct::LtEq(binary)) => binary_pred_n( + Predicate::le, + binary.left.as_deref(), + binary.right.as_deref(), + names, + ), + Some(ExprStruct::Gt(binary)) => binary_pred_n( + Predicate::gt, + binary.left.as_deref(), + binary.right.as_deref(), + names, + ), + Some(ExprStruct::GtEq(binary)) => binary_pred_n( + Predicate::ge, + binary.left.as_deref(), + binary.right.as_deref(), + names, + ), + Some(ExprStruct::And(binary)) => match (binary.left.as_deref(), binary.right.as_deref()) { + (Some(l), Some(r)) => { + Predicate::and(translate_predicate(l, names), translate_predicate(r, names)) + } + _ => Predicate::unknown("and_missing_child"), + }, + Some(ExprStruct::Or(binary)) => match (binary.left.as_deref(), binary.right.as_deref()) { + (Some(l), Some(r)) => { + Predicate::or(translate_predicate(l, names), translate_predicate(r, names)) + } + _ => Predicate::unknown("or_missing_child"), + }, + Some(ExprStruct::Not(unary)) => match unary.child.as_deref() { + Some(child) => Predicate::not(translate_predicate(child, names)), + None => Predicate::unknown("not_missing_child"), + }, + Some(ExprStruct::In(in_expr)) => translate_in(in_expr, names), + // Unwrap Cast: kernel stats don't need type coercion, pass child through + Some(ExprStruct::Cast(cast)) => match cast.child.as_deref() { + Some(child) => translate_predicate(child, names), + None => Predicate::unknown("cast_missing_child"), + }, + _ => Predicate::unknown("unsupported_catalyst_expr"), + } +} + +fn translate_in(in_expr: &spark_expression::In, names: &[String]) -> Predicate { + let value = match in_expr.in_value.as_deref() { + Some(v) => catalyst_to_kernel_expression_with_names(v, names), + None => return Predicate::unknown("in_missing_value"), + }; + + let scalars: Vec = in_expr + .lists + .iter() + .filter_map(catalyst_literal_to_scalar) + .collect(); + + if scalars.is_empty() { + return Predicate::unknown("in_no_literal_values"); + } + + let kernel_type = scalar_to_kernel_type(&scalars[0]); + let array_data = match ArrayData::try_new(ArrayType::new(kernel_type, true), scalars) { + Ok(ad) => ad, + Err(_) => return Predicate::unknown("in_array_type_mismatch"), + }; + let array = Expression::literal(Scalar::Array(array_data)); + + let pred = Predicate::binary(BinaryPredicateOp::In, value, array); + if in_expr.negated { + Predicate::not(pred) + } else { + pred + } +} + +fn scalar_to_kernel_type(s: &Scalar) -> DataType { + match s { + Scalar::Boolean(_) => DataType::BOOLEAN, + Scalar::Byte(_) => DataType::BYTE, + Scalar::Short(_) => DataType::SHORT, + Scalar::Integer(_) => DataType::INTEGER, + Scalar::Long(_) => DataType::LONG, + Scalar::Float(_) => DataType::FLOAT, + Scalar::Double(_) => DataType::DOUBLE, + Scalar::String(_) => DataType::STRING, + _ => DataType::STRING, + } +} + +fn catalyst_literal_to_scalar(expr: &Expr) -> Option { + match expr.expr_struct.as_ref() { + Some(ExprStruct::Literal(lit)) => match &lit.value { + Some(literal::Value::BoolVal(b)) => Some(Scalar::Boolean(*b)), + Some(literal::Value::ByteVal(v)) => Some(Scalar::Byte(*v as i8)), + Some(literal::Value::ShortVal(v)) => Some(Scalar::Short(*v as i16)), + Some(literal::Value::IntVal(v)) => Some(Scalar::Integer(*v)), + Some(literal::Value::LongVal(v)) => Some(Scalar::Long(*v)), + Some(literal::Value::FloatVal(v)) => Some(Scalar::Float(*v)), + Some(literal::Value::DoubleVal(v)) => Some(Scalar::Double(*v)), + Some(literal::Value::StringVal(s)) => Some(Scalar::String(s.clone())), + _ => None, + }, + _ => None, + } +} + +fn binary_pred_n( + builder: impl Fn(Expression, Expression) -> Predicate, + left: Option<&Expr>, + right: Option<&Expr>, + names: &[String], +) -> Predicate { + match (left, right) { + (Some(l), Some(r)) => builder( + catalyst_to_kernel_expression_with_names(l, names), + catalyst_to_kernel_expression_with_names(r, names), + ), + _ => Predicate::unknown("binary_missing_child"), + } +} + +/// Translate a Catalyst-proto `Expr` into a kernel value `Expression`. +/// +/// `column_names` maps BoundReference indices to column names. When +/// empty, BoundReferences become unknown expressions (disabling file +/// skipping for that sub-expression but never producing wrong results). +pub fn catalyst_to_kernel_expression_with_names( + expr: &Expr, + column_names: &[String], +) -> Expression { + match expr.expr_struct.as_ref() { + Some(ExprStruct::Bound(bound)) => { + let idx = bound.index as usize; + if idx < column_names.len() { + Expression::column([column_names[idx].as_str()]) + } else { + Expression::unknown("bound_ref_out_of_range") + } + } + Some(ExprStruct::Literal(lit)) => catalyst_literal_to_kernel(lit), + // Unwrap Cast: pass child expression through for kernel stats evaluation + Some(ExprStruct::Cast(cast)) => match cast.child.as_deref() { + Some(child) => catalyst_to_kernel_expression_with_names(child, column_names), + None => Expression::unknown("cast_missing_child"), + }, + _ => Expression::unknown("unsupported_expr_operand"), + } +} + +fn catalyst_literal_to_kernel(lit: &spark_expression::Literal) -> Expression { + match &lit.value { + Some(literal::Value::BoolVal(b)) => Expression::literal(*b), + Some(literal::Value::ByteVal(v)) => Expression::literal(*v), + Some(literal::Value::ShortVal(v)) => Expression::literal(*v), + Some(literal::Value::IntVal(v)) => Expression::literal(*v), + Some(literal::Value::LongVal(v)) => Expression::literal(*v), + Some(literal::Value::FloatVal(v)) => Expression::literal(*v), + Some(literal::Value::DoubleVal(v)) => Expression::literal(*v), + Some(literal::Value::StringVal(s)) => Expression::literal(s.as_str()), + _ => Expression::null_literal(DataType::STRING), + } +} diff --git a/contrib/delta/native/src/scan.rs b/contrib/delta/native/src/scan.rs new file mode 100644 index 0000000000..0decb8d900 --- /dev/null +++ b/contrib/delta/native/src/scan.rs @@ -0,0 +1,400 @@ +// 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. + +//! Delta log replay: given a table URL, return the list of active parquet +//! files with partition values, record-count stats, and deletion-vector +//! flags. +//! +//! Ported from tantivy4java's `delta_reader/scan.rs`. The API is the +//! smallest possible surface that still proves end-to-end kernel +//! integration: `Snapshot::builder_for(url)` → `scan_builder().build()` → +//! `scan_metadata(&engine)` → `visit_scan_files(...)`. +//! +//! **Critical gotcha** preserved from the reference implementation: kernel +//! internally does `table_root.join("_delta_log/")`, and `Url::join` will +//! *replace* the last path segment if the base URL does not end in `/`. So +//! `normalize_url` always appends a trailing slash. + +use std::collections::HashMap; +use std::sync::Arc; +use url::Url; + +use delta_kernel::snapshot::Snapshot; + +use super::engine::{get_or_create_engine, DeltaStorageConfig}; +use super::error::{DeltaError, DeltaResult}; + +/// Metadata for a single active parquet file in a Delta table. +/// +/// Plain Rust types only — no arrow / parquet / object_store types. This is +/// the boundary at which kernel's isolated dep subtree meets the rest of +/// Comet. +#[derive(Debug, Clone)] +pub struct DeltaFileEntry { + /// Parquet file path, relative to the table root. + pub path: String, + /// File size in bytes. + pub size: i64, + /// Last-modified time as epoch millis. + pub modification_time: i64, + /// Record count from log stats, if known. + pub num_records: Option, + /// Partition column → value mapping from the add action. + pub partition_values: HashMap, + /// Deleted row indexes materialized from the file's deletion vector by + /// kernel on the driver. Empty vector means the file has no DV in use. + /// Sorted ascending; indexes are 0-based into the file's physical parquet + /// row space, matching `DvInfo::get_row_indexes` semantics. + pub deleted_row_indexes: Vec, +} + +impl DeltaFileEntry { + /// True if this entry has a deletion vector in use. + pub fn has_deletion_vector(&self) -> bool { + !self.deleted_row_indexes.is_empty() + } +} + +/// Result of planning a Delta scan: the active file list plus the pinned +/// snapshot version plus a list of reader features that Comet's native path +/// doesn't yet handle. The Scala side uses the feature list to decide +/// whether to fall back to Spark's vanilla Delta reader. +#[derive(Debug, Clone)] +pub struct DeltaScanPlan { + pub entries: Vec, + pub version: u64, + pub unsupported_features: Vec, + /// Logical→physical column name mapping for column-mapped tables. + /// Empty when column_mapping_mode is None. + pub column_mappings: Vec<(String, String)>, +} + +/// List every active parquet file in a Delta table at the given version. +/// +/// Returns `(entries, actual_version)` where `actual_version` is the +/// snapshot version that was actually read — equal to `version` when +/// specified, or the latest version otherwise. +/// +/// Thin wrapper around [`plan_delta_scan`] that drops the feature list. +/// New code should call `plan_delta_scan` directly so it can honor the +/// unsupported-feature gate. +pub fn list_delta_files( + url_str: &str, + config: &DeltaStorageConfig, + version: Option, +) -> DeltaResult<(Vec, u64)> { + let plan = plan_delta_scan(url_str, config, version)?; + Ok((plan.entries, plan.version)) +} + +/// Plan a Delta scan against the given URL + optional snapshot version. +/// +/// This is the full-fat variant of [`list_delta_files`]: it also reports +/// which reader features are *in use* for this snapshot and NOT yet +/// supported by Comet's native path. +/// +/// Feature detection blends two signals: +/// 1. [`delta_kernel::snapshot::Snapshot::table_properties`] — the +/// protocol-level flags (`column_mapping_mode`, `enable_type_widening`, +/// `enable_row_tracking`). +/// 2. The per-file `ScanFile::dv_info.has_vector()` flag — set to true +/// only when the specific file actually has a deletion vector attached. +/// This is tighter than the `enable_deletion_vectors` table property +/// because a DV-enabled table with no deletes yet is still safe for +/// Comet to read natively. +pub fn plan_delta_scan( + url_str: &str, + config: &DeltaStorageConfig, + version: Option, +) -> DeltaResult { + plan_delta_scan_with_predicate(url_str, config, version, None) +} + +pub fn plan_delta_scan_with_predicate( + url_str: &str, + config: &DeltaStorageConfig, + version: Option, + kernel_predicate: Option, +) -> DeltaResult { + let url = normalize_url(url_str)?; + let engine = get_or_create_engine(&url, config)?; + + let snapshot = { + let mut builder = Snapshot::builder_for(url); + if let Some(v) = version { + builder = builder.at_version(v); + } + builder.build(&*engine)? + }; + let actual_version = snapshot.version(); + + // Protocol-level feature gate. Collect the names of features we don't + // yet handle so the Scala side can decide to fall back. Note that we + // explicitly do NOT treat the following as fallback-worthy: + // - `change_data_feed`: only affects CDF queries, not regular reads + // - `in_commit_timestamps`: regular reads work fine + // - `iceberg_compat_v1/v2`: doesn't change Delta read correctness + // - `append_only`: write-side constraint, reads are unaffected + let unsupported_features: Vec = Vec::new(); + let props = snapshot.table_properties(); + // columnMapping is now handled by Phase 4 — no longer a fallback trigger. + // typeWidening: DataFusion's parquet schema adapter handles widening reads + // (parquet stores the file's original type; the adapter casts to the table's + // current widened type at read time). Removed from the gate; verified by + // TypeWidening{TableFeature,Metadata,...}Suite in the Delta regression. + // rowTracking: tables with `enable_row_tracking=true` are scannable + // natively. Queries that explicitly select `_metadata.row_id` / + // `_metadata.row_commit_version` are handled in CometScanRule's + // `applyRowTrackingRewrite` (it rewrites the scan to read the materialized + // physical column, or declines when no materialized name is available). + // No need to gate the whole table's scan path here. + + // Phase 4: extract logical→physical column name mapping from schema metadata. + // For column_mapping_mode = id or name, each StructField carries a + // `delta.columnMapping.physicalName` metadata entry that tells us what the + // parquet file's column name actually is. + let column_mappings: Vec<(String, String)> = if props.column_mapping_mode.is_some() { + snapshot + .schema() + .fields() + .filter_map(|field| { + use delta_kernel::schema::{ColumnMetadataKey, MetadataValue}; + field + .metadata + .get(ColumnMetadataKey::ColumnMappingPhysicalName.as_ref()) + .and_then(|v| match v { + MetadataValue::String(phys) => Some((field.name().clone(), phys.clone())), + _ => None, + }) + }) + .collect() + } else { + Vec::new() + }; + + // `Snapshot::build()` returns `Arc`, and `scan_builder` consumes + // it. Clone the Arc so we can still reach `table_root()` after building + // the scan — we need the URL to materialize DVs below. + let snapshot_arc: Arc<_> = snapshot; + let table_root_url = snapshot_arc.table_root().clone(); + let mut scan_builder = Arc::clone(&snapshot_arc).scan_builder(); + if let Some(pred) = kernel_predicate { + scan_builder = scan_builder.with_predicate(Arc::new(pred)); + } + let scan = scan_builder.build()?; + + // Temporary collection that keeps the raw kernel `DvInfo` alongside the + // rest of the metadata. We need the `DvInfo` to materialize the deleted + // row indexes below; it doesn't escape this function. + struct RawEntry { + path: String, + size: i64, + modification_time: i64, + num_records: Option, + partition_values: HashMap, + dv_info: delta_kernel::scan::state::DvInfo, + } + + let mut raw: Vec = Vec::new(); + let scan_metadata = scan.scan_metadata(&*engine)?; + + for meta_result in scan_metadata { + let meta: delta_kernel::scan::ScanMetadata = meta_result?; + raw = meta.visit_scan_files( + raw, + |acc: &mut Vec, scan_file: delta_kernel::scan::state::ScanFile| { + let num_records = scan_file.stats.as_ref().map(|s| s.num_records); + acc.push(RawEntry { + path: scan_file.path, + size: scan_file.size, + modification_time: scan_file.modification_time, + num_records, + partition_values: scan_file.partition_values, + dv_info: scan_file.dv_info, + }); + }, + )?; + } + + // For each file that has a DV attached, ask kernel to materialize the + // deleted row indexes. Kernel handles inline bitmaps, on-disk DV files, + // and the various storage-type variants transparently. This runs on the + // driver (same process that's building the scan plan), so we only pay + // the DV-fetch latency once per query. + // + // Note: for very large tables (millions of files), this collects all + // entries into memory before returning. Consider streaming/chunked + // processing if driver OOM becomes an issue at extreme scale. + let mut entries: Vec = Vec::with_capacity(raw.len()); + for r in raw { + let deleted_row_indexes = if r.dv_info.has_vector() { + r.dv_info + .get_row_indexes(&*engine, &table_root_url)? + .ok_or_else(|| { + DeltaError::Internal(format!( + "DV has_vector() true but get_row_indexes() returned None for {}", + r.path + )) + })? + } else { + Vec::new() + }; + entries.push(DeltaFileEntry { + path: r.path, + size: r.size, + modification_time: r.modification_time, + num_records: r.num_records, + partition_values: r.partition_values, + deleted_row_indexes, + }); + } + + Ok(DeltaScanPlan { + entries, + version: actual_version, + unsupported_features, + column_mappings, + }) +} + +/// Normalize a table URL so kernel's `table_root.join("_delta_log/")` +/// appends rather than replaces. Bare paths become `file://` URLs. +/// +/// Accepts three shapes: +/// 1. `s3://`, `s3a://`, `az://`, `azure://`, `abfs://`, `abfss://`, +/// `file://` — already-formed URLs, parsed directly. +/// 2. `file:/Users/...` — Hadoop's `Path.toUri.toString` output, which +/// uses a *single* slash and is NOT a valid `Url::parse` input. We +/// rewrite this to `file://` before parsing. +/// 3. Bare local paths — canonicalized and turned into `file://` via +/// `Url::from_directory_path`. +pub(crate) fn normalize_url(url_str: &str) -> DeltaResult { + // Hadoop's java.net.URI.toString emits `file:/path/to/t` (one slash) + // for local files. Rewrite into the `file:///path` form that + // `Url::parse` understands. + if url_str.starts_with("file:/") && !url_str.starts_with("file://") { + let rewritten = format!("file://{}", &url_str["file:".len()..]); + let mut url = Url::parse(&rewritten).map_err(|e| DeltaError::InvalidUrl { + url: url_str.to_string(), + source: e, + })?; + ensure_trailing_slash(&mut url); + return Ok(url); + } + + if url_str.starts_with("s3://") + || url_str.starts_with("s3a://") + || url_str.starts_with("az://") + || url_str.starts_with("azure://") + || url_str.starts_with("abfs://") + || url_str.starts_with("abfss://") + || url_str.starts_with("file://") + { + let mut url = Url::parse(url_str).map_err(|e| DeltaError::InvalidUrl { + url: url_str.to_string(), + source: e, + })?; + ensure_trailing_slash(&mut url); + Ok(url) + } else { + let abs_path = std::path::Path::new(url_str).canonicalize().map_err(|e| { + DeltaError::PathResolution { + path: url_str.to_string(), + source: e, + } + })?; + Url::from_directory_path(&abs_path).map_err(|_| DeltaError::PathToUrl { + path: abs_path.display().to_string(), + }) + } +} + +fn ensure_trailing_slash(url: &mut Url) { + let path = url.path().to_string(); + if !path.ends_with('/') { + url.set_path(&format!("{path}/")); + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_normalize_url_trailing_slash() { + let url = normalize_url("file:///tmp/my_table").unwrap(); + assert!(url.path().ends_with('/'), "URL should end with /: {url}"); + assert_eq!(url.as_str(), "file:///tmp/my_table/"); + + let url = normalize_url("file:///tmp/my_table/").unwrap(); + assert_eq!(url.as_str(), "file:///tmp/my_table/"); + + let url = normalize_url("s3://bucket/path/to/table").unwrap(); + assert!(url.path().ends_with('/'), "URL should end with /: {url}"); + } + + #[test] + fn test_normalize_url_hadoop_single_slash_form() { + // Hadoop's Path.toUri.toString produces `file:/path` (single slash), + // not `file:///path`. Must be normalized to a Url::parse-able form. + let url = normalize_url("file:/Users/alice/tmp/t").unwrap(); + assert_eq!(url.as_str(), "file:///Users/alice/tmp/t/"); + + let url = normalize_url("file:/tmp/t/").unwrap(); + assert_eq!(url.as_str(), "file:///tmp/t/"); + } + + #[test] + fn test_normalize_url_join_behavior() { + // The critical invariant: joining `_delta_log/` onto a normalized + // URL must *append*, not replace the last segment. + let url = normalize_url("file:///tmp/my_table").unwrap(); + let log_url = url.join("_delta_log/").unwrap(); + assert_eq!(log_url.as_str(), "file:///tmp/my_table/_delta_log/"); + } + + #[test] + fn test_list_delta_files_local() { + // Hand-build a minimal Delta table in a tempdir: one protocol action, + // one metadata action, one add action. No Parquet data needed — + // we're exercising the log-replay path only. + let tmp = tempfile::tempdir().unwrap(); + let table_dir = tmp.path().join("test_delta"); + let delta_log = table_dir.join("_delta_log"); + std::fs::create_dir_all(&delta_log).unwrap(); + + let commit0 = [ + r#"{"protocol":{"minReaderVersion":1,"minWriterVersion":2}}"#, + r#"{"metaData":{"id":"test-id","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":[],"configuration":{},"createdTime":1700000000000}}"#, + r#"{"add":{"path":"part-00000.parquet","partitionValues":{},"size":5000,"modificationTime":1700000000000,"dataChange":true,"stats":"{\"numRecords\":50}"}}"#, + ] + .join("\n"); + std::fs::write(delta_log.join("00000000000000000000.json"), &commit0).unwrap(); + std::fs::write(table_dir.join("part-00000.parquet"), [0u8]).unwrap(); + + let config = DeltaStorageConfig::default(); + let (entries, version) = + list_delta_files(table_dir.to_str().unwrap(), &config, None).unwrap(); + + assert_eq!(version, 0); + assert_eq!(entries.len(), 1); + assert_eq!(entries[0].path, "part-00000.parquet"); + assert_eq!(entries[0].size, 5000); + assert_eq!(entries[0].num_records, Some(50)); + assert!(!entries[0].has_deletion_vector()); + } +} diff --git a/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/CometDeltaNativeScan.scala b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/CometDeltaNativeScan.scala new file mode 100644 index 0000000000..a6172f8f99 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/CometDeltaNativeScan.scala @@ -0,0 +1,966 @@ +/* + * 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. + */ + +package org.apache.comet.contrib.delta + +import java.util.Locale + +import scala.collection.mutable.ListBuffer +import scala.jdk.CollectionConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{And, BoundReference, Expression, InterpretedPredicate} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.comet.{CometDeltaNativeScanExec, CometNativeExec, CometScanExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import org.apache.comet.{CometConf, ConfigEntry} +// Contrib-private Java proto types generated by protoc-jar-maven-plugin from +// contrib/delta/native/src/proto/delta_operator.proto. The proto declares +// `option java_package = "org.apache.comet.contrib.delta.proto"` so the generated +// outer class lands under a Comet-prefixed Java package. +// Typed Delta proto messages now live in core's operator.proto (alongside IcebergScan) +// instead of a contrib-private proto package. +import org.apache.comet.serde.OperatorOuterClass.{DeltaScan, DeltaScanCommon, DeltaScanTaskList} +import org.apache.comet.objectstore.NativeConfig +import org.apache.comet.serde.{CometOperatorSerde, Compatible, ExprOuterClass, OperatorOuterClass, SupportLevel} +import org.apache.comet.serde.ExprOuterClass.Expr +import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.QueryPlanSerde.exprToProto +import org.apache.comet.serde.operator.schema2Proto + +/** + * Validation and serde logic for the native Delta Lake scan. + * + * `convert()` calls `Native.planDeltaScan` to enumerate files via `delta-kernel-rs`, builds the + * `DeltaScanCommon` proto with schemas/filters/options, applies static partition pruning, and + * stashes the task list in a ThreadLocal. `createExec()` retrieves it and builds a + * `CometDeltaNativeScanExec` with split-mode serialization: common data serialized once at + * planning time, per-partition task lists materialized lazily at execution time. DPP filters are + * applied at execution time in the exec's `serializedPartitionData`. + */ +/** + * Delta-scan serde + exec factory. Extends Comet's core `CometOperatorSerde` trait so + * the existing convertToComet path in `CometExecRule` invokes it just like the + * built-in handlers (CometNativeScan, CometIcebergNativeScan, ...). What is NOT here + * is any *extension/discovery* SPI -- core's `CometExecRule` resolves this object via + * `DeltaIntegration.scanHandler` (one reflective class lookup, no ServiceLoader, no + * registry). The wire format is the typed `OpStruct::DeltaScan` variant. + */ +object CometDeltaNativeScan extends CometOperatorSerde[CometScanExec] with Logging { + + /** + * `kind` string for the `ContribOp` envelope this serde produces. The native side's + * `comet-contrib-delta` rlib registers `DeltaScanPlanner` under this same kind via + * `register_contrib_planner(DELTA_SCAN_KIND, ...)` in `contrib/delta/native/src/lib.rs`. Keep + * the two in sync. + */ + val DeltaScanKind: String = "delta-scan" + + /** + * `scanImpl` tag the contrib uses on `CometScanExec` markers produced by + * `DeltaScanRuleExtension.transformV1`. Contrib-local constant (not in core's CometConf), + * declared as `nativeParquetScanImpls` in `DeltaOperatorSerdeExtension` so + * `CometScanExec.supportedDataFilters` applies the right exclusions, and matched in + * `DeltaOperatorSerdeExtension.matchOperator` to route through this serde. + */ + val ScanImpl: String = "native_delta_compat" + + /** Private lazy handle to the native library - one instance per JVM. */ + private lazy val nativeLib = new org.apache.comet.contrib.delta.Native() + + // Phase 5: stash the raw task-list bytes between convert() and createExec() + // so the exec can do per-partition splitting at execution time. Single-threaded + // during planning so a simple ThreadLocal is safe. + private val lastTaskListBytes = new ThreadLocal[Array[Byte]]() + + // #75 design A: when the surrounding plan references `input_file_name()` / + // `input_file_block_*`, CometScanRule tags the relation's options with + // `CometScanRule.NeedsInputFileNameOption`. We read it here to (a) skip + // byte-range splitting in splitTasks and (b) emit `oneTaskPerPartition = true` + // on the CometDeltaNativeScanExec so packTasks keeps each task in its own + // partition. With 1 task per partition, `CometExecRDD.compute` (via `InputFileBlockHolder.set`) + // sets InputFileBlockHolder to the correct path and Spark's JVM-side + // input_file_name() evaluation (no native serde exists) returns the right + // value. + /** Visible to `DeltaOperatorSerdeExtension.matchOperator` for routing decisions. */ + private[delta] def scanNeedsInputFileName(scan: CometScanExec): Boolean = + scan.relation.options + .get(DeltaConf.NeedsInputFileNameOption) + .contains("true") + + override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( + DeltaConf.COMET_DELTA_NATIVE_ENABLED) + + override def getSupportLevel(operator: CometScanExec): SupportLevel = Compatible() + + override def convert( + scan: CometScanExec, + builder: Operator.Builder, + childOp: OperatorOuterClass.Operator*): Option[OperatorOuterClass.Operator] = { + + // Resolve the table root via the HadoopFsRelation API - standard Spark, no spark-delta + // compile-time dep required. + val relation = scan.relation + val tableRoot = DeltaReflection.extractTableRoot(relation).getOrElse { + logWarning( + s"CometDeltaNativeScan: unable to extract table root from relation " + + s"${relation.location}; falling back to Spark's Delta reader.") + return None + } + + // Belt-and-suspenders DV-rewrite gate. The primary gate runs earlier in + // CometScanRule so the scan never becomes a CometScanExec in the first place. + // This is a defensive check in case a caller constructs a DV-rewritten + // CometScanExec by some other path. + if (scan.requiredSchema.fieldNames.contains(DeltaReflection.IsRowDeletedColumnName)) { + logWarning( + "CometDeltaNativeScan: DV-rewritten schema reached serde; this should have " + + "been caught in CometScanRule. Falling back.") + return None + } + + val ignoreMissingFiles = + SQLConf.get.ignoreMissingFiles || + relation.options.get("ignoremissingfiles").contains("true") + + // Cloud storage options, keyed identically to NativeScan. Kernel's DefaultEngine picks + // up aws_* / azure_* keys; anything else is ignored on the native side (for now). + // + // We key off the table root URI rather than `inputFiles.head` because data file names + // can contain characters that aren't URI-safe when Spark's test harness injects + // prefixes like `test%file%prefix-` (breaks `java.net.URI.create`). The table root + // string comes straight from `HadoopFsRelation.location.rootPaths.head.toUri` inside + // `DeltaReflection.extractTableRoot`, so it's already properly encoded. Storage options + // are bucket-level anyway - any file under the same root resolves to the same config. + val hadoopConf = + relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options) + val tableRootUri = java.net.URI.create(tableRoot) + val storageOptions: java.util.Map[String, String] = + NativeConfig.extractObjectStoreOptions(hadoopConf, tableRootUri).asJava + + // Honor Delta's time-travel options (versionAsOf / timestampAsOf) via the Delta- + // resolved snapshot version sitting on the FileIndex. Delta's analysis phase pins + // the exact snapshot before we ever see the plan, so by the time `CometScanExec` is + // built, `relation.location` is a `PreparedDeltaFileIndex` whose toString looks like + // `Delta[version=0, file:/...]`. We parse the version out via + // `DeltaReflection.extractSnapshotVersion` and pass it through to kernel. + // + // When no version can be extracted (non-Delta file index, parser miss, etc.) we pass + // -1 which asks kernel for the current latest snapshot. + val snapshotVersion: Long = + DeltaReflection.extractSnapshotVersion(relation).getOrElse(-1L) + + // Phase 2: serialize the data filters so kernel can apply stats-based file + // pruning during log replay. The same filters will also be pushed down into + // ParquetSource for row-group-level pruning - the two layers are additive. + // + // We combine all supported data filters into a single AND conjunction so + // kernel receives one predicate tree. BoundReferences carry the column INDEX + // into scan.output; the native side resolves indices to column names using + // the columnNames array we pass alongside. + val predicateBytes: Array[Byte] = { + val protoFilters = new ListBuffer[Expr]() + scan.supportedDataFilters.foreach { filter => + exprToProto(filter, scan.output) match { + case Some(proto) => protoFilters += proto + case _ => + } + } + if (protoFilters.isEmpty) { + Array.emptyByteArray + } else if (protoFilters.size == 1) { + protoFilters.head.toByteArray + } else { + // Combine filters into a balanced AND tree (depth O(log N) instead of + // O(N)). A linear left-deep fold overflows protobuf's default 100-level + // recursion limit for plans with many ANDed conditions (Delta data + // skipping predicates routinely build deep stats expressions: e.g. + // DataSkippingDeltaTests "remove redundant stats column references"). + // Both the JVM serde (CometNativeColumnarToRowExec re-parses the plan + // for explain output) and the Rust prost decoder are subject to that + // limit, so balancing the tree fixes both sides. + def balancedAnd(slice: IndexedSeq[Expr]): Expr = { + if (slice.size == 1) { + slice.head + } else { + val mid = slice.size / 2 + val left = balancedAnd(slice.slice(0, mid)) + val right = balancedAnd(slice.slice(mid, slice.size)) + val and = ExprOuterClass.BinaryExpr + .newBuilder() + .setLeft(left) + .setRight(right) + .build() + Expr.newBuilder().setAnd(and).build() + } + } + balancedAnd(protoFilters.toIndexedSeq).toByteArray + } + } + + // Column name list for resolving BoundReference indices to kernel column + // names. Must match the order of scan.output because exprToProto binds + // attribute references by position in that schema. + val columnNames: Array[String] = scan.output.map(_.name).toArray + + // --- 1. Get the active file list. --- + // + // Two code paths: + // (a) Pre-materialized FileIndex (`TahoeBatchFileIndex`, `CdcAddFileIndex`): + // Delta's streaming micro-batch reads AND MERGE / UPDATE / DELETE + // post-join rewrites both carry an exact `addFiles: Seq[AddFile]` on + // the FileIndex. Kernel log replay against the snapshot would return a + // DIFFERENT file set (the whole snapshot, or a version's deltas), which + // is a correctness hazard -- empty streaming batches, MERGE rewrites + // that see the whole table instead of only touched files. Build the + // DeltaScanTaskList proto directly from those AddFiles, skipping kernel. + // (b) Regular scan against a snapshot: call kernel for log replay as before. + val taskListBytes = + if (DeltaReflection.isBatchFileIndex(relation.location)) { + DeltaReflection.extractBatchAddFiles(relation.location) match { + case Some(addFiles) => + // Under column mapping, Delta stores partition values in AddFile keyed by the + // PHYSICAL column name. `relation.partitionSchema.fields[*].metadata` has had + // Delta's columnMapping metadata stripped by HadoopFsRelation, so look in the + // authoritative Snapshot schema (via reflection) and restrict to fields that + // appear in the relation's partition schema. + val partitionNames = relation.partitionSchema.fields.map(_.name).toSet + val snapshotFields = DeltaReflection + .extractSnapshotSchema(relation) + .map(_.fields) + .getOrElse(Array.empty[StructField]) + val physToLogical = snapshotFields.flatMap { f => + if (partitionNames.contains(f.name) && + f.metadata.contains(DeltaReflection.PhysicalNameMetadataKey)) { + Some(f.metadata.getString(DeltaReflection.PhysicalNameMetadataKey) -> f.name) + } else { + None + } + }.toMap + // DV materialization for the pre-materialised-index path (streaming + MERGE). + // For AddFiles that carry a DeletionVectorDescriptor, read the DV via Delta's + // `HadoopFileSystemDVStore` on the driver and feed the resulting row-index list + // through the proto's existing `deleted_row_indexes` field. The native side then + // wraps the file group in `DeltaDvFilterExec` (planner.rs ~1460) which already + // honours per-file deleted row indexes. If any DV fails to materialise we have + // to fall back -- silently dropping a DV is a correctness violation (would + // return rows that should have been hidden). + val hadoopConf = relation.sparkSession.sessionState + .newHadoopConfWithOptions(relation.options) + val deletedRowIndexesByPath: Map[String, Array[Long]] = { + val builder = scala.collection.mutable.Map.empty[String, Array[Long]] + val it = addFiles.iterator + var failed = false + while (it.hasNext && !failed) { + val af = it.next() + if (af.hasDeletionVector) { + DeltaReflection.materializeDeletedRowIndexes( + af.dvDescriptor, + tableRoot, + hadoopConf) match { + case Some(arr) => builder.put(af.path, arr) + case None => failed = true + } + } + } + if (failed) { + import org.apache.comet.CometSparkSessionExtensions.withInfo + withInfo( + scan, + "Native Delta scan: pre-materialised FileIndex with deletion vectors " + + "but failed to materialise one or more DVs (DV file missing, unsupported " + + "Delta version, or read error); falling back to Spark+Delta.") + return None + } + builder.toMap + } + buildTaskListFromAddFiles( + tableRoot, + snapshotVersion, + addFiles, + nativeOp = null, + columnNames, + physicalToLogicalPartitionNames = physToLogical, + deletedRowIndexesByPath = deletedRowIndexesByPath).toByteArray + case None => + // Reflection failed; fall back conservatively. + import org.apache.comet.CometSparkSessionExtensions.withInfo + withInfo( + scan, + s"Native Delta scan could not extract AddFiles from " + + s"${relation.location.getClass.getName}; falling back.") + return None + } + } else { + // Non-batch indexes (TahoeLogFileIndex, ...). DV-bearing + // PreparedDeltaFileIndex is now classified as a batch index above + // (see `isBatchFileIndex`), so its DV-fallback case is already + // handled by the `case Some(_)` arm at the top of this match. For + // remaining non-batch indexes the Delta-PreprocessTableWithDVs + // wrapper detection upstream in `CometScanRule.scanBelowFallsBackForDvs` + // is responsible for keeping DV-aware internal reads on vanilla. + try { + nativeLib.planDeltaScan( + tableRoot, + snapshotVersion, + storageOptions, + predicateBytes, + columnNames) + } catch { + case scala.util.control.NonFatal(e) => + logWarning( + s"CometDeltaNativeScan: delta-kernel-rs log replay failed for $tableRoot", + e) + return None + } + } + val taskList0 = DeltaScanTaskList.parseFrom(taskListBytes) + // The kernel path populates `column_mappings` from kernel's schema metadata. + // The pre-materialised-index path (`buildTaskListFromAddFiles`) doesn't have + // that information yet, so re-derive the mapping from the relation's data + // + partition schema -- each StructField carries + // `delta.columnMapping.physicalName` in its metadata when the table uses + // column mapping. Without this the native scan can't translate logical + // column references to physical parquet column names and returns nulls. + // Fetch the Snapshot-level schema via reflection once here; used both to populate column + // mappings from the data-schema side (metadata on relation.dataSchema is stripped) and + // later to physicalise nested field names before serialisation. + val snapshotSchemaEarly: Option[StructType] = DeltaReflection.extractSnapshotSchema(relation) + // Only honour physicalName metadata when the table actually has column mapping + // mode enabled. Some Delta test helpers (e.g. `DeltaSourceSuiteBase.withMetadata`) + // call `DeltaColumnMapping.assignColumnIdAndPhysicalName` unconditionally, which + // attaches `delta.columnMapping.physicalName` to every StructField even when the + // table's `delta.columnMapping.mode` is unset / `none`. In that case the writer + // still uses LOGICAL names in the parquet file, so physicalising our scan would + // look up non-existent physical column names and return empty rows. + val tableColumnMappingMode = DeltaReflection + .extractMetadataConfiguration(relation) + .flatMap(_.get("delta.columnMapping.mode")) + .filter(m => m != null && !m.equalsIgnoreCase("none")) + val taskList = + if (!taskList0.getColumnMappingsList.isEmpty || tableColumnMappingMode.isEmpty) { + taskList0 + } else { + // `relation.dataSchema.fields[*].metadata` is stripped of Delta's column-mapping + // metadata by HadoopFsRelation, so the lookup here nearly always returns empty. + // Use the Snapshot schema we extracted (which preserves physical names at every + // level) for the data-column mappings, and `relation.partitionSchema` only for + // partition columns (whose metadata isn't stripped). + val dataFieldsSource: Array[StructField] = + snapshotSchemaEarly.map(_.fields).getOrElse(relation.dataSchema.fields) + val allFields = dataFieldsSource ++ relation.partitionSchema.fields + val logicalToPhysical = allFields.flatMap { f => + if (f.metadata.contains(DeltaReflection.PhysicalNameMetadataKey)) { + Some(f.name -> f.metadata.getString(DeltaReflection.PhysicalNameMetadataKey)) + } else { + None + } + } + if (logicalToPhysical.isEmpty) { + taskList0 + } else { + val b = DeltaScanTaskList.newBuilder(taskList0) + logicalToPhysical.foreach { case (logical, physical) => + b.addColumnMappings( + OperatorOuterClass.DeltaColumnMapping + .newBuilder() + .setLogicalName(logical) + .setPhysicalName(physical) + .build()) + } + b.build() + } + } + + // Phase 6 reader-feature gate. Kernel reports any Delta reader features that + // are currently in use in this snapshot and that Comet's native path does NOT + // correctly handle. Falling back is mandatory for correctness: reading through + // the native path would silently produce wrong results (e.g. returning rows + // that a deletion vector should have hidden). The gate becomes obsolete feature + // by feature as later phases ship: + // deletionVectors -> Phase 3 + // columnMapping -> Phase 4 + // typeWidening -> future phase + // rowTracking -> future phase + val unsupportedFeatures = taskList.getUnsupportedFeaturesList.asScala.toSeq + if (unsupportedFeatures.nonEmpty && + DeltaConf.COMET_DELTA_FALLBACK_ON_UNSUPPORTED_FEATURE.get(scan.conf)) { + logInfo( + s"CometDeltaNativeScan: falling back for table $tableRoot " + + s"due to unsupported reader features: ${unsupportedFeatures.mkString(", ")}") + import org.apache.comet.CometSparkSessionExtensions.withInfo + withInfo( + scan, + s"Native Delta scan does not yet support these features in use on this " + + s"snapshot: ${unsupportedFeatures.mkString(", ")}. Falling back to Spark's " + + s"Delta reader. Set ${DeltaConf.COMET_DELTA_FALLBACK_ON_UNSUPPORTED_FEATURE.key}=false " + + s"to bypass this check (NOT recommended - may produce incorrect results).") + return None + } + + // Apply Spark's partition filters to the task list so that queries like + // `WHERE partition_col = X` don't drag in files from other partitions. Kernel + // itself is given the whole snapshot (no predicate yet - that lands in Phase 2), + // so we do the pruning in Scala by evaluating each task's partition-value map + // against Spark's `partitionFilters`. This is a single driver-side loop; filtered + // tasks never go over the wire to executors. + val filteredTasks0 = + prunePartitions(taskList.getTasksList.asScala.toSeq, scan, relation.partitionSchema) + + // Split files larger than `maxSplitBytes` into byte-range chunks so a single + // big parquet file can be read across multiple Spark partitions, matching + // Spark's `FilePartition.splitFiles` semantics. This is what makes + // FILES_MAX_PARTITION_BYTES, files.openCostInBytes, and + // files.minPartitionNum take effect on Delta tables: without it every file + // is exactly one partition and the *.size assertions in + // DeletionVectorsSuite's PredicatePushdown tests fail (they configure + // FILES_MAX_PARTITION_BYTES=2MB on a multi-row-group fixture and assert + // exactly 2 splits). + val filteredTasks = + splitTasks(scan, filteredTasks0) + + // --- 2. Build the common block --- + val commonBuilder = DeltaScanCommon.newBuilder() + commonBuilder.setSource(scan.simpleStringWithNodeId()) + commonBuilder.setTableRoot(taskList.getTableRoot) + commonBuilder.setSnapshotVersion(taskList.getSnapshotVersion) + commonBuilder.setSessionTimezone(scan.conf.sessionLocalTimeZone) + commonBuilder.setCaseSensitive(scan.conf.getConf[Boolean](SQLConf.CASE_SENSITIVE)) + commonBuilder.setIgnoreMissingFiles(ignoreMissingFiles) + commonBuilder.setDataFileConcurrencyLimit( + DeltaConf.COMET_DELTA_DATA_FILE_CONCURRENCY_LIMIT.get()) + + // Schemas. Delta is different from vanilla Parquet: `relation.dataSchema` on a Delta + // table INCLUDES partition columns, but the physical parquet files on disk do NOT. + // So we compute the actual file schema by subtracting the partition columns from + // `relation.dataSchema`. Mirrors what delta-kernel itself reports as the scan schema. + val partitionNames = + relation.partitionSchema.fields.map(_.name.toLowerCase(Locale.ROOT)).toSet + val fileDataSchemaFields = + relation.dataSchema.fields.filterNot(f => + partitionNames.contains(f.name.toLowerCase(Locale.ROOT))) + + // When column mapping (id or name) is active, Delta writes parquet files using physical + // names at EVERY level of nesting -- struct inner fields, array elements, map keys/values. + // `schema2Proto` otherwise serialises the Spark StructField tree with logical names, so the + // native parquet reader would look for e.g. `b1` and its inner `c` but the file has + // `col-` and `col-`, yielding a null-struct read. Substitute physical names + // recursively before serialising so the proto schema matches the on-disk names at every + // level. The `column_mappings` proto carries only top-level logical->physical so that + // filter column references (expressed with logical names) still translate correctly. + // Detect column mapping from the most reliable sources: + // 1. Kernel-side proto already populated the flat logical->physical map, OR + // 2. `relation.dataSchema` StructField metadata carries the physical-name key (rare -- + // HadoopFsRelation strips this on construction, but iceberg-compat paths don't), OR + // 3. the Delta snapshot's Metadata.configuration declares `delta.columnMapping.mode` + // not equal to `none`. This is the authoritative source and catches the case where + // (1) and (2) both miss. + // A false negative here is silent data-corruption (physicalisation skipped, native reader + // looks for logical names in physical-named parquet), so the fallback probe is important. + val columnMappingActive = taskList.getColumnMappingsList.asScala.nonEmpty || + relation.dataSchema.fields.exists( + _.metadata.contains(DeltaReflection.PhysicalNameMetadataKey)) || + DeltaReflection + .extractMetadataConfiguration(relation) + .flatMap(_.get("delta.columnMapping.mode")) + .exists(m => m != null && !m.equalsIgnoreCase("none")) + // `relation.dataSchema` has its StructField metadata stripped by Spark's HadoopFsRelation + // construction, so nested physical names are invisible. Reuse the snapshot schema fetched + // above (or None when column mapping isn't active). + val snapshotSchema: Option[StructType] = + if (columnMappingActive) snapshotSchemaEarly else None + val physicalByLogicalName: Map[String, StructField] = + snapshotSchema.map(_.fields.map(f => f.name -> f).toMap).getOrElse(Map.empty) + // Preserve the top-level LOGICAL name and substitute only NESTED (struct/map/array) inner + // field names with their physical equivalents. The native planner (planner.rs ~1383) + // already handles top-level logical->physical substitution using the flat `column_mappings` + // proto. Fields not present in the snapshot (e.g. synthetic `_tmp_metadata_row_index`) are + // passed through untouched. + def physicaliseNestedTypesOnly(f: StructField): StructField = + physicalByLogicalName.get(f.name) match { + case Some(metaField) => + StructField(f.name, physicaliseDataType(metaField.dataType), f.nullable, f.metadata) + case None => f + } + // For `required_schema` we MUST preserve the field's pruned shape (Spark's + // nested column pruning can leave a struct with only the accessed children) while + // still rewriting nested names to their physical equivalents. Using the data-schema + // helper above (which replaces the whole struct with the snapshot's full shape) + // would lose pruning and produce nested children Spark's plan does not expect, + // causing GetStructField ordinals to point at the wrong child. Walks `req`'s tree + // and pairs each node with the corresponding snapshot node by logical name to find + // the physical name; fields not present in the snapshot pass through untouched. + def physicaliseDataTypePreserving(req: DataType, snap: DataType): DataType = + (req, snap) match { + case (rs: StructType, ms: StructType) => + val snapByLogical = ms.fields.map(f => f.name -> f).toMap + StructType(rs.fields.map { rf => + snapByLogical.get(rf.name) match { + case Some(mf) => + val physName = + if (mf.metadata.contains(DeltaReflection.PhysicalNameMetadataKey)) { + mf.metadata.getString(DeltaReflection.PhysicalNameMetadataKey) + } else rf.name + StructField( + physName, + physicaliseDataTypePreserving(rf.dataType, mf.dataType), + rf.nullable, + rf.metadata) + case None => rf + } + }) + case (ra: ArrayType, ma: ArrayType) => + ArrayType( + physicaliseDataTypePreserving(ra.elementType, ma.elementType), + ra.containsNull) + case (rm: MapType, mm: MapType) => + MapType( + physicaliseDataTypePreserving(rm.keyType, mm.keyType), + physicaliseDataTypePreserving(rm.valueType, mm.valueType), + rm.valueContainsNull) + case _ => req + } + def physicaliseRequiredField(f: StructField): StructField = + physicalByLogicalName.get(f.name) match { + case Some(metaField) => + StructField( + f.name, + physicaliseDataTypePreserving(f.dataType, metaField.dataType), + f.nullable, + f.metadata) + case None => f + } + // `data_schema` describes what we want the native parquet reader to read from + // the file. Under column mapping, parquet column matching is by PHYSICAL name + // (at every level of nesting). The reader projects by leaf column path -- it + // can read just `b.col-d` even if the file's `b` also has `col-c`. To make + // that happen, we send data_schema with the SAME shape as the required output: + // top-level fields that are required carry the pruned + physicalised nested + // shape; non-required top-level fields keep their full physicalised shape (no + // read attempt is made for them anyway because they don't appear in + // projection_vector). Without this overlay, the reader would emit a struct + // with ALL nested children (full file shape), and upstream GetStructField + // ordinals -- computed by Catalyst against the PRUNED required_schema -- + // would pick the wrong child. Manifested as "Invalid comparison Utf8 <= Int32" + // on `b.d > 0` (d is INT, ordinal 0 in pruned `b: struct`, but ordinal 0 + // in the file struct is `c` STRING). #79 fix 2026-05-13. + // `requiredSchema` on the wire is the SCAN's output schema -- i.e. data columns the + // scan reads from parquet PLUS partition columns it materialises from + // PartitionedFile.partition_values. Upstream operators in the native plan tree bind + // their column references by index into this schema. For non-partitioned tables + // `scan.requiredSchema` is the whole output already; for partitioned tables Spark + // gives us just the data half here, so we append the partition fields at the tail to + // match the layout indices in `projection_vector` resolve into. + val partitionFieldsForRequired: Array[StructField] = { + val haveLc = scan.requiredSchema.fields.map(_.name.toLowerCase(Locale.ROOT)).toSet + relation.partitionSchema.fields.filterNot(f => + haveLc.contains(f.name.toLowerCase(Locale.ROOT))) + } + val requiredSchemaFields = { + val base = + if (columnMappingActive) scan.requiredSchema.fields.map(physicaliseRequiredField) + else scan.requiredSchema.fields + base ++ partitionFieldsForRequired + } + val physicalFileDataSchemaFields = if (columnMappingActive) { + val requiredByName = requiredSchemaFields + .map(f => f.name.toLowerCase(Locale.ROOT) -> f) + .toMap + fileDataSchemaFields.map { f => + requiredByName.get(f.name.toLowerCase(Locale.ROOT)) match { + // Required asks for this field -- adopt its pruned, physicalised shape so + // the parquet reader projects only the required nested children. + case Some(req) => StructField(f.name, req.dataType, f.nullable, f.metadata) + // Field not required -- physicalise the full snapshot shape (used only if + // some other consumer references it; harmless when projection_vector skips it). + case None => physicaliseNestedTypesOnly(f) + } + } + } else fileDataSchemaFields + + val dataSchema = schema2Proto(physicalFileDataSchemaFields) + val requiredSchema = schema2Proto(requiredSchemaFields) + val partitionSchema = schema2Proto(relation.partitionSchema.fields) + commonBuilder.addAllDataSchema(dataSchema.toIterable.asJava) + commonBuilder.addAllRequiredSchema(requiredSchema.toIterable.asJava) + commonBuilder.addAllPartitionSchema(partitionSchema.toIterable.asJava) + + // Projection vector maps output positions to (file_data_schema ++ partition_schema) + // indices. Spark's `FileSourceScanExec` splits its visible schema into + // `requiredSchema` (data-only columns that must be read from parquet) and an + // implicit partition tail that is materialised from `PartitionedFile.partition_values`. + // The scan's `output` is `requiredSchema ++ partitionSchema` in that order. + // + // We mirror that layout: first emit one index per required (data) field pointing + // into `fileDataSchemaFields`, then append one index per partition field pointing + // at `fileDataSchemaFields.length + partitionIdx` so the native side resolves those + // positions against `PartitionedFile.partition_values`. + // + // If `scan.requiredSchema` ever contains a partition column (some Delta code paths + // leak one in), we resolve it through the partition tail without re-reading from + // parquet. + val partitionNameToIndex: Map[String, Int] = + relation.partitionSchema.fields.zipWithIndex.map { case (f, i) => + f.name.toLowerCase(Locale.ROOT) -> i + }.toMap + val requiredIndexes: Seq[Int] = scan.requiredSchema.fields.map { field => + val nameLower = field.name.toLowerCase(Locale.ROOT) + val dataIdx = + fileDataSchemaFields.indexWhere(_.name.toLowerCase(Locale.ROOT) == nameLower) + if (dataIdx >= 0) { + dataIdx + } else { + partitionNameToIndex + .get(nameLower) + .map(p => fileDataSchemaFields.length + p) + .getOrElse(-1) + } + } + val partitionTailIndexes: Seq[Int] = + relation.partitionSchema.fields.indices.map(i => fileDataSchemaFields.length + i) + val projectionVector: Seq[Int] = requiredIndexes ++ partitionTailIndexes + commonBuilder.addAllProjectionVector( + projectionVector.map(idx => idx.toLong.asInstanceOf[java.lang.Long]).toIterable.asJava) + + // Pushed-down data filters. Gated by Spark's parquet filter pushdown config, same as + // CometNativeScan, so we behave consistently across scan implementations. + // + // Filters referencing nested (struct/array/map) columns aren't safe to push into + // `ParquetSource`: DataFusion currently produces "Invalid comparison operation: Utf8 <= + // Int32" (or similar) when the filter references an array element through + // `GetArrayItem`/`GetStructField`/`GetMapValue`, because the expression tree is walked + // against the file schema where the child types don't match the literal. The filter is + // still evaluated correctly by Spark post-scan, so dropping it from pushdown keeps the + // scan results correct at the cost of some row-group-level pruning. + def referencesNestedAccess(e: Expression): Boolean = e.exists { + case _: org.apache.spark.sql.catalyst.expressions.GetArrayItem => true + case _: org.apache.spark.sql.catalyst.expressions.GetArrayStructFields => true + case _: org.apache.spark.sql.catalyst.expressions.GetMapValue => true + case _ => false + } + if (scan.conf.getConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED) && + CometConf.COMET_RESPECT_PARQUET_FILTER_PUSHDOWN.get(scan.conf)) { + // Partition columns are NOT in the file's data schema; the native parquet path + // evaluates pushed-down filters against the file-data schema only, so a filter + // that references a partition column would resolve to an out-of-bounds Bound + // index ("Column index N is out of bound. Schema: Field {}"). + // Spark normally separates `partitionFilters` from `dataFilters` at planning + // time, but `scan.supportedDataFilters` can still surface filters that touch + // both data + partition columns (or pure-partition filters when the optimizer + // didn't peel them off cleanly). Skip any filter that references a partition + // attribute; partition pruning is handled separately by `prunePartitions` + // driver-side via the kernel/AddFile path. + val partitionNamesLc: Set[String] = + relation.partitionSchema.fields.map(_.name.toLowerCase(Locale.ROOT)).toSet + def referencesPartitionColumn(e: Expression): Boolean = e.exists { + case a: org.apache.spark.sql.catalyst.expressions.AttributeReference => + partitionNamesLc.contains(a.name.toLowerCase(Locale.ROOT)) + case _ => false + } + val dataFilters = new ListBuffer[Expr]() + scan.supportedDataFilters.foreach { filter => + if (referencesNestedAccess(filter)) { + logInfo(s"CometDeltaNativeScan: skipping pushdown of nested-access filter $filter") + } else if (referencesPartitionColumn(filter)) { + logInfo(s"CometDeltaNativeScan: skipping pushdown of partition-column filter $filter") + } else { + exprToProto(filter, scan.output) match { + case Some(proto) => dataFilters += proto + case _ => logWarning(s"CometDeltaNativeScan: unsupported data filter $filter") + } + } + } + commonBuilder.addAllDataFilters(dataFilters.asJava) + } + + storageOptions.asScala.foreach { case (key, value) => + commonBuilder.putObjectStoreOptions(key, value) + } + + // Phase 4: pass column mapping from kernel through to the native planner. + val columnMappings = taskList.getColumnMappingsList.asScala + columnMappings.foreach { cm => + commonBuilder.addColumnMappings( + OperatorOuterClass.DeltaColumnMapping + .newBuilder() + .setLogicalName(cm.getLogicalName) + .setPhysicalName(cm.getPhysicalName) + .build()) + } + + // --- 3. Pack into a DeltaScan with COMMON ONLY (split-mode, Phase 5). + // Tasks are NOT included in the proto at planning time. They'll be + // serialized per-partition in CometDeltaNativeScanExec.serializedPartitionData + // at execution time, and merged via DeltaPlanDataInjector. + val deltaScanBuilder = DeltaScan.newBuilder() + deltaScanBuilder.setCommon(commonBuilder.build()) + // No addAllTasks: tasks stay in taskListBytes for the exec's lazy split. + + // Stash the full task-list bytes for createExec to retrieve. The ThreadLocal + // bridges the convert() -> createExec() gap in CometExecRule.convertToComet. + // Build a modified taskList with ONLY the filtered tasks (partition-pruned). + val filteredTaskList = DeltaScanTaskList + .newBuilder() + .setSnapshotVersion(taskList.getSnapshotVersion) + .setTableRoot(taskList.getTableRoot) + .addAllTasks(filteredTasks.asJava) + .addAllColumnMappings(taskList.getColumnMappingsList) + .addAllUnsupportedFeatures(taskList.getUnsupportedFeaturesList) + .build() + lastTaskListBytes.set(filteredTaskList.toByteArray) + + // Use the typed DeltaScan proto variant. Core's planner dispatches via the + // OpStruct::DeltaScan match arm under `#[cfg(feature = "contrib-delta")]`. + builder.clearChildren() + Some(builder.setDeltaScan(deltaScanBuilder.build()).build()) + } + + /** + * Filter `tasks` down to the subset whose partition values satisfy Spark's + * `scan.partitionFilters`. Returns the original list unchanged when the scan has no partition + * filters. + * + * Recursively rewrite a `StructField` and its `DataType` so every field name at every level of + * nesting reflects the column-mapping physical name stored in its metadata. For fields without + * the physical-name metadata (e.g. partition columns, or inner struct fields on a + * non-column-mapped table), the logical name is retained. Only reached for nested struct/map/ + * array elements -- top-level columns keep their logical name (the native planner does that + * substitution via the `column_mappings` proto). + */ + private def physicaliseStructField(f: StructField): StructField = { + val physName = + if (f.metadata.contains(DeltaReflection.PhysicalNameMetadataKey)) { + f.metadata.getString(DeltaReflection.PhysicalNameMetadataKey) + } else { + f.name + } + StructField(physName, physicaliseDataType(f.dataType), f.nullable, f.metadata) + } + + private def physicaliseDataType(dt: DataType): DataType = dt match { + case s: StructType => StructType(s.fields.map(physicaliseStructField)) + case a: ArrayType => ArrayType(physicaliseDataType(a.elementType), a.containsNull) + case m: MapType => + MapType( + physicaliseDataType(m.keyType), + physicaliseDataType(m.valueType), + m.valueContainsNull) + case other => other + } + + /** + * Compute Spark's `maxSplitBytes` for a Delta scan. Mirrors + * `org.apache.spark.sql.execution.datasources.FilePartition.maxSplitBytes` verbatim so a + * Delta-native scan splits files the same way a vanilla `FileSourceScanExec` would. Inputs are + * file sizes (bytes); other knobs come from session conf and the relation's spark session. + */ + private def maxSplitBytes(scan: CometScanExec, fileSizes: Seq[Long]): Long = { + val sparkSession = scan.relation.sparkSession + val conf = sparkSession.sessionState.conf + val openCostInBytes = conf.filesOpenCostInBytes + val maxPartitionBytes = conf.filesMaxPartitionBytes + val minPartitionNum = conf.filesMinPartitionNum + .getOrElse(sparkSession.sparkContext.defaultParallelism) + val totalBytes = fileSizes.map(_ + openCostInBytes).sum + val bytesPerCore = totalBytes / math.max(1, minPartitionNum) + math.min(maxPartitionBytes, math.max(openCostInBytes, bytesPerCore)) + } + + /** + * Expand `tasks` so any task whose file is larger than `maxSplitBytes` is replaced by a + * sequence of byte-range chunks. Each chunk inherits the task's metadata (partition values, DV + * row indexes, row-tracking ids) but carries `byte_range_start` / `byte_range_end` so the + * native parquet reader only materialises row groups whose start offset falls in this range. + * + * Tasks that fit in one chunk are emitted unchanged (no range fields), which preserves the + * original whole-file semantics on the native side. + * + * Note on DV semantics: deletion-vector indexes on the proto are absolute row positions within + * the file. They are copied to every chunk; the native scan filters out rows whose absolute + * index is in the DV regardless of which chunk produced them, so duplicating the index list + * across chunks is correct (just slightly wasteful). + */ + private def splitTasks( + scan: CometScanExec, + tasks: Seq[OperatorOuterClass.DeltaScanTask]): Seq[OperatorOuterClass.DeltaScanTask] = { + if (tasks.isEmpty) return tasks + // #75 design A: when the plan needs input_file_name(), keep each task 1:1 with + // a file so `CometExecRDD.compute` (which reads only the first task) sets + // the correct path. Without this, byte-range chunking would create multiple + // tasks for one file -- still same path -- BUT combined with packTasks below + // could end up with multiple FILES per partition. + if (scanNeedsInputFileName(scan)) return tasks + val sizes = tasks.map(_.getFileSize) + val msb = maxSplitBytes(scan, sizes) + if (msb <= 0) return tasks + tasks.flatMap { task => + val size = task.getFileSize + if (size <= msb) Seq(task) + else { + val chunks = scala.collection.mutable.ArrayBuffer[OperatorOuterClass.DeltaScanTask]() + var offset = 0L + while (offset < size) { + val end = math.min(offset + msb, size) + chunks += task.toBuilder + .setByteRangeStart(offset) + .setByteRangeEnd(end) + .build() + offset = end + } + chunks.toSeq + } + } + } + + private def prunePartitions( + tasks: Seq[OperatorOuterClass.DeltaScanTask], + scan: CometScanExec, + partitionSchema: StructType): Seq[OperatorOuterClass.DeltaScanTask] = { + if (scan.partitionFilters.isEmpty || partitionSchema.isEmpty) return tasks + + // Phase 5b: filter out DPP expressions (DynamicPruningExpression wrapping + // InSubqueryExec) because they aren't resolved at planning time. Spark + // applies them post-scan at runtime. Static partition filters are still + // evaluated here for file-level pruning. + val staticFilters = scan.partitionFilters.filterNot( + _.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]])) + if (staticFilters.isEmpty) return tasks + + // Build an `InterpretedPredicate` that expects a row whose schema matches + // `partitionSchema`. Rewrite attribute references to `BoundReference`s keyed by + // partition-schema field index, respecting case sensitivity. + val caseSensitive = scan.conf.getConf[Boolean](SQLConf.CASE_SENSITIVE) + val combined = staticFilters.reduce(And) + val bound = combined.transform { + case a: org.apache.spark.sql.catalyst.expressions.AttributeReference => + val idx = if (caseSensitive) { + partitionSchema.fieldIndex(a.name) + } else { + partitionSchema.fields.indexWhere( + _.name.toLowerCase(Locale.ROOT) == a.name.toLowerCase(Locale.ROOT)) + } + if (idx < 0) return tasks // Can't resolve; skip pruning + BoundReference(idx, partitionSchema(idx).dataType, partitionSchema(idx).nullable) + } + val predicate = InterpretedPredicate(bound) + predicate.initialize(0) + + val sessionZoneId = java.time.ZoneId.of(scan.conf.sessionLocalTimeZone) + tasks.filter { task => + val row = InternalRow.fromSeq(partitionSchema.fields.toSeq.map { field => + val proto = task.getPartitionValuesList.asScala.find(_.getName == field.name) + val strValue = + if (proto.exists(_.hasValue)) Some(proto.get.getValue) else None + DeltaReflection.castPartitionString(strValue, field.dataType, sessionZoneId) + }) + predicate.eval(row) + } + } + + /** + * Build a kernel-independent `DeltaScanTaskList` from a caller-provided AddFile list. Used when + * the Delta scan has a pre-materialized FileIndex (streaming micro-batch, MERGE/UPDATE/DELETE + * post-join) so we can honour its exact file list instead of re-running log replay (which would + * return a different set). + * + * Each AddFile becomes one `DeltaScanTask`. Absolute path resolution mirrors + * `DeltaFileOperations.absolutePath`: if `AddFile.path` is already absolute (has a URI scheme), + * keep it verbatim; otherwise join against `tableRoot`. + */ + private def buildTaskListFromAddFiles( + tableRoot: String, + snapshotVersion: Long, + addFiles: Seq[DeltaReflection.ExtractedAddFile], + nativeOp: AnyRef, + columnNames: Array[String], + physicalToLogicalPartitionNames: Map[String, String] = Map.empty, + deletedRowIndexesByPath: Map[String, Array[Long]] = Map.empty) + : OperatorOuterClass.DeltaScanTaskList = { + val tlBuilder = OperatorOuterClass.DeltaScanTaskList.newBuilder() + tlBuilder.setTableRoot(tableRoot) + if (snapshotVersion >= 0) tlBuilder.setSnapshotVersion(snapshotVersion) + + addFiles.foreach { af => + val absPath = + if (af.path.contains(":/")) af.path + else { + val sep = if (tableRoot.endsWith("/")) "" else "/" + tableRoot + sep + af.path + } + val taskBuilder = OperatorOuterClass.DeltaScanTask.newBuilder() + taskBuilder.setFilePath(absPath) + taskBuilder.setFileSize(af.size) + DeltaReflection.parseNumRecords(af.statsJson).foreach(taskBuilder.setRecordCount) + af.partitionValues.foreach { case (k, v) => + // Under column mapping, Delta stores partition values keyed by the + // PHYSICAL column name (e.g. `col--part`). Our partition_schema + // on the wire uses LOGICAL names, and `build_delta_partitioned_files` + // native-side matches by name. Translate when we have a physical + // ->logical map (the kernel-path jni.rs already performs the same + // translation for its own extraction). + val logicalName = physicalToLogicalPartitionNames.getOrElse(k, k) + val pvBuilder = + OperatorOuterClass.DeltaPartitionValue.newBuilder().setName(logicalName) + if (v != null) pvBuilder.setValue(v) + taskBuilder.addPartitionValues(pvBuilder.build()) + } + af.baseRowId.foreach(taskBuilder.setBaseRowId) + af.defaultRowCommitVersion.foreach(taskBuilder.setDefaultRowCommitVersion) + deletedRowIndexesByPath.get(af.path).foreach { rowIndexes => + var i = 0 + while (i < rowIndexes.length) { + taskBuilder.addDeletedRowIndexes(rowIndexes(i)) + i += 1 + } + } + tlBuilder.addTasks(taskBuilder.build()) + } + tlBuilder.build() + } + + def createExec(nativeOp: Operator, op: CometScanExec): CometNativeExec = { + val tableRoot = DeltaReflection.extractTableRoot(op.relation).getOrElse("unknown") + val tlBytes = + try { + Option(lastTaskListBytes.get()).getOrElse(Array.emptyByteArray) + } finally { + lastTaskListBytes.remove() + } + val oneTaskPerPartition = scanNeedsInputFileName(op) + + val dppFilters = op.partitionFilters.filter( + _.exists(_.isInstanceOf[org.apache.spark.sql.catalyst.expressions.PlanExpression[_]])) + val partitionSchema = op.relation.partitionSchema + + CometDeltaNativeScanExec( + nativeOp, + op.output, + org.apache.spark.sql.comet.SerializedPlan(None), + op.wrapped, + tableRoot, + tlBytes, + dppFilters, + partitionSchema, + oneTaskPerPartition = oneTaskPerPartition) + } +} diff --git a/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaConf.scala b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaConf.scala new file mode 100644 index 0000000000..9b519c3f82 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaConf.scala @@ -0,0 +1,78 @@ +/* + * 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. + */ + +package org.apache.comet.contrib.delta + +import org.apache.comet.{ConfigBuilder, ConfigEntry} + +/** + * Contrib-local config entries for the Delta integration. Lives in the contrib's package rather + * than in core's `CometConf` so PR1 stays format-agnostic. Side-effect of object construction is + * registering the entries with `CometConf.allConfs` (via the `ConfigBuilder` machinery), so they + * show up in the generated user-guide docs and `SQLConf` resolution works the usual way. + */ +object DeltaConf { + + // CometConf.register asserts every config has a non-empty category — used for grouping + // entries in the generated user-guide docs. "scan" matches the existing core + // CATEGORY_SCAN string (CATEGORY_* constants in CometConf are `private val` so contribs + // can't reference the symbol; the assertion only checks `nonEmpty`). + private val CATEGORY = "scan" + + val COMET_DELTA_NATIVE_ENABLED: ConfigEntry[Boolean] = + ConfigBuilder("spark.comet.scan.deltaNative.enabled") + .category(CATEGORY) + .doc( + "Whether to enable native Delta table scans via delta-kernel-rs. When enabled, " + + "Delta tables are read directly through Comet's tuned ParquetSource + " + + "DV-filter wrapper, bypassing Spark's Delta reader for better performance.") + .booleanConf + .createWithDefault(true) + + val COMET_DELTA_FALLBACK_ON_UNSUPPORTED_FEATURE: ConfigEntry[Boolean] = + ConfigBuilder("spark.comet.scan.deltaNative.fallbackOnUnsupportedFeature") + .category(CATEGORY) + .doc( + "When true (default), the Delta contrib falls back to Spark's Delta reader on " + + "any Delta protocol feature it doesn't yet support. When false, the contrib " + + "raises an error instead -- useful for tests that want to assert the native " + + "path is reachable for a particular query.") + .booleanConf + .createWithDefault(true) + + val COMET_DELTA_DATA_FILE_CONCURRENCY_LIMIT: ConfigEntry[Int] = + ConfigBuilder("spark.comet.scan.deltaNative.dataFileConcurrencyLimit") + .category(CATEGORY) + .doc( + "Per-Spark-task concurrency when reading Delta data files. Higher values " + + "improve throughput on tables with many small files at the cost of memory. " + + "Values between 2 and 8 are typical.") + .intConf + .checkValue(v => v > 0, "Data file concurrency limit must be positive") + .createWithDefault(1) + + /** + * Relation-options key the contrib reads to know whether the surrounding plan references + * `input_file_name()` / `input_file_block_*`. When set to `"true"`, the contrib emits + * `oneTaskPerPartition = true` on the `CometDeltaNativeScanExec` so packTasks keeps each task + * in its own partition and `CometExecRDD.compute` (via `InputFileBlockHolder.set`) can set + * `InputFileBlockHolder` to the correct path. + */ + val NeedsInputFileNameOption: String = "comet.contrib.delta.needsInputFileName" +} diff --git a/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaReflection.scala b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaReflection.scala new file mode 100644 index 0000000000..b0aa823646 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaReflection.scala @@ -0,0 +1,705 @@ +/* + * 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. + */ + +package org.apache.comet.contrib.delta + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.read.{Scan => V2Scan} +import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation} +import org.apache.spark.sql.types.StructType + +/** + * Class-name-based probes for Delta Lake plan nodes. + * + * We deliberately avoid a compile-time dependency on `spark-delta` - the Scala API surface churns + * across Delta versions (2.x / 3.x / 4.x) and we want Comet's Delta detection to keep working + * against whichever Delta version the user has on their classpath. All detection is therefore + * done via fully-qualified class names and standard Spark APIs (HadoopFsRelation, V2Scan), which + * have been stable for years. + * + * What this object provides: + * - `isDeltaFileFormat(fileFormat)`: true for `DeltaParquetFileFormat` and any subclass exposed + * by the delta-spark package. + * - `isDeltaV2Scan(scan)`: true for the V2 `DeltaScan` / `DeltaTableV2`-backed scan. + * - `extractTableRoot(relation)`: pulls the table root URI out of a `HadoopFsRelation`. Works + * for both path-based reads (`format("delta").load("/tmp/t")`) and table-based reads + * (`spark.table("delta_tbl")`). + */ +object DeltaReflection extends Logging { + + /** Fully-qualified class names we match on. */ + object ClassNames { + val DELTA_PARQUET_FILE_FORMAT = "org.apache.spark.sql.delta.DeltaParquetFileFormat" + val DELTA_V2_SCAN_PACKAGE_PREFIX = "org.apache.spark.sql.delta." + val DELTA_V2_SCAN_SIMPLE_NAME = "DeltaScan" + } + + /** + * Synthetic column name that Delta's `PreprocessTableWithDVs` rule injects into a scan's output + * schema when the relation has deletion vectors in use. Value `0` means "keep the row", any + * other value means "drop it". Used to detect DV-rewritten Delta scans. + * + * Stable across Delta 2.x / 3.x - defined in + * `DeltaParquetFileFormat.IS_ROW_DELETED_COLUMN_NAME`. + */ + val IsRowDeletedColumnName: String = "__delta_internal_is_row_deleted" + + /** + * Synthetic column name Delta requests on the parquet scan when it needs the per-row physical + * position within the file (e.g. for downstream DV bitmap lookup in `useMetadataRowIndex` mode, + * or test-only reads of the metadata column). Produced only by `DeltaParquetFileFormat`'s + * reader; Comet's parquet reader has no equivalent synthesis. + * + * Stable across Delta 2.x / 3.x - defined in `DeltaParquetFileFormat.ROW_INDEX_COLUMN_NAME`. + */ + val RowIndexColumnName: String = "__delta_internal_row_index" + + /** + * Returns true if `fileFormat` is Delta's parquet-backed `FileFormat`. Checks the exact class + * plus any subclass, so variants like `DeletionVectorBoundFileFormat` (some Delta versions) + * also match. + */ + def isDeltaFileFormat(fileFormat: FileFormat): Boolean = { + val cls = fileFormat.getClass + isDeltaClassName(cls.getName) || isDeltaParquetSubclass(cls) + } + + /** Walks the class hierarchy looking for DeltaParquetFileFormat. */ + private def isDeltaParquetSubclass(cls: Class[_]): Boolean = { + var current: Class[_] = cls + while (current != null) { + if (current.getName == ClassNames.DELTA_PARQUET_FILE_FORMAT) return true + current = current.getSuperclass + } + false + } + + private def isDeltaClassName(name: String): Boolean = + name == ClassNames.DELTA_PARQUET_FILE_FORMAT || + (name.startsWith(ClassNames.DELTA_V2_SCAN_PACKAGE_PREFIX) && + name.endsWith("ParquetFileFormat")) + + /** + * Returns true if `scan` is the V2 scan implementation Delta produces for a + * `DeltaTableV2`-backed read. Delta ships this as `org.apache.spark.sql.delta.DeltaScan` (inner + * case class of `DeltaScanBuilder` or similar) - the enclosing class name varies by version, so + * we match on the simple name + package prefix rather than an exact FQN. + */ + def isDeltaV2Scan(scan: V2Scan): Boolean = { + val name = scan.getClass.getName + name.startsWith(ClassNames.DELTA_V2_SCAN_PACKAGE_PREFIX) && + name.contains(ClassNames.DELTA_V2_SCAN_SIMPLE_NAME) + } + + /** + * Extract the Delta table root from a V1 `HadoopFsRelation`. For Delta tables this is always a + * single path - Delta does not support multi-root relations. + * + * Returns the absolute URI as a string, with whatever scheme the relation was opened with + * (`file://`, `s3://`, etc.). + */ + def extractTableRoot(relation: HadoopFsRelation): Option[String] = { + try { + val roots = relation.location.rootPaths + roots.headOption.map(pathToSingleEncodedUri) + } catch { + case e: Exception => + logWarning(s"Failed to extract Delta table root path: ${e.getMessage}") + None + } + } + + /** + * Convert a Hadoop `Path` to a URI string whose decoded path component matches the literal + * on-disk filesystem path Hadoop uses when reading the file. + * + * Hadoop's `RawLocalFileSystem.pathToFile` treats the bytes of `path.toUri.getRawPath` -- i.e. + * the URI's path component WITHOUT decoding -- as the literal filesystem path. So if Hadoop's + * Path stores URI form `file:/T/spark%25dir%25prefix-uuid` (typical for Delta tests whose + * `defaultTempDirPrefix` is the literal `spark%dir%prefix`), the actual on-disk dir name is + * `spark%25dir%25prefix-uuid` (with `%25` literal in the filename, four chars `%`, `2`, `5`). + * + * To send a URI that the native side can decode back to that on-disk literal, we take the raw + * path component verbatim and URL-encode `%` one extra time, yielding + * `file:/T/spark%2525dir%2525prefix-uuid`. The native scan decodes once (`%2525` -> `%25`) and + * opens at the literal `%25` filename. + */ + def pathToSingleEncodedUri(p: org.apache.hadoop.fs.Path): String = { + // Hadoop's `Path` keeps two forms of the same URI: + // - `path.toString` returns a once-decoded form for display: any `%XX` + // escape stored in the URI is decoded once. For Delta tests whose + // `defaultTempDirPrefix` is the literal `spark%dir%prefix` and whose + // on-disk dir Spark actually creates is `spark%25dir%25prefix-uuid` + // (with `%25` four-char-literal in the filename), this returns + // `file:/T/spark%25dir%25prefix-uuid` -- which when fed to a URL + // parser would single-decode to a non-existent `spark%dir%prefix-uuid`. + // - `path.toUri.toString` returns the FULL URI form, double-encoding the + // literal `%` chars (`%25` -> `%2525`). When the native side parses + // this and percent-decodes once, it recovers the literal on-disk + // filename `spark%25dir%25prefix-uuid`. + // + // We always want the second form for native consumption, so the raw + // ParquetSource open path matches Hadoop's `RawLocalFileSystem` + // interpretation (which reads the URI's raw path component verbatim as + // the filesystem path). + p.toUri.toString + } + + /** + * Extract the resolved snapshot version from Delta's `FileIndex`. Delta's file index is a + * `TahoeLogFileIndex` / `PreparedDeltaFileIndex` which has already pinned a specific snapshot + * by the time we see it, including when the user supplied `versionAsOf` or `timestampAsOf`. + * + * The toString format is stable: `Delta[version=, ]`. We parse that rather than + * reaching into Delta's internals because the actual field names differ across Delta versions + * (snapshotAtAnalysis vs tahoeFileIndex.snapshot vs etc.). Regex is a single point of failure + * that's easy to update if the format ever changes. + * + * Returns the version as a `Long`, or `None` if parsing fails / the file index isn't a Delta + * one (callers should fall back to `-1` = latest). + */ + private val DeltaFileIndexVersionRegex = """^Delta\[version=(-?\d+),""".r + + /** + * Extract the Delta table `Metadata` action's configuration map from a `HadoopFsRelation`'s + * `TahoeFileIndex`-derivative location via reflection. Returns `None` when the lookup fails + * (e.g. non-Delta relation, or an index type that does not expose `metadata`). + * + * The configuration carries user- and system-set table properties keyed by dotted names like + * `delta.rowTracking.materializedRowIdColumnName`. Used by the CometScanRule row-tracking + * support to discover the physical column name into which Delta has materialised `row_id`. + */ + def extractMetadataConfiguration(relation: HadoopFsRelation): Option[Map[String, String]] = { + try { + val location: Any = relation.location + // Three-shape lookup. `TahoeBatchFileIndex` exposes only a `SnapshotDescriptor` + // (not `Snapshot`), so the `snapshot.metadata` chain misses for it; we walk + // `deltaLog.update().metadata` for the case-3 fallback. Keeping this in sync with + // `extractSnapshotSchema` below is critical: when CM is enabled and we miss the + // config, the contrib doesn't detect column mapping is active and falls back to + // logical-name reads on physically-renamed files. + val metadataObj: Option[AnyRef] = + findAccessor(location, Seq("metadata")) + .orElse(findAccessor(location, Seq("snapshot")).flatMap(findAccessor(_, Seq("metadata")))) + .orElse { + findAccessor(location, Seq("deltaLog")).flatMap { dl => + invokeNoArg(dl, "update").flatMap(findAccessor(_, Seq("metadata"))) + } + } + metadataObj.flatMap { m => + findAccessor(m, Seq("configuration")).collect { + case scalaMap: Map[_, _] => scalaMap.asInstanceOf[Map[String, String]] + case javaMap: java.util.Map[_, _] => + import scala.jdk.CollectionConverters._ + javaMap.asInstanceOf[java.util.Map[String, String]].asScala.toMap + } + } + } catch { + case e: Exception => + logWarning(s"Failed to extract Delta metadata configuration: ${e.getMessage}") + None + } + } + + /** StructField metadata key under which Delta stores the column-mapping physical name. */ + val PhysicalNameMetadataKey: String = "delta.columnMapping.physicalName" + + /** + * Extract the Delta table's Snapshot-level schema (`Metadata.schema()` in Delta terms) via + * reflection. Unlike the `relation.dataSchema` we get from Spark -- which has its StructField + * metadata stripped by HadoopFsRelation construction -- the Snapshot's schema preserves the + * `delta.columnMapping.physicalName` and `delta.columnMapping.id` metadata on every StructField + * at every level of nesting. This is the authoritative source for building a "physical schema" + * to hand to the native parquet reader. + */ + def extractSnapshotSchema(relation: HadoopFsRelation): Option[StructType] = { + try { + val location: Any = relation.location + // Three-shape lookup. `TahoeBatchFileIndex` (UPDATE/DELETE/MERGE post-rewrite and + // streaming micro-batches) exposes only a `SnapshotDescriptor` -- not a `Snapshot`, + // and the SnapshotDescriptor doesn't expose `Metadata` directly. For those, walk + // `deltaLog.update().metadata` (case 3) to get the LATEST snapshot's metadata. + // That's also correct for UPDATE/DELETE: those commands re-read the table at commit + // time, so the latest snapshot's column-mapping metadata is what governs how the + // parquet files we're about to read are interpreted. Without case 3, CM-name tables + // that have undergone RENAME COLUMN return wrong values from streaming/UPDATE/DELETE + // reads because the contrib falls back to `relation.dataSchema` whose StructField + // metadata is stripped by HadoopFsRelation, so `physicalName` is invisible, + // `column_mappings` proto stays empty, and the native parquet reader reads by + // logical name from a physically-renamed file. + val metadataObj: Option[AnyRef] = + findAccessor(location, Seq("metadata")) + .orElse(findAccessor(location, Seq("snapshot")).flatMap(findAccessor(_, Seq("metadata")))) + .orElse { + findAccessor(location, Seq("deltaLog")).flatMap { dl => + invokeNoArg(dl, "update").flatMap(findAccessor(_, Seq("metadata"))) + } + } + metadataObj.flatMap { m => + // Delta's Metadata exposes a `schema(): StructType` method that parses its stored JSON + // schema string. The returned StructType has full metadata preserved at every level. + val schema = invokeNoArg(m, "schema").orElse(findAccessor(m, Seq("schema"))) + schema.collect { case s: StructType => s } + } + } catch { + case scala.util.control.NonFatal(e) => + logWarning(s"Failed to extract Delta snapshot schema: ${e.getMessage}") + None + } + } + + private def invokeNoArg(obj: Any, methodName: String): Option[AnyRef] = { + if (obj == null) return None + try { + val m = lookupNoArgMethod(obj.getClass, methodName) + if (m == null) None else Option(m.invoke(obj)) + } catch { + case scala.util.control.NonFatal(_) => None + } + } + + /** Property key for the physical column name Delta materialises row IDs into. */ + val MaterializedRowIdColumnProp: String = + "delta.rowTracking.materializedRowIdColumnName" + + /** Property key for the physical column name Delta materialises row-commit-versions into. */ + val MaterializedRowCommitVersionColumnProp: String = + "delta.rowTracking.materializedRowCommitVersionColumnName" + + /** + * Row-tracking fields extracted per file for phase-3 synthesis of `_row_id_` and + * `_row_commit_version_` when the materialised physical columns are null. + */ + case class RowTrackingFileInfo(baseRowId: Option[Long], defaultRowCommitVersion: Option[Long]) + + /** + * Invoke `TahoeFileIndex.matchingFiles(partitionFilters = Nil, dataFilters = Nil)` on the given + * `location`, extract each returned `AddFile`'s `path`, `baseRowId`, and + * `defaultRowCommitVersion`, and return the resulting map keyed by file basename. + * + * Used by row-tracking Phase 3: we attach each file's starting row id and default commit + * version as per-file synthetic partition columns. Returns `Map.empty` on reflection failure. + */ + def extractRowTrackingInfoByFileName(location: Any): Map[String, RowTrackingFileInfo] = { + if (location == null) return Map.empty + try { + val addFilesAny = callMatchingFiles(location).getOrElse(return Map.empty) + val seq = addFilesAny match { + case s: scala.collection.Seq[_] => s + case a: Array[_] => a.toSeq + case _ => return Map.empty + } + val result = scala.collection.mutable.Map.empty[String, RowTrackingFileInfo] + seq.foreach { addFile => + val path = stringMember(addFile, "path") + val baseRowId = optionLongMember(addFile, "baseRowId") + val defaultVer = optionLongMember(addFile, "defaultRowCommitVersion") + path.foreach { p => + if (baseRowId.isDefined || defaultVer.isDefined) { + val name = new org.apache.hadoop.fs.Path(p).getName + result.put(name, RowTrackingFileInfo(baseRowId, defaultVer)) + } + } + } + result.toMap + } catch { + case _: Exception => Map.empty + } + } + + def extractSnapshotVersion(relation: HadoopFsRelation): Option[Long] = { + try { + val desc = relation.location.toString + DeltaFileIndexVersionRegex.findFirstMatchIn(desc).map(_.group(1).toLong) + } catch { + case _: Exception => None + } + } + + /** + * Read the LATEST committed version from the relation's underlying `DeltaLog`, via reflection + * so we keep zero compile-time dep on spark-delta. Returns `None` when the relation isn't + * backed by a Delta log (or reflection fails). + */ + def extractLatestSnapshotVersion(relation: HadoopFsRelation): Option[Long] = { + try { + val deltaLogObj = findAccessor(relation.location, Seq("deltaLog")).orNull + if (deltaLogObj == null) return None + // `deltaLog.update()` returns the latest Snapshot; `snapshot.version` is a Long. + val updated = invokeNoArg(deltaLogObj, "update").orNull + if (updated == null) return None + longMember(updated, "version") + } catch { + case _: Exception => None + } + } + + /** + * Convert a Delta partition value string to a Catalyst-internal representation. Delta stores + * partition values as strings in add actions; this converts them to the correct type for + * predicate evaluation. + */ + /** + * Normalized view of a single Delta `AddFile` extracted from a pre-materialized FileIndex + * (`TahoeBatchFileIndex` / `CdcAddFileIndex`). Used by the scan rule to build a + * kernel-independent `DeltaScanTask` list for streaming micro-batch reads and + * MERGE/UPDATE/DELETE post-join rewrites, both of which already have the exact AddFile list in + * hand and must NOT re-run kernel log replay (which would return a different file set). + */ + case class ExtractedAddFile( + /** Path as stored in the AddFile action -- may be relative or absolute. */ + path: String, + size: Long, + /** Raw partition values as Delta stores them, keyed by logical column name. */ + partitionValues: Map[String, String], + /** Raw `stats` JSON string, or null. */ + statsJson: String, + /** True if this AddFile has a non-null DeletionVectorDescriptor. */ + hasDeletionVector: Boolean, + /** + * The raw `DeletionVectorDescriptor` object (opaque via reflection -- the concrete type is + * `org.apache.spark.sql.delta.actions.DeletionVectorDescriptor` but we keep it as `AnyRef` + * to preserve the no-compile-time-dep-on-spark-delta property). `null` when the AddFile has + * no DV. Pass to `materializeDeletedRowIndexes` to convert into a `Array[Long]` of deleted + * row indexes. + */ + dvDescriptor: AnyRef, + /** + * Delta row-tracking fields. `baseRowId` is the first logical row id covered by this file; + * `defaultRowCommitVersion` is the commit that last wrote it. Both are `None` for tables + * that don't have the rowTracking table feature enabled (or for pre-backfill files on a + * table where row tracking was just enabled). + */ + baseRowId: Option[Long], + defaultRowCommitVersion: Option[Long]) + + /** + * Is this FileIndex a pre-materialized Delta index (batch or CDC)? + * + * CDC reads (`CdcAddFileIndex`, `TahoeRemoveFileIndex`, `TahoeChangeFileIndex`) all derive from + * `TahoeBatchFileIndex` (conceptually or concretely) and stash the CDC metadata + * (`_change_type`, `_commit_version`, `_commit_timestamp`) into `AddFile.partitionValues` with + * a matching `partitionSchema`, so the native scan can materialise them as partition columns + * without any special CDC-specific handling. + */ + def isBatchFileIndex(location: Any): Boolean = { + val cls = location.getClass.getName + cls.contains("TahoeBatchFileIndex") || + cls.contains("CdcAddFileIndex") || + cls.contains("TahoeRemoveFileIndex") || + cls.contains("TahoeChangeFileIndex") || + cls.contains("PreparedDeltaFileIndex") + } + + /** + * Extract the AddFile list from a `TahoeBatchFileIndex`-like FileIndex via reflection (no + * compile-time dep on spark-delta). Returns `None` when: + * - the FileIndex class doesn't expose an `addFiles: Seq[AddFile]` method + * - reflection fails for any entry + * - any AddFile's stats / fields can't be read + * + * Callers should fall back to Spark's Delta reader when this returns `None`. + * + * For CDC indexes (`CdcAddFileIndex`, `TahoeRemoveFileIndex`, `TahoeChangeFileIndex`) the raw + * `addFiles` field does NOT contain the CDC metadata columns (`_change_type`, + * `_commit_version`, `_commit_timestamp`); those are injected inside the index's + * `matchingFiles(partitionFilters, dataFilters)` override. We therefore prefer + * `matchingFiles(Seq.empty, Seq.empty)` when it's available, so the returned `partitionValues` + * maps already carry the CDC metadata. + */ + def extractBatchAddFiles(location: Any): Option[Seq[ExtractedAddFile]] = { + try { + // PreparedDeltaFileIndex carries the pre-skipped scan result -- using + // `matchingFiles(Nil, Nil)` on it falls into Delta's "Reselecting files + // to query" branch (different filter set) and returns the FULL snapshot + // of files (no stats-based skipping), which breaks tests like + // StatsCollectionSuite "gather stats" that expect file-level pruning. + // Read `preparedScan.files` directly to honour the prepared skipping. + val preparedFiles: Option[AnyRef] = + if (location.getClass.getName.contains("PreparedDeltaFileIndex")) { + findAccessor(location, Seq("preparedScan")) + .flatMap(ps => findAccessor(ps, Seq("files"))) + } else None + // Prefer matchingFiles(Seq.empty, Seq.empty) -- it returns CDC-augmented + // AddFiles on CDC indexes and the plain list on TahoeBatchFileIndex. + // Fall back to the raw `addFiles`/`filesList` accessors for indexes that + // don't expose a no-arg-safe matchingFiles. + val addFilesOpt = preparedFiles + .orElse(callMatchingFiles(location)) + .orElse(findAccessor(location, Seq("addFiles", "filesList"))) + addFilesOpt.flatMap { addFilesAny => + val seq = addFilesAny match { + case s: scala.collection.Seq[_] => s + case a: Array[_] => a.toSeq + case _ => return None + } + val out = new scala.collection.mutable.ArrayBuffer[ExtractedAddFile](seq.size) + seq.foreach { addFile => + val path = stringMember(addFile, "path").getOrElse(return None) + val size = longMember(addFile, "size").getOrElse(return None) + val rawPV = findAccessor(addFile, Seq("partitionValues")).getOrElse(return None) + val pv: Map[String, String] = rawPV match { + case m: Map[_, _] => m.asInstanceOf[Map[String, String]] + case m: java.util.Map[_, _] => + import scala.jdk.CollectionConverters._ + m.asInstanceOf[java.util.Map[String, String]].asScala.toMap + case _ => return None + } + val stats = stringMember(addFile, "stats").orNull + val dv = findAccessor(addFile, Seq("deletionVector")).orNull + val baseRowId = optionLongMember(addFile, "baseRowId") + val defaultRowCommitVersion = optionLongMember(addFile, "defaultRowCommitVersion") + out += ExtractedAddFile( + path, + size, + pv, + stats, + hasDeletionVector = dv != null, + dvDescriptor = dv, + baseRowId = baseRowId, + defaultRowCommitVersion = defaultRowCommitVersion) + } + Some(out.toSeq) + } + } catch { + case e: Exception => + logWarning( + s"Failed to extract AddFiles from ${location.getClass.getName}: ${e.getMessage}") + None + } + } + + /** + * Materialize a `DeletionVectorDescriptor` into the list of deleted row indexes (0-based, + * sorted ascending) using Delta's own `HadoopFileSystemDVStore` + `RoaringBitmapArray.toArray`. + * + * Returns `None` when: + * - `dvDescriptor` is null (no DV on this file) + * - the Delta classes aren't on the classpath (different Delta version layout, etc.) + * - the read itself fails (corrupt DV file, missing file, etc.) + * + * Callers that need DV semantics must fall back to Spark+Delta when this returns `None`. + * + * Driver-side only: don't call this on executors, since it touches the filesystem and the DV + * store may not be initialised. The native side then plumbs the row-index array into the proto + * task's `deleted_row_indexes` field, which `DeltaDvFilterExec` already consumes. + */ + def materializeDeletedRowIndexes( + dvDescriptor: AnyRef, + tableRoot: String, + hadoopConf: org.apache.hadoop.conf.Configuration): Option[Array[Long]] = { + if (dvDescriptor == null) return None + try { + // scalastyle:off classforname + val storeCls = + Class.forName("org.apache.spark.sql.delta.storage.dv.HadoopFileSystemDVStore") + // scalastyle:on classforname + val store = storeCls + .getConstructor(classOf[org.apache.hadoop.conf.Configuration]) + .newInstance(hadoopConf) + .asInstanceOf[AnyRef] + val readMethod = storeCls.getMethods + .find { m => + m.getName == "read" && + m.getParameterCount == 2 && + m.getParameterTypes()(1) == classOf[org.apache.hadoop.fs.Path] + } + .getOrElse(return None) + val tablePath = new org.apache.hadoop.fs.Path(tableRoot) + val bitmap = readMethod.invoke(store, dvDescriptor, tablePath) + // RoaringBitmapArray.toArray returns Array[Long] of all set bits (= deleted row indexes). + val toArrayMethod = bitmap.getClass.getMethod("toArray") + val indexes = toArrayMethod.invoke(bitmap).asInstanceOf[Array[Long]] + Some(indexes) + } catch { + case scala.util.control.NonFatal(e) => + logWarning(s"materializeDeletedRowIndexes failed for table $tableRoot: ${e.getMessage}") + None + } + } + + /** + * Extract number-of-records from an AddFile's `stats` JSON. Returns `None` if stats is missing + * / malformed. The JSON structure is stable across Delta versions: `{"numRecords": N, ...}`. + */ + def parseNumRecords(statsJson: String): Option[Long] = { + if (statsJson == null) return None + val idx = statsJson.indexOf("\"numRecords\"") + if (idx < 0) return None + // Find the colon after the key, then the first numeric sequence. + val colon = statsJson.indexOf(':', idx) + if (colon < 0) return None + var i = colon + 1 + while (i < statsJson.length && !statsJson.charAt(i).isDigit && statsJson.charAt(i) != '-') { + i += 1 + } + val start = i + while (i < statsJson.length && (statsJson.charAt(i).isDigit || statsJson.charAt(i) == '-')) { + i += 1 + } + if (start == i) { + None + } else { + try Some(statsJson.substring(start, i).toLong) + catch { case _: NumberFormatException => None } + } + } + + /** + * Invoke `FileIndex.matchingFiles(partitionFilters: Seq[Expression], dataFilters: + * Seq[Expression]): Seq[AddFile]` with empty filter sequences via reflection. + * + * Returns `None` if the method is missing or the invocation throws. Comet does not have a + * compile-time dep on spark-delta, so we reach for reflection here. + */ + private def callMatchingFiles(location: Any): Option[AnyRef] = { + if (location == null) return None + try { + // Method.matchingFiles has two parameters of type `Seq[Expression]`; we + // can pass Nil for both. We find the method by name + arity to keep the + // lookup tolerant of Scala's generic-erasure bridging. + val candidate = location.getClass.getMethods.find { m => + m.getName == "matchingFiles" && m.getParameterCount == 2 + } + candidate.flatMap { m => + val nil = scala.collection.immutable.Nil + try Option(m.invoke(location, nil, nil)) + catch { + case scala.util.control.NonFatal(_) => None + } + } + } catch { + case scala.util.control.NonFatal(_) => None + } + } + + private def findAccessor(obj: Any, names: Seq[String]): Option[AnyRef] = { + if (obj == null) return None + val cls = obj.getClass + names.foreach { n => + val m = lookupNoArgMethod(cls, n) + if (m != null) { + try return Option(m.invoke(obj)) + catch { case scala.util.control.NonFatal(_) => return None } + } + } + None + } + + // Cache no-arg java.lang.reflect.Method handles by (class, name). Hot path for plan + // walks: every CometScanRule call into Delta does many name-based lookups per file. + // `MISSING` sentinel caches negative lookups so we don't re-scan getMethods on misses. + private val MISSING: java.lang.reflect.Method = classOf[Object].getMethod("toString") + private val noArgMethodCache = + new java.util.concurrent.ConcurrentHashMap[(Class[_], String), java.lang.reflect.Method]() + + private def lookupNoArgMethod(cls: Class[_], name: String): java.lang.reflect.Method = { + val key = (cls, name) + val cached = noArgMethodCache.get(key) + if (cached ne null) return if (cached eq MISSING) null else cached + val resolved = + try { + val m = cls.getMethod(name) + if (m.getParameterCount == 0) m else null + } catch { + case _: NoSuchMethodException => null + } + noArgMethodCache.putIfAbsent(key, if (resolved == null) MISSING else resolved) + resolved + } + + private def stringMember(obj: Any, name: String): Option[String] = + findAccessor(obj, Seq(name)).flatMap { + case s: String => Some(s) + case null => None + case _ => None + } + + private def longMember(obj: Any, name: String): Option[Long] = + findAccessor(obj, Seq(name)).flatMap { + case l: java.lang.Long => Some(l) + case i: java.lang.Integer => Some(i.toLong) + case _ => None + } + + /** + * Read a Scala `Option[Long]` (or `Option[java.lang.Long]`) field by name. Returns `None` for + * both `None` and a field that contains `Some(null)`. Used for optional Delta fields like + * `AddFile.baseRowId` that only exist when rowTracking is enabled on the table. + */ + private def optionLongMember(obj: Any, name: String): Option[Long] = + findAccessor(obj, Seq(name)).flatMap { + case None => None + case Some(l: java.lang.Long) => Some(l) + case Some(i: java.lang.Integer) => Some(i.toLong) + case Some(l: Long) => Some(l) + case Some(null) | null => None + case l: java.lang.Long => Some(l) // defensive: caller extracted value already + case _ => None + } + + def castPartitionString( + str: Option[String], + dt: org.apache.spark.sql.types.DataType, + sessionZoneId: java.time.ZoneId = java.time.ZoneOffset.UTC): Any = { + import org.apache.spark.sql.catalyst.util.DateTimeUtils + import org.apache.spark.sql.types._ + import org.apache.spark.unsafe.types.UTF8String + str match { + case None | Some(null) => null + case Some(s) => + try { + dt match { + case StringType => UTF8String.fromString(s) + case IntegerType => s.toInt + case LongType => s.toLong + case ShortType => s.toShort + case ByteType => s.toByte + case FloatType => s.toFloat + case DoubleType => s.toDouble + case BooleanType => s.toBoolean + case DateType => + DateTimeUtils + .stringToDate(UTF8String.fromString(s)) + .getOrElse(null) + case _: TimestampType => + // Delta serializes TIMESTAMP partition values in the session TZ at write time, so + // parse them in the session TZ at read time to round-trip correctly (defaults to + // UTC when the caller hasn't plumbed the session TZ through). + DateTimeUtils + .stringToTimestamp(UTF8String.fromString(s), sessionZoneId) + .getOrElse(null) + case _: TimestampNTZType => + DateTimeUtils + .stringToTimestampWithoutTimeZone(UTF8String.fromString(s)) + .getOrElse(null) + case d: DecimalType => + val dec = + org.apache.spark.sql.types.Decimal(new java.math.BigDecimal(s)) + dec.changePrecision(d.precision, d.scale) + dec + case _ => UTF8String.fromString(s) + } + } catch { + case _: NumberFormatException | _: IllegalArgumentException => + null + } + } + } +} diff --git a/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaScanRule.scala b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaScanRule.scala new file mode 100644 index 0000000000..52854fec99 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/DeltaScanRule.scala @@ -0,0 +1,601 @@ +/* + * 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. + */ + +package org.apache.comet.contrib.delta + +import java.util.Locale + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Attribute, AttributeReference, Coalesce, EqualTo, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, Literal} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.comet.CometScanExec +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, DataType, LongType, MapType, StructField, StructType} + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} +import org.apache.comet.rules.CometScanRule + +/** + * `CometScanRuleExtension` for Delta tables. + * + * Three responsibilities, ported from the pre-SPI `delta-kernel-phase-1` branch's in-core + * `CometScanRule.scala`: + * + * - [[preTransform]] runs `stripDeltaDvWrappers` -- undoes Delta's `PreprocessTableWithDVs` + * Catalyst-strategy rewrite for DV-bearing scans so the clean scan reaches [[transformV1]]. + * Some scans must stay Spark-native (Delta's reader synthesises a + * `__delta_internal_is_row_deleted` column Comet's reader can't); those are tagged with + * [[DvProtectedTag]] for `transformV1` to decline. + * - [[matchesV1]] probes the relation's file format via reflection (no compile-time + * `io.delta.spark` dependency required). + * - [[transformV1]] runs `nativeDeltaScan`: schema / encryption / parquet-field-ID gates, + * column-mapping metadata re-attachment, row-tracking rewrite, and finally + * `CometScanExec(scan, session, CometDeltaNativeScan.ScanImpl)`. [[CometExecRule]] picks up + * the marker via [[DeltaOperatorSerdeExtension.matchOperator]] and routes it through + * [[CometDeltaNativeScan]]. + * + * SPI surfaces used: + * - `CometScanRule.isSchemaSupported` (private[comet]) -- avoids duplicating ~25 lines of + * schema check + fallback-reason emission. + * - `CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported}` -- same. + * - `CometSparkSessionExtensions.withInfo` -- same. + * - Spark TreeNodeTag for cross-method (preTransform -> transformV1) state passing. + * + * The mutable.Set[FileSourceScanExec] of dv-protected scans on the pre-SPI branch is replaced + * with the TreeNodeTag mechanism, which is the SPI's documented pattern. + */ +/** + * Static entry points for Delta scan detection / transformation. Called via reflection + * from core's `org.apache.comet.rules.DeltaIntegration` only when the contrib's classes + * are bundled into `comet-spark.jar` (i.e. when Maven was invoked with `-Pcontrib-delta`). + */ +object DeltaScanRule { + + import DeltaScanRuleExtension._ + + /** Convenience: returns `Some(plan)` if this is a Delta scan we handled. */ + def transformV1IfDelta( + plan: SparkPlan, + session: SparkSession, + scanExec: FileSourceScanExec, + relation: HadoopFsRelation): Option[SparkPlan] = { + if (!DeltaReflection.isDeltaFileFormat(relation.fileFormat)) return None + val pre = preTransform(plan, session) + val target = pre.find(_.fastEquals(scanExec)).getOrElse(scanExec).asInstanceOf[FileSourceScanExec] + transformV1(pre, target, session) + } + + def preTransform(plan: SparkPlan, session: SparkSession): SparkPlan = { + if (!DeltaConf.COMET_DELTA_NATIVE_ENABLED.get()) return plan + stripDeltaDvWrappers(plan) + } + + private def stripDeltaDvWrappers(plan: SparkPlan): SparkPlan = { + plan.transformUp { + case proj @ ProjectExec(projectList, FilterExec(cond, inner)) + if isDeltaDvFilterPattern(cond) => + val userOutput = projectList.map(_.toAttribute) + if (scanBelowFallsBackForDvs(inner)) { + collectDeltaScanBelow(inner).foreach(_.setTagValue(DvProtectedTag, ())) + proj + } else { + findAndStripDeltaScanBelow(inner, userOutput).getOrElse(proj) + } + } + } + + private def collectDeltaScanBelow(plan: SparkPlan): Option[FileSourceScanExec] = plan match { + case scan: FileSourceScanExec + if DeltaReflection.isDeltaFileFormat(scan.relation.fileFormat) => + Some(scan) + case other if other.children.size == 1 => collectDeltaScanBelow(other.children.head) + case _ => None + } + + /** + * True when the child subtree contains a Delta `FileSourceScanExec` Comet's native path will + * not apply the DV on. Two shapes both fall back: `TahoeBatchFileIndex` with DV-bearing + * AddFiles, and any Delta scan whose schema already contains the synthetic + * `__delta_internal_is_row_deleted` column. + */ + private def scanBelowFallsBackForDvs(plan: SparkPlan): Boolean = { + def check(p: SparkPlan): Boolean = p match { + case scan: FileSourceScanExec + if DeltaReflection.isDeltaFileFormat(scan.relation.fileFormat) => + val batchFallback = + DeltaReflection.isBatchFileIndex(scan.relation.location) && + DeltaReflection + .extractBatchAddFiles(scan.relation.location) + .exists(_.exists(_.hasDeletionVector)) + val outputHasIsRowDeleted = + scan.output.exists(_.name.equalsIgnoreCase(DeltaReflection.IsRowDeletedColumnName)) || + scan.requiredSchema.fieldNames.exists( + _.equalsIgnoreCase(DeltaReflection.IsRowDeletedColumnName)) + batchFallback || outputHasIsRowDeleted + case other if other.children.size == 1 => check(other.children.head) + case _ => false + } + check(plan) + } + + /** Matches `__delta_internal_is_row_deleted = 0` (the filter Delta injects). */ + private def isDeltaDvFilterPattern(cond: Expression): Boolean = { + def isRowDeletedRef(name: String): Boolean = + name.equalsIgnoreCase(DeltaReflection.IsRowDeletedColumnName) + cond match { + case EqualTo(attr: AttributeReference, lit: Literal) if isRowDeletedRef(attr.name) => + lit.value != null && lit.value.toString == "0" + case EqualTo(lit: Literal, attr: AttributeReference) if isRowDeletedRef(attr.name) => + lit.value != null && lit.value.toString == "0" + case _ => false + } + } + + private def findAndStripDeltaScanBelow( + plan: SparkPlan, + userOutput: Seq[Attribute]): Option[SparkPlan] = plan match { + case scan: FileSourceScanExec + if DeltaReflection.isDeltaFileFormat(scan.relation.fileFormat) && + scan.output.exists(_.name.equalsIgnoreCase(DeltaReflection.IsRowDeletedColumnName)) => + Some(rebuildDeltaScanWithoutDvColumn(scan, userOutput)) + case other if other.children.size == 1 => + findAndStripDeltaScanBelow(other.children.head, userOutput) + case _ => None + } + + private def rebuildDeltaScanWithoutDvColumn( + scan: FileSourceScanExec, + userOutput: Seq[Attribute]): FileSourceScanExec = { + val dvName = DeltaReflection.IsRowDeletedColumnName + val scanByExprId = scan.output.map(a => a.exprId -> a).toMap + val scanByName = scan.output.map(a => a.name.toLowerCase(Locale.ROOT) -> a).toMap + val resolved = userOutput.map { u => + scanByExprId + .get(u.exprId) + .orElse(scanByName.get(u.name.toLowerCase(Locale.ROOT))) + .getOrElse(u) + } + val newOutput = resolved.filterNot(_.name == dvName) + val newRequiredSchema = + StructType(newOutput.map(a => StructField(a.name, a.dataType, a.nullable))) + val newDataSchema = + StructType(scan.relation.dataSchema.fields.filterNot(_.name == dvName)) + val newRelation = scan.relation.copy(dataSchema = newDataSchema)(scan.relation.sparkSession) + val newDataFilters = scan.dataFilters.filterNot { f => + f.references.exists(_.name == dvName) + } + scan.copy( + relation = newRelation, + output = newOutput, + requiredSchema = newRequiredSchema, + dataFilters = newDataFilters) + } + + // =========================================================================== + // transformV1: nativeDeltaScan body. + // =========================================================================== + + def transformV1( + plan: SparkPlan, + scanExec: FileSourceScanExec, + session: SparkSession): Option[SparkPlan] = { + if (scanExec.getTagValue(DvProtectedTag).isDefined) { + withInfo( + scanExec, + "Leaving scan to Delta so its DV filter above can apply deletion vectors") + return None + } + // Detect references to `input_file_name()` / `input_file_block_*` anywhere in the + // surrounding plan tree. When present, the contrib's serde MUST emit one task per + // partition so `CometExecRDD`'s per-partition `InputFileBlockHolder` hook attributes + // every row to the correct file path. Delta's UPDATE/DELETE/MERGE flows use + // `input_file_name()` to find "touched files"; without this tag, multiple files + // packed into one Spark partition share the FIRST task's file path, and Delta + // rewrites the wrong files (or fails to find rows to rewrite at all). Triggered by + // tests like `UpdateBaseMiscTests "data and partition predicates - + // Partition=true Skipping=false"` which has multiple files in one partition. + val needsInputFileName = plan.exists { node => + node.expressions.exists(_.exists { + case _: InputFileName | _: InputFileBlockStart | _: InputFileBlockLength => true + case _ => false + }) + } + val scanForDelta = if (needsInputFileName) { + val taggedOptions = scanExec.relation.options + + (DeltaConf.NeedsInputFileNameOption -> "true") + val taggedRelation = scanExec.relation.copy(options = taggedOptions)( + scanExec.relation.sparkSession) + scanExec.copy(relation = taggedRelation) + } else scanExec + nativeDeltaScan(session, scanForDelta, scanForDelta.relation) + } + + private def nativeDeltaScan( + session: SparkSession, + scanExec: FileSourceScanExec, + r: HadoopFsRelation): Option[SparkPlan] = { + if (!DeltaConf.COMET_DELTA_NATIVE_ENABLED.get()) { + withInfo( + scanExec, + s"Native Delta scan disabled because ${DeltaConf.COMET_DELTA_NATIVE_ENABLED.key} " + + "is not enabled") + return None + } + if (!CometConf.COMET_EXEC_ENABLED.get()) { + withInfo( + scanExec, + s"Native Delta scan requires ${CometConf.COMET_EXEC_ENABLED.key} to be enabled") + return None + } + val hadoopConf = r.sparkSession.sessionState.newHadoopConfWithOptions(r.options) + if (encryptionEnabled(hadoopConf) && !isEncryptionConfigSupported(hadoopConf)) { + withInfo(scanExec, s"${CometDeltaNativeScan.ScanImpl} does not support encryption config") + return None + } + // CometScanRule.isSchemaSupported is private[comet]; inline the equivalent check + // (schema check + fallback-reason emission) for the contrib's needs. + if (!isSchemaCometCompatible(scanExec, r)) { + return None + } + if (session.sessionState.conf.getConf(SQLConf.PARQUET_FIELD_ID_READ_ENABLED) && + ParquetUtils.hasFieldIds(scanExec.requiredSchema)) { + withInfo( + scanExec, + s"${CometDeltaNativeScan.ScanImpl} does not support Parquet field ID matching") + return None + } + val cmMode = DeltaReflection + .extractMetadataConfiguration(r) + .flatMap(_.get("delta.columnMapping.mode")) + if (cmMode.exists(_.equalsIgnoreCase("id"))) { + withInfo( + scanExec, + s"${CometDeltaNativeScan.ScanImpl} does not support Delta column-mapping 'id' mode " + + "(parquet field-ID resolution required)") + return None + } + if (cmMode.exists(_.equalsIgnoreCase("name")) && + !session.sessionState.conf + .getConfString("spark.databricks.delta.checkLatestSchemaOnRead", "true") + .equalsIgnoreCase("true")) { + withInfo( + scanExec, + s"${CometDeltaNativeScan.ScanImpl} declines CM-name reads when " + + "checkLatestSchemaOnRead is disabled (potential stale-snapshot read)") + return None + } + val fileIndexClassName = r.location.getClass.getName + if (fileIndexClassName.endsWith(".TahoeLogFileIndexWithCloudFetch")) { + withInfo( + scanExec, + s"Native Delta scan has not validated the cloud-fetch variant ($fileIndexClassName).") + return None + } + val supportedSchemes = + Set("file", "s3", "s3a", "gs", "gcs", "abfss", "abfs", "wasbs", "wasb", "oss") + val rootPaths = scanExec.relation.location.rootPaths + if (rootPaths.nonEmpty) { + val schemes = rootPaths.map(p => p.toUri.getScheme).filter(_ != null).toSet + val unsupported = schemes -- supportedSchemes + if (unsupported.nonEmpty) { + withInfo( + scanExec, + s"Native Delta scan does not support filesystem schemes: " + unsupported.mkString(", ")) + return None + } + } + if (r.location.getClass.getName.contains("PreparedDeltaFileIndex")) { + try { + val sample = r.location.inputFiles.take(2) + sample.foreach { p => + val colonSlash = p.indexOf(":/") + if (colonSlash >= 0) { + val afterColon = p.substring(colonSlash + 1) + val scheme = p.substring(0, colonSlash) + if (!afterColon.startsWith("//") && scheme != "file") { + withInfo( + scanExec, + s"Native Delta scan declines: file path '$p' uses malformed URL form " + + s"'$scheme:/...' (real URLs are 'scheme://...'); likely a test-only " + + s"shallow-clone mock or cross-filesystem clone our reader can't open.") + return None + } + } + } + } catch { + case scala.util.control.NonFatal(_) => // best-effort; fall through + } + } + val scanWithMappedSchema = withDeltaColumnMappingMetadata(scanExec) + val hasDeltaSyntheticCol = scanExec.output.exists { a => + a.name.equalsIgnoreCase(DeltaReflection.IsRowDeletedColumnName) || + a.name.equalsIgnoreCase(DeltaReflection.RowIndexColumnName) + } + if (hasDeltaSyntheticCol) { + withInfo( + scanExec, + "Native Delta scan declines reads that carry Delta's synthetic " + + "__delta_internal_is_row_deleted / __delta_internal_row_index columns in their " + + "output -- those are produced only by Delta's reader.") + return None + } + applyRowTrackingRewrite(scanWithMappedSchema, r, session).getOrElse { + Some(CometScanExec(scanWithMappedSchema, session, CometDeltaNativeScan.ScanImpl)) + } + } + + private def withDeltaColumnMappingMetadata(scanExec: FileSourceScanExec): FileSourceScanExec = { + val r = scanExec.relation + val snapshotSchemaOpt = DeltaReflection.extractSnapshotSchema(r) + if (snapshotSchemaOpt.isEmpty) return scanExec + val snapshotByName: Map[String, StructField] = + snapshotSchemaOpt.get.fields.map(f => f.name -> f).toMap + def attach(f: StructField): StructField = + snapshotByName.get(f.name) match { + case Some(meta) => + StructField( + f.name, + attachDataType(f.dataType, meta.dataType), + f.nullable, + meta.metadata) + case None => f + } + def attachDataType(child: DataType, withMeta: DataType): DataType = (child, withMeta) match { + case (cs: StructType, ms: StructType) => + val metaByName = ms.fields.map(f => f.name -> f).toMap + StructType(cs.fields.map { f => + metaByName.get(f.name) match { + case Some(mf) => + StructField( + f.name, + attachDataType(f.dataType, mf.dataType), + f.nullable, + mf.metadata) + case None => f + } + }) + case (ca: ArrayType, ma: ArrayType) => + ArrayType(attachDataType(ca.elementType, ma.elementType), ca.containsNull) + case (cm: MapType, mm: MapType) => + MapType( + attachDataType(cm.keyType, mm.keyType), + attachDataType(cm.valueType, mm.valueType), + cm.valueContainsNull) + case _ => child + } + val newDataFields = r.dataSchema.fields.map(attach) + val newRequiredFields = scanExec.requiredSchema.fields.map(attach) + val anyChange = !newDataFields.sameElements(r.dataSchema.fields) || + !newRequiredFields.sameElements(scanExec.requiredSchema.fields) + if (!anyChange) return scanExec + val newRelation = r.copy(dataSchema = StructType(newDataFields))(r.sparkSession) + scanExec.copy(relation = newRelation, requiredSchema = StructType(newRequiredFields)) + } + + /** + * Returns `Some(Some(plan))` when a row-tracking rewrite was applied, `Some(None)` when we + * detected row-tracking columns we can't translate, and `None` when the scan has no + * row-tracking columns. Caller uses the outer Option to distinguish "applied" / "decline" / "no + * rewrite needed". + */ + private def applyRowTrackingRewrite( + scanExec: FileSourceScanExec, + r: HadoopFsRelation, + session: SparkSession): Option[Option[SparkPlan]] = { + val RowIdName = "row_id" + val RowCommitVersionName = "row_commit_version" + val hasRowIdField = scanExec.requiredSchema.fieldNames.exists { n => + n.equalsIgnoreCase(RowIdName) || n.equalsIgnoreCase(RowCommitVersionName) + } + if (!hasRowIdField) return None + + val cfg = DeltaReflection.extractMetadataConfiguration(r).getOrElse(Map.empty) + if (cfg.get("delta.enableRowTracking").exists(_.equalsIgnoreCase("false"))) { + withInfo( + scanExec, + "Native Delta scan: row-tracking columns requested but table has " + + "delta.enableRowTracking=false; falling back.") + return Some(None) + } + val rowIdPhysical = cfg.get(DeltaReflection.MaterializedRowIdColumnProp) + val rowVerPhysical = cfg.get(DeltaReflection.MaterializedRowCommitVersionColumnProp) + if (rowIdPhysical.isEmpty && rowVerPhysical.isEmpty) { + withInfo( + scanExec, + "Native Delta scan: row-tracking columns present but no materialised column " + + "names in Delta metadata; synthesis from baseRowId + row_index is Phase 3.") + return Some(None) + } + + val renames = scala.collection.mutable.ArrayBuffer.empty[(String, String)] + def physicalFor(logical: String): Option[String] = + if (logical.equalsIgnoreCase(RowIdName)) rowIdPhysical + else if (logical.equalsIgnoreCase(RowCommitVersionName)) rowVerPhysical + else None + + val newRequiredFields = scanExec.requiredSchema.fields.map { f => + physicalFor(f.name) match { + case Some(phys) => + renames += ((f.name, phys)) + StructField(phys, f.dataType, nullable = true, f.metadata) + case None => f + } + } + val newDataFields = r.dataSchema.fields.map { f => + physicalFor(f.name) match { + case Some(phys) => StructField(phys, f.dataType, nullable = true, f.metadata) + case None => f + } + } + if (renames.isEmpty) return None + + val RowIndexColName = "_tmp_metadata_row_index" + val BaseRowIdColName = "__comet_base_row_id" + val DefaultRowCommitVersionColName = "__comet_default_row_commit_version" + val includeRowIdSynth = renames.exists { case (logical, _) => + logical.equalsIgnoreCase(RowIdName) + } + val includeRowVerSynth = renames.exists { case (logical, _) => + logical.equalsIgnoreCase(RowCommitVersionName) + } + val needSynth = includeRowIdSynth || includeRowVerSynth + + if (needSynth) { + val existingNames = + (r.dataSchema.fieldNames ++ r.partitionSchema.fieldNames) + .map(_.toLowerCase(Locale.ROOT)) + .toSet + val syntheticNames = Seq(RowIndexColName, BaseRowIdColName, DefaultRowCommitVersionColName) + val collisions = + syntheticNames.filter(n => existingNames.contains(n.toLowerCase(Locale.ROOT))) + if (collisions.nonEmpty) { + withInfo( + scanExec, + s"Native Delta scan: table has columns that collide with Comet row-tracking " + + s"synthetic columns (${collisions.mkString(", ")}); falling back.") + return Some(None) + } + } + + val infoByFileName: Map[String, DeltaReflection.RowTrackingFileInfo] = + if (needSynth) DeltaReflection.extractRowTrackingInfoByFileName(r.location) + else Map.empty + + val extraRequiredFields = scala.collection.mutable.ArrayBuffer.empty[StructField] + val extraDataFields = scala.collection.mutable.ArrayBuffer.empty[StructField] + if (includeRowIdSynth) { + val rowIndexField = StructField(RowIndexColName, LongType, nullable = true) + extraRequiredFields += rowIndexField + extraDataFields += rowIndexField + } + + val finalRequiredSchema = StructType(newRequiredFields ++ extraRequiredFields) + val finalDataSchema = StructType(newDataFields ++ extraDataFields) + val finalLocation = + if (needSynth) { + new RowTrackingAugmentedFileIndex( + r.location, + infoByFileName, + BaseRowIdColName, + DefaultRowCommitVersionColName) + } else { + r.location + } + + val origOutput = scanExec.output + val renameMap: Map[String, String] = renames.toMap + val baseNewOutput = origOutput.map { a => + renameMap.get(a.name) match { + case Some(phys) => + AttributeReference(phys, a.dataType, nullable = true, a.metadata)(qualifier = + a.qualifier) + case None => a + } + } + val rowIndexAttr = AttributeReference(RowIndexColName, LongType, nullable = true)() + val baseRowIdAttr = AttributeReference(BaseRowIdColName, LongType, nullable = true)() + val defaultVerAttr = + AttributeReference(DefaultRowCommitVersionColName, LongType, nullable = true)() + val extraOutputAttrs = scala.collection.mutable.ArrayBuffer.empty[AttributeReference] + if (includeRowIdSynth) extraOutputAttrs += rowIndexAttr + if (needSynth) { + extraOutputAttrs += baseRowIdAttr + extraOutputAttrs += defaultVerAttr + } + val newOutput: Seq[Attribute] = + if (extraOutputAttrs.isEmpty) baseNewOutput + else baseNewOutput ++ extraOutputAttrs + + val newPartitionSchema = + if (needSynth) { + r.partitionSchema + .add(StructField(BaseRowIdColName, LongType, nullable = true)) + .add(StructField(DefaultRowCommitVersionColName, LongType, nullable = true)) + } else { + r.partitionSchema + } + + val newRelation = r.copy( + location = finalLocation, + dataSchema = finalDataSchema, + partitionSchema = newPartitionSchema)(r.sparkSession) + val newScan = scanExec.copy( + relation = newRelation, + output = newOutput, + requiredSchema = finalRequiredSchema) + val cometScan = CometScanExec(newScan, session, CometDeltaNativeScan.ScanImpl) + + val projectExprs = origOutput.map { a => + renameMap.get(a.name).flatMap(phys => baseNewOutput.find(_.name == phys)) match { + case Some(physAttr) if a.name.equalsIgnoreCase(RowIdName) && includeRowIdSynth => + val synth = Add(baseRowIdAttr, rowIndexAttr) + Alias(Coalesce(Seq(physAttr, synth)), a.name)( + exprId = a.exprId, + qualifier = a.qualifier) + case Some(physAttr) + if a.name.equalsIgnoreCase(RowCommitVersionName) && includeRowVerSynth => + Alias(Coalesce(Seq(physAttr, defaultVerAttr)), a.name)( + exprId = a.exprId, + qualifier = a.qualifier) + case Some(physAttr) => + Alias(physAttr, a.name)(exprId = a.exprId, qualifier = a.qualifier) + case None => a + } + } + Some(Some(ProjectExec(projectExprs, cometScan))) + } + + /** + * Inline schema check + fallback-reason emission, mirroring core's + * `private[comet] CometScanRule.isSchemaSupported`. Kept local to the contrib so the + * contrib doesn't need to widen core's visibility. + */ + // Reused across scans -- CometScanTypeChecker is stateless w.r.t. its scanImpl and + // doesn't mutate per-call state; the per-scan fallback-reasons ListBuffer is the only + // per-call mutable input. + private val typeChecker = + org.apache.comet.rules.CometScanTypeChecker(CometDeltaNativeScan.ScanImpl) + + private def isSchemaCometCompatible( + scanExec: FileSourceScanExec, + r: HadoopFsRelation): Boolean = { + val fallbackReasons = new scala.collection.mutable.ListBuffer[String]() + val ok = typeChecker.isSchemaSupported(scanExec.requiredSchema, fallbackReasons) && + typeChecker.isSchemaSupported(r.partitionSchema, fallbackReasons) + if (!ok) withInfo(scanExec, fallbackReasons.mkString("; ")) + ok + } +} + +/** Companion holding plan-tree tags used by the static `DeltaScanRule` object. */ +object DeltaScanRuleExtension { + + /** + * Plan-tree tag attached during `preTransform` to mark `FileSourceScanExec`s whose native + * conversion `transformV1` must decline -- Comet's reader can't produce the + * `__delta_internal_is_row_deleted` column the outer DV-filter wrapper requires. + */ + val DvProtectedTag: TreeNodeTag[Unit] = + TreeNodeTag[Unit]("org.apache.comet.contrib.delta.dv_protected") +} diff --git a/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/Native.scala b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/Native.scala new file mode 100644 index 0000000000..dc2f714fd4 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/Native.scala @@ -0,0 +1,61 @@ +/* + * 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. + */ + +package org.apache.comet.contrib.delta + +import org.apache.comet.NativeBase + +/** + * Contrib-local JVM handle to the Delta-specific native entry point. + * + * Extends `NativeBase` so the libcomet load triggers on first use of any subclass -- the contrib + * doesn't reload the library itself (there is exactly one libcomet at runtime), but inheriting + * from `NativeBase` ensures the static initializer ordering works the same way as core's + * `org.apache.comet.Native`. The `@native` method below binds to + * `Java_org_apache_comet_contrib_delta_Native_planDeltaScan` exported by the contrib's Rust crate + * (compiled INTO libcomet via the `contrib-delta` Cargo feature on `native/core`). + */ +class Native extends NativeBase { + + /** + * Driver-side Delta log replay. Returns a prost-encoded `DeltaScanTaskList` proto (raw bytes) + * which the caller decodes via `DeltaScanTaskList.parseFrom(...)`. + * + * @param tableUrl + * absolute URL or bare path of the Delta table root + * @param snapshotVersion + * `-1` for the latest snapshot, otherwise an exact version + * @param storageOptions + * cloud credentials / endpoint overrides (Hadoop-style keys) + * @param predicateBytes + * prost-encoded Catalyst data filter for kernel-side stats-based file pruning, or an empty + * array for no predicate + * @param columnNames + * logical column names the caller requires (kernel uses this for column-mapping resolution + * before stats-based file pruning). + * @return + * `byte[]` containing the encoded DeltaScanTaskList + */ + @native def planDeltaScan( + tableUrl: String, + snapshotVersion: Long, + storageOptions: java.util.Map[String, String], + predicateBytes: Array[Byte], + columnNames: Array[String]): Array[Byte] +} diff --git a/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/RowTrackingAugmentedFileIndex.scala b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/RowTrackingAugmentedFileIndex.scala new file mode 100644 index 0000000000..e58ef557e3 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/comet/contrib/delta/RowTrackingAugmentedFileIndex.scala @@ -0,0 +1,113 @@ +/* + * 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. + */ + +package org.apache.comet.contrib.delta + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow} +import org.apache.spark.sql.execution.datasources.{FileIndex, PartitionDirectory} +import org.apache.spark.sql.types.{LongType, StructField, StructType} + +/** + * Wraps a Delta `FileIndex` and augments each per-file `PartitionDirectory` with two synthetic + * partition values drawn from the corresponding Delta `AddFile`: + * + * - `baseRowIdColumnName` <- `AddFile.baseRowId` + * - `defaultRowCommitVersionColumnName` <- `AddFile.defaultRowCommitVersion` + * + * Used by Comet's row-tracking phase 3: `_row_id_` and `_row_commit_version_` values that are + * still null in their materialised physical columns get synthesised by the outer Project as: + * + * row_id = coalesce(materialised_row_id, baseRowIdCol + _tmp_metadata_row_index) + * row_commit_version = coalesce(materialised_row_commit_version, defaultRowCommitVersionCol) + * + * once the scan can see the per-file values as constant columns. + * + * The map from file basename (tail of `AddFile.path`) to `RowTrackingFileInfo` is supplied by the + * caller (via reflection on the delegate's `matchingFiles` API) so we don't need a compile-time + * dep on spark-delta. Each listed directory entry is split into one `PartitionDirectory` per file + * so each file's values travel with it. + */ +class RowTrackingAugmentedFileIndex( + delegate: FileIndex, + infoByFileName: Map[String, DeltaReflection.RowTrackingFileInfo], + baseRowIdColumnName: String, + defaultRowCommitVersionColumnName: String) + extends FileIndex { + + override def rootPaths: Seq[Path] = delegate.rootPaths + + override def inputFiles: Array[String] = delegate.inputFiles + + override def refresh(): Unit = delegate.refresh() + + override def sizeInBytes: Long = delegate.sizeInBytes + + /** Appends both synthetic columns as Long, nullable partition columns. */ + override def partitionSchema: StructType = + delegate.partitionSchema + .add(StructField(baseRowIdColumnName, LongType, nullable = true)) + .add(StructField(defaultRowCommitVersionColumnName, LongType, nullable = true)) + + /** + * Delegates listing to the underlying FileIndex, then splits each returned `PartitionDirectory` + * into one-per-file directories, each carrying the original partition values PLUS the per-file + * baseRowId and defaultRowCommitVersion. + * + * The per-file split is unavoidable for correctness: `AddFile.baseRowId` is unique per file, so + * two files that share a Delta partition cannot share a `PartitionDirectory` once we inject the + * per-file synthetic columns. Scheduling parallelism is unaffected -- `FileSourceScanExec` + * flattens all PDs' files into `PartitionedFile`s and bin-packs them by `maxSplitBytes`, so PD + * granularity only governs how partition values get serialised with each file, not the number + * of tasks. + */ + override def listFiles( + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + val underlying = delegate.listFiles(partitionFilters, dataFilters) + underlying.flatMap { pd => + pd.files.map { fileStatus => + val info = infoByFileName.getOrElse( + fileStatus.getPath.getName, + DeltaReflection.RowTrackingFileInfo(None, None)) + // Use `pd.copy(...)` rather than `PartitionDirectory.apply(...)` so this + // compiles against both Spark 3.x (files: Seq[FileStatus]) and Spark 4.0 + // (files: Seq[FileStatusWithMetadata]) without a per-version shim -- we + // round-trip the same element type we got from `pd.files`. + pd.copy(values = augmentPartitionValues(pd.values, info), files = Seq(fileStatus)) + } + } + } + + private def augmentPartitionValues( + original: InternalRow, + info: DeltaReflection.RowTrackingFileInfo): InternalRow = { + val n = original.numFields + val values = new Array[Any](n + 2) + var i = 0 + while (i < n) { + values(i) = original.get(i, delegate.partitionSchema.fields(i).dataType) + i += 1 + } + values(n) = info.baseRowId.map(Long.box).orNull + values(n + 1) = info.defaultRowCommitVersion.map(Long.box).orNull + new GenericInternalRow(values) + } +} diff --git a/contrib/delta/src/main/scala/org/apache/spark/sql/comet/CometDeltaNativeScanExec.scala b/contrib/delta/src/main/scala/org/apache/spark/sql/comet/CometDeltaNativeScanExec.scala new file mode 100644 index 0000000000..decf6d0f83 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/spark/sql/comet/CometDeltaNativeScanExec.scala @@ -0,0 +1,451 @@ +/* + * 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. + */ + +package org.apache.spark.sql.comet + +import java.util.Locale + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.execution.{FileSourceScanExec, InSubqueryExec} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.AccumulatorV2 + +import com.google.common.base.Objects + +import org.apache.comet.serde.OperatorOuterClass +import org.apache.comet.serde.OperatorOuterClass.Operator + +/** + * Native Delta Lake scan operator with split-mode serialization and DPP support. + * + * Common scan metadata (schemas, filters, projections, storage options, column mappings) is + * serialized once at planning time in `nativeOp`. Per-partition file lists are materialized + * lazily in `serializedPartitionData` at execution time so each Spark task receives only its own + * slice of the file list, reducing driver memory. + * + * DPP (Dynamic Partition Pruning) is supported by deferring partition pruning of DPP expressions + * to execution time. Static partition filters are applied at planning time in + * `CometDeltaNativeScan.prunePartitions`; DPP filters are resolved in `serializedPartitionData`. + */ +case class CometDeltaNativeScanExec( + override val nativeOp: Operator, + override val output: Seq[Attribute], + override val serializedPlanOpt: SerializedPlan, + @transient originalPlan: FileSourceScanExec, + tableRoot: String, + @transient taskListBytes: Array[Byte], + @transient dppFilters: Seq[Expression] = Seq.empty, + partitionSchema: StructType = new StructType(), + /** + * #75 design A: when true, `packTasks` emits one group (= one partition) per task so + * `CometExecRDD.compute` (via `InputFileBlockHolder.set`) correctly sets `InputFileBlockHolder` to the + * partition's only file path. Set by `CometDeltaNativeScan.createExec` when the surrounding + * plan references `input_file_name()` / `input_file_block_*`. + */ + oneTaskPerPartition: Boolean = false) + extends CometLeafExec { + + // Per-partition task list bytes are exposed via the public accessors below; core's + // CometExecRDD reads them directly when serialising the Comet plan for execution. + // (Was a PlanDataSource SPI implementation under PR1; the SPI was rejected so we + // expose the helpers as plain methods on this exec class.) + def planDataSourceKey: String = sourceKey + def planDataCommonBytes: Array[Byte] = commonData + def planDataPerPartitionBytes: Array[Array[Byte]] = perPartitionData + + override val supportsColumnar: Boolean = true + + override val nodeName: String = s"CometDeltaNativeScan $tableRoot" + + override protected def doPrepare(): Unit = { + dppFilters.foreach { + case DynamicPruningExpression(e: InSubqueryExec) => + e.plan.prepare() + case _ => + } + super.doPrepare() + } + + @transient private lazy val commonBytes: Array[Byte] = { + // The typed DeltaScan variant of OpStruct carries the common block directly. + nativeOp.getDeltaScan.getCommon.toByteArray + } + + @transient private lazy val allTasks: Seq[OperatorOuterClass.DeltaScanTask] = + OperatorOuterClass.DeltaScanTaskList + .parseFrom(taskListBytes) + .getTasksList + .asScala + .toSeq + + /** + * Synthesise a `Seq[FilePartition]` from this scan's tasks, with each task becoming one + * `PartitionedFile` carrying its partition values as an `InternalRow`. Delta tests (e.g. + * `DeltaSinkSuite`) inspect `executedPlan.collect[DataSourceScanExec]` and read + * `inputRDDs.head.asInstanceOf[FileScanRDD].filePartitions` to verify partition pruning; those + * tests find nothing under Comet because we replace the scan with this exec. The test diff in + * `dev/diffs/delta/.diff` patches the helper to fall back to this accessor, so the + * same partition-pruning assertions pass against Comet's scan. + */ + def synthesizedFilePartitions: Seq[org.apache.spark.sql.execution.datasources.FilePartition] = { + if (allTasks.isEmpty) return Nil + val sessionTz = java.time.ZoneId.of(SQLConf.get.sessionLocalTimeZone) + val files = allTasks.zipWithIndex.map { case (task, _) => + val pvRow = InternalRow.fromSeq(partitionSchema.fields.toSeq.map { f => + val proto = task.getPartitionValuesList.asScala.find(_.getName == f.name) + val s = if (proto.exists(_.hasValue)) Some(proto.get.getValue) else None + org.apache.comet.contrib.delta.DeltaReflection + .castPartitionString(s, f.dataType, sessionTz) + }) + val sparkPath = + org.apache.spark.paths.SparkPath.fromUrlString(task.getFilePath) + org.apache.spark.sql.execution.datasources.PartitionedFile( + partitionValues = pvRow, + filePath = sparkPath, + start = if (task.hasByteRangeStart) task.getByteRangeStart else 0L, + length = { + if (task.hasByteRangeStart && task.hasByteRangeEnd) { + task.getByteRangeEnd - task.getByteRangeStart + } else task.getFileSize + }, + modificationTime = 0L, + fileSize = task.getFileSize) + } + files.zipWithIndex.map { case (pf, i) => + org.apache.spark.sql.execution.datasources.FilePartition(i, Array(pf)) + } + } + + /** + * Build per-partition bytes from the current DPP-pruned task list. DPP filters that are still + * `SubqueryAdaptiveBroadcastExec` placeholders at planning time materialise lazily once AQE + * runs the broadcast; by recomputing this at `doExecuteColumnar` (rather than memoising the + * result in a lazy val) we pick up the resolved values and actually skip partitions, instead of + * reading the full table every time AQE is in the loop. + */ + private def buildPerPartitionBytes(): Array[Array[Byte]] = { + val tasks = + if (dppFilters.nonEmpty && partitionSchema.nonEmpty) applyDppFilters(allTasks) + else allTasks + if (tasks.isEmpty) { + Array.empty[Array[Byte]] + } else { + packTasks(tasks).map { group => + val builder = OperatorOuterClass.DeltaScan.newBuilder() + group.foreach(builder.addTasks) + builder.build().toByteArray + }.toArray + } + } + + // #75 design A: when input_file_name() is needed (signal threaded from CometScanRule + // via CometDeltaNativeScan.createExec into `oneTaskPerPartition`), short-circuit + // packing so each task gets its own partition. `CometExecRDD.compute` reads + // task[0]'s path; with 1 task per partition that path correctly attributes every row. + private def packTasks( + tasks: Seq[OperatorOuterClass.DeltaScanTask]): Seq[Seq[OperatorOuterClass.DeltaScanTask]] = { + if (oneTaskPerPartition) return tasks.map(t => Seq(t)) + val conf = originalPlan.relation.sparkSession.sessionState.conf + val openCostInBytes = conf.filesOpenCostInBytes + val maxPartitionBytes = conf.filesMaxPartitionBytes + val minPartitionNum = conf.filesMinPartitionNum + .getOrElse(originalPlan.relation.sparkSession.sparkContext.defaultParallelism) + def taskSize(t: OperatorOuterClass.DeltaScanTask): Long = { + if (t.hasByteRangeStart && t.hasByteRangeEnd) { + math.max(0L, t.getByteRangeEnd - t.getByteRangeStart) + } else t.getFileSize + } + val totalBytes = tasks.map(t => taskSize(t) + openCostInBytes).sum + val bytesPerCore = totalBytes / math.max(1, minPartitionNum) + val msb = math.min(maxPartitionBytes, math.max(openCostInBytes, bytesPerCore)) + val out = scala.collection.mutable.ArrayBuffer[Seq[OperatorOuterClass.DeltaScanTask]]() + val current = scala.collection.mutable.ArrayBuffer[OperatorOuterClass.DeltaScanTask]() + var currentSize = 0L + tasks.foreach { task => + val size = taskSize(task) + if (currentSize + size > msb && current.nonEmpty) { + out += current.toList + current.clear() + currentSize = 0L + } + current += task + currentSize += size + openCostInBytes + } + if (current.nonEmpty) out += current.toList + out.toSeq + } + + // Planning-time snapshot used by metrics, sourceKey derivations, and `numPartitions`. + // Execution-time recomputation happens inside `doExecuteColumnar`. + @transient private lazy val planningPerPartitionBytes: Array[Array[Byte]] = + buildPerPartitionBytes() + + private def applyDppFilters( + tasks: Seq[OperatorOuterClass.DeltaScanTask]): Seq[OperatorOuterClass.DeltaScanTask] = { + // If any DPP subquery is still a `SubqueryAdaptiveBroadcastExec` placeholder, + // AQE hasn't yet replaced it with the real broadcast plan. We can't execute + // it ourselves (that plan's `doExecute` throws), so skip pruning for this + // batch — the scan just reads all tasks, which is correct but slower. + val hasUnresolvedAdaptive = dppFilters.exists { + case DynamicPruningExpression(inSub: InSubqueryExec) => + inSub.plan.isInstanceOf[org.apache.spark.sql.execution.SubqueryAdaptiveBroadcastExec] + case _ => false + } + if (hasUnresolvedAdaptive) return tasks + dppFilters.foreach { + case DynamicPruningExpression(inSub: InSubqueryExec) if inSub.values().isEmpty => + inSub.updateResult() + case _ => + } + + val resolvedFilters = dppFilters.map { + case DynamicPruningExpression(e) => e + case other => other + } + if (resolvedFilters.isEmpty) return tasks + + val caseSensitive = SQLConf.get.getConf[Boolean](SQLConf.CASE_SENSITIVE) + val combined = resolvedFilters.reduce(And) + val bound = combined.transform { case a: AttributeReference => + val idx = partitionSchema.fields.indexWhere(f => + if (caseSensitive) f.name == a.name + else f.name.toLowerCase(Locale.ROOT) == a.name.toLowerCase(Locale.ROOT)) + if (idx < 0) return tasks + BoundReference(idx, partitionSchema(idx).dataType, partitionSchema(idx).nullable) + } + val predicate = InterpretedPredicate(bound) + predicate.initialize(0) + + val sessionZoneId = java.time.ZoneId.of(SQLConf.get.sessionLocalTimeZone) + tasks.filter { task => + val row = InternalRow.fromSeq(partitionSchema.fields.toSeq.map { field => + val proto = task.getPartitionValuesList.asScala.find(_.getName == field.name) + val strValue = + if (proto.exists(_.hasValue)) Some(proto.get.getValue) else None + org.apache.comet.contrib.delta.DeltaReflection + .castPartitionString(strValue, field.dataType, sessionZoneId) + }) + predicate.eval(row) + } + } + + def commonData: Array[Byte] = commonBytes + def perPartitionData: Array[Array[Byte]] = planningPerPartitionBytes + + /** + * Unique key for matching this scan's common/per-partition data to its operator in the native + * plan. Must be distinct across multiple Delta scans in the same plan tree -- e.g. a self-join + * reading two snapshot versions of the same table, where `tableRoot` alone is not unique. + * + * Derived identically in `DeltaPlanDataInjector.getKey` from the serialized `DeltaScanCommon` + * proto so the driver-side map and the executor-side lookup agree. + * + * Mirrors the pattern used by `CometNativeScanExec.sourceKey`. + */ + def sourceKey: String = CometDeltaNativeScanExec.computeSourceKey(nativeOp) + + def numPartitions: Int = perPartitionData.length + + override lazy val outputPartitioning: Partitioning = + UnknownPartitioning(math.max(1, numPartitions)) + + override lazy val outputOrdering: Seq[SortOrder] = Nil + + private class ImmutableSQLMetric(metricType: String) extends SQLMetric(metricType, 0) { + override def merge(other: AccumulatorV2[Long, Long]): Unit = {} + override def reset(): Unit = {} + } + + override lazy val metrics: Map[String, SQLMetric] = { + val taskList = + if (taskListBytes != null) { + OperatorOuterClass.DeltaScanTaskList.parseFrom(taskListBytes) + } else { + null + } + + // Key these under both the Comet-native-side name (`output_rows`, used by the metric + // collector on the native side) and the Spark streaming ProgressReporter name + // (`numOutputRows`, read by `extractSourceToNumInputRows` to populate + // `q.recentProgress.numInputRows`). Without the `numOutputRows` alias, streaming + // workloads that this scan feeds report 0 input rows per batch even when data flows + // correctly -- DeltaSourceSuiteBase.CheckProgress then fails with + // "Execute: 0 did not equal N Expected batches don't match". + val outputRowsMetric = SQLMetrics.createMetric(sparkContext, "number of output rows") + val baseMetrics = Map( + "output_rows" -> outputRowsMetric, + "numOutputRows" -> outputRowsMetric, + "num_splits" -> SQLMetrics.createMetric(sparkContext, "number of file splits processed")) + + val planningMetrics = if (taskList != null) { + val totalFiles = new ImmutableSQLMetric("sum") + totalFiles.set(taskList.getTasksCount.toLong) + sparkContext.register(totalFiles, "total files") + + val dvFiles = new ImmutableSQLMetric("sum") + dvFiles.set(taskList.getTasksList.asScala.count(!_.getDeletedRowIndexesList.isEmpty).toLong) + sparkContext.register(dvFiles, "files with deletion vectors") + + // `numFiles` alias mirrors Spark's `FileSourceScanExec` metric name so + // tests like DeltaSuite.scala "query with predicates should skip + // partitions" -- which read `metrics.get("numFiles")` to verify + // partition skipping -- find the same value on Comet's scan exec. + Map("total_files" -> totalFiles, "numFiles" -> totalFiles, "dv_files" -> dvFiles) + } else { + Map.empty[String, SQLMetric] + } + + baseMetrics ++ planningMetrics + } + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val nativeMetrics = CometMetricNode.fromCometPlan(this) + val serializedPlan = CometExec.serializeNativePlan(nativeOp) + // Recompute DPP pruning at execution time so we pick up broadcast results AQE has now + // materialised (the lazy `planningPerPartitionBytes` was computed before AQE ran). When DPP + // is absent or was already resolved at planning time, the two arrays are identical. + val execPerPartitionBytes = buildPerPartitionBytes() + // Mirror `CometNativeScanExec`'s encryption wiring: when parquet encryption is + // enabled on the table's hadoop conf, broadcast the conf to executors and + // gather every input file path (so the parquet reader can decrypt per file). + val sparkSession = originalPlan.relation.sparkSession + val hadoopConf = sparkSession.sessionState + .newHadoopConfWithOptions(originalPlan.relation.options) + val (broadcastedHadoopConfForEncryption, encryptedFilePaths) = + if (org.apache.comet.parquet.CometParquetUtils.encryptionEnabled(hadoopConf)) { + val broadcastedConf = sparkSession.sparkContext + .broadcast(new org.apache.spark.util.SerializableConfiguration(hadoopConf)) + val paths = execPerPartitionBytes.flatMap { bytes => + OperatorOuterClass.DeltaScan.parseFrom(bytes).getTasksList.asScala.map(_.getFilePath) + }.toSeq + (Some(broadcastedConf), paths) + } else { + (None, Seq.empty[String]) + } + val baseRDD = CometExecRDD( + sparkContext, + inputRDDs = Seq.empty, + commonByKey = Map(sourceKey -> commonData), + perPartitionByKey = Map(sourceKey -> execPerPartitionBytes), + serializedPlan = serializedPlan, + numPartitions = execPerPartitionBytes.length, + numOutputCols = output.length, + nativeMetrics = nativeMetrics, + subqueries = Seq.empty, + broadcastedHadoopConfForEncryption = broadcastedHadoopConfForEncryption, + encryptedFilePaths = encryptedFilePaths) + + // InputFileBlockHolder for downstream `input_file_name()` is populated in + // `CometExecRDD.compute` (via `InputFileBlockHolder.set`) so it also fires when this scan + // is embedded inside a larger Comet native tree (where this exec's own + // `doExecuteColumnar` is bypassed in favour of the parent's). + baseRDD + } + + override def convertBlock(): CometDeltaNativeScanExec = { + val newSerializedPlan = if (serializedPlanOpt.isEmpty) { + val bytes = CometExec.serializeNativePlan(nativeOp) + SerializedPlan(Some(bytes)) + } else { + serializedPlanOpt + } + CometDeltaNativeScanExec( + nativeOp, + output, + newSerializedPlan, + originalPlan, + tableRoot, + taskListBytes, + dppFilters, + partitionSchema) + } + + override protected def doCanonicalize(): CometDeltaNativeScanExec = { + copy( + output = output.map(QueryPlan.normalizeExpressions(_, output)), + serializedPlanOpt = SerializedPlan(None), + originalPlan = null, + taskListBytes = null, + dppFilters = Seq.empty) + } + + override def stringArgs: Iterator[Any] = { + val taskCount = + if (taskListBytes != null) { + OperatorOuterClass.DeltaScanTaskList.parseFrom(taskListBytes).getTasksCount + } else { + 0 + } + val dppStr = if (dppFilters.nonEmpty) { + s", dpp=${dppFilters.mkString("[", ", ", "]")}" + } else { + "" + } + Iterator(output, s"$tableRoot ($taskCount files$dppStr)") + } + + override def equals(obj: Any): Boolean = obj match { + case other: CometDeltaNativeScanExec => + // Include `sourceKey` so two scans of the same table at different snapshot versions + // are NOT considered equal. Without this, Spark's ReuseExchangeAndSubquery rule + // collapses a self-join across versions into a single exchange and reuses v0's + // shuffle output for both sides of the join. + tableRoot == other.tableRoot && + output == other.output && + serializedPlanOpt == other.serializedPlanOpt && + sourceKey == other.sourceKey + case _ => false + } + + override def hashCode(): Int = + Objects.hashCode(tableRoot, output.asJava, serializedPlanOpt, sourceKey) +} + +object CometDeltaNativeScanExec { + + /** + * Compute a stable, per-scan unique key from a `DeltaScan` operator proto. Must be + * deterministic and identical between the driver side (`CometDeltaNativeScanExec.sourceKey`) + * and the injector side (`DeltaPlanDataInjector.getKey`). + * + * Includes `snapshot_version` so that two scans of the same table at different time-travel + * versions produce distinct keys -- otherwise `findAllPlanData` collapses their per-partition + * data into a single map entry and one scan inherits the other's file list. + */ + def computeSourceKey(nativeOp: Operator): String = { + val common = nativeOp.getDeltaScan.getCommon + val components = Seq( + common.getTableRoot, + common.getSnapshotVersion.toString, + common.getRequiredSchemaList.toString, + common.getDataFiltersList.toString, + common.getProjectionVectorList.toString, + common.getColumnMappingsList.toString) + s"${common.getSource}_${components.mkString("|").hashCode}" + } +} diff --git a/contrib/delta/src/main/scala/org/apache/spark/sql/comet/DeltaInputFileBlockHolder.scala b/contrib/delta/src/main/scala/org/apache/spark/sql/comet/DeltaInputFileBlockHolder.scala new file mode 100644 index 0000000000..4a7c890259 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/spark/sql/comet/DeltaInputFileBlockHolder.scala @@ -0,0 +1,53 @@ +/* + * 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. + */ + +package org.apache.spark.sql.comet + +import org.apache.spark.TaskContext +import org.apache.spark.rdd.InputFileBlockHolder + +/** + * Thin wrapper around Spark's `InputFileBlockHolder` so the Delta contrib can populate the + * executor's input-file thread-local without trying to import a `private[spark]` symbol from + * `org.apache.comet.contrib.delta` (which would fail at scalac access-check time even though + * the underlying JVM class is public). Lives under `org.apache.spark.sql.comet` for the same + * reason `CometDeltaNativeScanExec` does -- the contrib's source-injection model lets us put + * helper classes anywhere on the classpath at build time. + * + * Public-API surface is intentionally minimal: set the file, register an unset on task + * completion, no holding of state across tasks. + */ +object DeltaInputFileBlockHolder { + + /** + * Set Spark's `InputFileBlockHolder` to the given file path and size for the duration of the + * current task. Registers a `TaskCompletionListener` (when `context` is non-null) to clear + * the thread-local on task end so the value doesn't leak into subsequent tasks on the same + * executor thread. + * + * `startOffset` is fixed at 0 — Delta partitions reference whole files; range-splitting that + * surfaces a non-zero offset would invalidate `_metadata.file_path` anyway. + */ + def set(filePath: String, fileSize: Long, context: TaskContext): Unit = { + InputFileBlockHolder.set(filePath, 0L, fileSize) + Option(context).foreach { ctx => + ctx.addTaskCompletionListener[Unit](_ => InputFileBlockHolder.unset()) + } + } +} diff --git a/contrib/delta/src/main/scala/org/apache/spark/sql/comet/DeltaPlanDataInjector.scala b/contrib/delta/src/main/scala/org/apache/spark/sql/comet/DeltaPlanDataInjector.scala new file mode 100644 index 0000000000..0c11afd121 --- /dev/null +++ b/contrib/delta/src/main/scala/org/apache/spark/sql/comet/DeltaPlanDataInjector.scala @@ -0,0 +1,74 @@ +/* + * 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. + */ + +package org.apache.spark.sql.comet + +import org.apache.comet.serde.OperatorOuterClass +import org.apache.comet.serde.OperatorOuterClass.Operator + +/** + * `PlanDataInjector` for the typed `OpStruct::DeltaScan` operator. + * + * The contrib serialises the Delta scan in two parts to keep the closure sent to every + * task small: + * - At planning time `CometDeltaNativeScan.convert` emits a `DeltaScan` proto with + * the `common` block (schemas, table root, filters, ...) and NO tasks; this lands + * in the `Operator` tree as the typed variant `OpStruct.delta_scan`. + * - Per partition, `CometDeltaNativeScanExec` puts the partition's `DeltaScan` + * (tasks-only) bytes into `perPartitionByKey` under a `sourceKey` derived from + * the common block. + * + * Core's `PlanDataInjector.injectPlanData` discovers this object via the reflective + * `Class.forName("org.apache.spark.sql.comet.DeltaPlanDataInjector")` lookup added to + * `PlanDataInjector.injectors`; default builds get no DeltaPlanDataInjector class on + * the classpath and the injector list is unchanged. + * + * Without this injection the native side decodes a tasks-empty `DeltaScan` -> `EmptyExec` + * (0 rows) for every Delta scan. + */ +object DeltaPlanDataInjector extends PlanDataInjector { + + override val opStructCase: Operator.OpStructCase = Operator.OpStructCase.DELTA_SCAN + + override def canInject(op: Operator): Boolean = { + if (!op.hasDeltaScan) return false + // The common-only proto produced at planning time has zero tasks. After injection + // the operator carries the partition's tasks -- skip those (idempotent canInject). + op.getDeltaScan.getTasksCount == 0 + } + + override def getKey(op: Operator): Option[String] = + Some(CometDeltaNativeScanExec.computeSourceKey(op)) + + override def inject( + op: Operator, + commonBytes: Array[Byte], + partitionBytes: Array[Byte]): Operator = { + // `partitionBytes` is the serialised `DeltaScan` that packs only this partition's + // tasks (no common block) to avoid duplicating schemas across partitions. Splice + // the partition's tasks into the original common-only envelope. + val tasksOnlyScan = OperatorOuterClass.DeltaScan.parseFrom(partitionBytes) + val originalScan = op.getDeltaScan + val mergedScan = OperatorOuterClass.DeltaScan + .newBuilder(originalScan) + .addAllTasks(tasksOnlyScan.getTasksList) + .build() + op.toBuilder.setDeltaScan(mergedScan).build() + } +} diff --git a/native/Cargo.lock b/native/Cargo.lock index df3c3b03c0..3e2a8fdb7f 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -228,25 +228,60 @@ version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" +[[package]] +name = "arrow" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3bd47f2a6ddc39244bd722a27ee5da66c03369d087b9e024eafdb03e98b98ea7" +dependencies = [ + "arrow-arith 57.3.1", + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-cast 57.3.1", + "arrow-csv 57.3.1", + "arrow-data 57.3.1", + "arrow-ipc 57.3.1", + "arrow-json 57.3.1", + "arrow-ord 57.3.1", + "arrow-row 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "arrow-string 57.3.1", +] + [[package]] name = "arrow" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "607e64bb911ee4f90483e044fe78f175989148c2892e659a2cd25429e782ec54" dependencies = [ - "arrow-arith", - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-csv", - "arrow-data", - "arrow-ipc", - "arrow-json", - "arrow-ord", - "arrow-row", - "arrow-schema", - "arrow-select", - "arrow-string", + "arrow-arith 58.2.0", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-cast 58.2.0", + "arrow-csv 58.2.0", + "arrow-data 58.2.0", + "arrow-ipc 58.2.0", + "arrow-json 58.2.0", + "arrow-ord 58.2.0", + "arrow-row 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", + "arrow-string 58.2.0", +] + +[[package]] +name = "arrow-arith" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c7bbd679c5418b8639b92be01f361d60013c4906574b578b77b63c78356594c" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "num-traits", ] [[package]] @@ -255,11 +290,30 @@ version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e754319ed8a85d817fe7adf183227e0b5308b82790a737b426c1124626b48118" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", + "chrono", + "num-traits", +] + +[[package]] +name = "arrow-array" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8a4ab47b3f3eac60f7fd31b81e9028fda018607bcc63451aca4f2b755269862" +dependencies = [ + "ahash", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", "chrono", + "chrono-tz", + "half", + "hashbrown 0.16.1", + "num-complex", + "num-integer", "num-traits", ] @@ -270,9 +324,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "841321891f247aa86c6112c80d83d89cb36e0addd020fa2425085b8eb6c3f579" dependencies = [ "ahash", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", "chrono", "chrono-tz", "half", @@ -282,6 +336,18 @@ dependencies = [ "num-traits", ] +[[package]] +name = "arrow-buffer" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d18b89b4c4f4811d0858175e79541fe98e33e18db3b011708bc287b1240593f" +dependencies = [ + "bytes", + "half", + "num-bigint", + "num-traits", +] + [[package]] name = "arrow-buffer" version = "58.2.0" @@ -294,18 +360,40 @@ dependencies = [ "num-traits", ] +[[package]] +name = "arrow-cast" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "722b5c41dd1d14d0a879a1bce92c6fe33f546101bb2acce57a209825edd075b3" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-ord 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "atoi", + "base64", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num-traits", + "ryu", +] + [[package]] name = "arrow-cast" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ca5e686972523798f76bef355145bc1ae25a84c731e650268d31ab763c701663" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-ord", - "arrow-schema", - "arrow-select", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-ord 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", "atoi", "base64", "chrono", @@ -316,47 +404,113 @@ dependencies = [ "ryu", ] +[[package]] +name = "arrow-csv" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "27ddb80a4848e03b1655af496d5ac2563a779e5742fcb48f2ca2e089c9cd2197" +dependencies = [ + "arrow-array 57.3.1", + "arrow-cast 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "csv", + "csv-core", + "regex", +] + [[package]] name = "arrow-csv" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "86c276756867fc8186ec380c72c290e6e3b23a1d4fb05df6b1d62d2e62666d48" dependencies = [ - "arrow-array", - "arrow-cast", - "arrow-schema", + "arrow-array 58.2.0", + "arrow-cast 58.2.0", + "arrow-schema 58.2.0", "chrono", "csv", "csv-core", "regex", ] +[[package]] +name = "arrow-data" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1683705c63dcf0d18972759eda48489028cbbff67af7d6bef2c6b7b74ab778a" +dependencies = [ + "arrow-buffer 57.3.1", + "arrow-schema 57.3.1", + "half", + "num-integer", + "num-traits", +] + [[package]] name = "arrow-data" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "db3b5846209775b6dc8056d77ff9a032b27043383dd5488abd0b663e265b9373" dependencies = [ - "arrow-buffer", - "arrow-schema", + "arrow-buffer 58.2.0", + "arrow-schema 58.2.0", "half", "num-integer", "num-traits", ] +[[package]] +name = "arrow-ipc" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8cf72d04c07229fbf4dbebe7145cac37d7cf7ec582fe705c6b92cb314af096ab" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "flatbuffers", +] + [[package]] name = "arrow-ipc" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fd8907ddd8f9fbabf91ec2c85c1d81fe2874e336d2443eb36373595e28b98dd5" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", "flatbuffers", - "lz4_flex", + "lz4_flex 0.13.0", +] + +[[package]] +name = "arrow-json" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a84a905f41fedfcd7679813c89a61dc369c0f932b27aa8dcc6aa051cc781a97d" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-cast 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "chrono", + "half", + "indexmap 2.14.0", + "itoa", + "lexical-core", + "memchr", + "num-traits", + "ryu", + "serde_core", + "serde_json", + "simdutf8", ] [[package]] @@ -365,12 +519,12 @@ version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f4518c59acc501f10d7dcae397fe12b8db3d81bc7de94456f8a58f9165d6f502" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-ord", - "arrow-schema", - "arrow-select", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-cast 58.2.0", + "arrow-ord 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", "chrono", "half", "indexmap 2.14.0", @@ -384,17 +538,43 @@ dependencies = [ "simdutf8", ] +[[package]] +name = "arrow-ord" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "082342947d4e5a2bcccf029a0a0397e21cb3bb8421edd9571d34fb5dd2670256" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", +] + [[package]] name = "arrow-ord" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "efa70d9d6b1356f1fb9f1f651b84a725b7e0abb93f188cf7d31f14abfa2f2e6f" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", +] + +[[package]] +name = "arrow-row" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3a931b520a2a5e22033e01a6f2486b4cdc26f9106b759abeebc320f125e94d7" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "half", ] [[package]] @@ -403,13 +583,22 @@ version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "faec88a945338192beffbbd4be0def70135422930caa244ac3cec0cd213b26b4" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", "half", ] +[[package]] +name = "arrow-schema" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4cf0d4a6609679e03002167a61074a21d7b1ad9ea65e462b2c0a97f8a3b2bc6" +dependencies = [ + "bitflags 2.11.1", +] + [[package]] name = "arrow-schema" version = "58.2.0" @@ -421,6 +610,20 @@ dependencies = [ "serde_json", ] +[[package]] +name = "arrow-select" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b320d86a9806923663bb0fd9baa65ecaba81cb0cd77ff8c1768b9716b4ef891" +dependencies = [ + "ahash", + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "num-traits", +] + [[package]] name = "arrow-select" version = "58.2.0" @@ -428,24 +631,41 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a657ab5132e9c8ca3b24eb15a823d0ced38017fe3930ff50167466b02e2d592c" dependencies = [ "ahash", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", "num-traits", ] +[[package]] +name = "arrow-string" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b493e99162e5764077e7823e50ba284858d365922631c7aaefe9487b1abd02c2" +dependencies = [ + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-data 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "memchr", + "num-traits", + "regex", + "regex-syntax", +] + [[package]] name = "arrow-string" version = "58.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f6de2efbbd1a9f9780ceb8d1ff5d20421b35863b361e3386b4f571f1fc69fcb8" dependencies = [ - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-schema", - "arrow-select", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", "memchr", "num-traits", "regex", @@ -1485,12 +1705,36 @@ dependencies = [ "memchr", ] +[[package]] +name = "comet-contrib-delta" +version = "0.17.0" +dependencies = [ + "arrow 58.2.0", + "chrono", + "chrono-tz", + "datafusion", + "datafusion-comet-jni-bridge", + "datafusion-comet-proto", + "delta_kernel", + "futures", + "jni 0.22.4", + "log", + "object_store 0.12.5", + "object_store 0.13.2", + "parquet 58.1.0", + "prost", + "roaring 0.10.12", + "thiserror 2.0.18", + "url", +] + [[package]] name = "comfy-table" version = "7.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "958c5d6ecf1f214b4c2bbbbf6ab9523a864bd136dcf71a7e8904799acfe1ad47" dependencies = [ + "crossterm", "unicode-segmentation", "unicode-width", ] @@ -1606,6 +1850,21 @@ dependencies = [ "libc", ] +[[package]] +name = "crc" +version = "3.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5eb8a2a1cd12ab0d987a5d5e825195d372001a4094a0376319d5a0ad71c1ba0d" +dependencies = [ + "crc-catalog", +] + +[[package]] +name = "crc-catalog" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "217698eaf96b4a3f0bc4f3662aaa55bdf913cd54d7204591faa790070c6d0853" + [[package]] name = "crc32c" version = "0.6.8" @@ -1693,6 +1952,29 @@ version = "0.8.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" +[[package]] +name = "crossterm" +version = "0.29.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d8b9f2e4c67f833b660cdb0a3523065869fb35570177239812ed4c905aeff87b" +dependencies = [ + "bitflags 2.11.1", + "crossterm_winapi", + "document-features", + "parking_lot", + "rustix 1.1.4", + "winapi", +] + +[[package]] +name = "crossterm_winapi" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acdd7c62a3665c7f6830a51635d9ac9b23ed385797f70a83bb8bafe9c572ab2b" +dependencies = [ + "winapi", +] + [[package]] name = "crunchy" version = "0.2.4" @@ -1863,8 +2145,8 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "93db0e623840612f7f2cd757f7e8a8922064192363732c88692e0870016e141b" dependencies = [ - "arrow", - "arrow-schema", + "arrow 58.2.0", + "arrow-schema 58.2.0", "async-trait", "bytes", "chrono", @@ -1896,9 +2178,9 @@ dependencies = [ "futures", "itertools 0.14.0", "log", - "object_store", + "object_store 0.13.2", "parking_lot", - "parquet", + "parquet 58.1.0", "rand 0.9.4", "regex", "sqlparser", @@ -1914,7 +2196,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "37cefde60b26a7f4ff61e9d2ff2833322f91df2b568d7238afe67bde5bdffb66" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "dashmap", "datafusion-common", @@ -1928,7 +2210,7 @@ dependencies = [ "futures", "itertools 0.14.0", "log", - "object_store", + "object_store 0.13.2", "parking_lot", "tokio", ] @@ -1939,7 +2221,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "17e112307715d6a7a331111a4c2330ff54bc237183511c319e3708a4cff431fb" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "datafusion-catalog", "datafusion-common", @@ -1953,19 +2235,20 @@ dependencies = [ "futures", "itertools 0.14.0", "log", - "object_store", + "object_store 0.13.2", ] [[package]] name = "datafusion-comet" version = "0.17.0" dependencies = [ - "arrow", + "arrow 58.2.0", "assertables", "async-trait", "aws-config", "aws-credential-types", "bytes", + "comet-contrib-delta", "criterion", "datafusion", "datafusion-comet-common", @@ -1991,12 +2274,12 @@ dependencies = [ "log4rs", "mimalloc", "num", - "object_store", + "object_store 0.13.2", "object_store_opendal", "once_cell", "opendal 0.56.0", "parking_lot", - "parquet", + "parquet 58.1.0", "paste", "pprof", "procfs", @@ -2016,7 +2299,7 @@ dependencies = [ name = "datafusion-comet-common" version = "0.17.0" dependencies = [ - "arrow", + "arrow 58.2.0", "datafusion", "serde", "serde_json", @@ -2042,14 +2325,14 @@ dependencies = [ name = "datafusion-comet-jni-bridge" version = "0.17.0" dependencies = [ - "arrow", + "arrow 58.2.0", "assertables", "datafusion", "datafusion-comet-common", "jni 0.22.4", "lazy_static", "once_cell", - "parquet", + "parquet 58.1.0", "paste", "prost", "regex", @@ -2066,7 +2349,7 @@ dependencies = [ "datafusion-comet-fs-hdfs3", "fs-hdfs3", "futures", - "object_store", + "object_store 0.13.2", "tokio", ] @@ -2082,7 +2365,7 @@ dependencies = [ name = "datafusion-comet-shuffle" version = "0.17.0" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "bytes", "clap", @@ -2097,8 +2380,8 @@ dependencies = [ "itertools 0.14.0", "jni 0.21.1", "log", - "lz4_flex", - "parquet", + "lz4_flex 0.13.0", + "parquet 58.1.0", "simd-adler32", "snap", "tempfile", @@ -2110,7 +2393,7 @@ dependencies = [ name = "datafusion-comet-spark-expr" version = "0.17.0" dependencies = [ - "arrow", + "arrow 58.2.0", "base64", "chrono", "chrono-tz", @@ -2136,8 +2419,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d72a11ca44a95e1081870d3abb80c717496e8a7acb467a1d3e932bb636af5cc2" dependencies = [ "ahash", - "arrow", - "arrow-ipc", + "arrow 58.2.0", + "arrow-ipc 58.2.0", "chrono", "half", "hashbrown 0.16.1", @@ -2146,8 +2429,8 @@ dependencies = [ "itertools 0.14.0", "libc", "log", - "object_store", - "parquet", + "object_store 0.13.2", + "parquet 58.1.0", "paste", "sqlparser", "tokio", @@ -2171,7 +2454,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e9fb386e1691355355a96419978a0022b7947b44d4a24a6ea99f00b6b485cbb6" dependencies = [ - "arrow", + "arrow 58.2.0", "async-compression", "async-trait", "bytes", @@ -2192,7 +2475,7 @@ dependencies = [ "itertools 0.14.0", "liblzma", "log", - "object_store", + "object_store 0.13.2", "rand 0.9.4", "tokio", "tokio-util", @@ -2206,8 +2489,8 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ffa6c52cfed0734c5f93754d1c0175f558175248bf686c944fb05c373e5fc096" dependencies = [ - "arrow", - "arrow-ipc", + "arrow 58.2.0", + "arrow-ipc 58.2.0", "async-trait", "bytes", "datafusion-common", @@ -2220,7 +2503,7 @@ dependencies = [ "datafusion-session", "futures", "itertools 0.14.0", - "object_store", + "object_store 0.13.2", "tokio", ] @@ -2230,7 +2513,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "503f29e0582c1fc189578d665ff57d9300da1f80c282777d7eb67bb79fb8cdca" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "bytes", "datafusion-common", @@ -2242,7 +2525,7 @@ dependencies = [ "datafusion-physical-plan", "datafusion-session", "futures", - "object_store", + "object_store 0.13.2", "regex", "tokio", ] @@ -2253,7 +2536,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e33804749abc8d0c8cb7473228483cb8070e524c6f6086ee1b85a64debe2b3d2" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "bytes", "datafusion-common", @@ -2265,7 +2548,7 @@ dependencies = [ "datafusion-physical-plan", "datafusion-session", "futures", - "object_store", + "object_store 0.13.2", "serde_json", "tokio", "tokio-stream", @@ -2277,7 +2560,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a8e0365e0e08e8ff94d912f0ababcf9065a1a304018ba90b1fc83c855b4997" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "bytes", "datafusion-common", @@ -2295,9 +2578,9 @@ dependencies = [ "futures", "itertools 0.14.0", "log", - "object_store", + "object_store 0.13.2", "parking_lot", - "parquet", + "parquet 58.1.0", "tokio", ] @@ -2313,8 +2596,8 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c03c7fbdaefcca4ef6ffe425a5fc2325763bfb426599bb0bf4536466efabe709" dependencies = [ - "arrow", - "arrow-buffer", + "arrow 58.2.0", + "arrow-buffer 58.2.0", "async-trait", "chrono", "dashmap", @@ -2323,9 +2606,9 @@ dependencies = [ "datafusion-physical-expr-common", "futures", "log", - "object_store", + "object_store 0.13.2", "parking_lot", - "parquet", + "parquet 58.1.0", "rand 0.9.4", "tempfile", "url", @@ -2337,7 +2620,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "574b9b6977fedbd2a611cbff12e5caf90f31640ad9dc5870f152836d94bad0dd" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "chrono", "datafusion-common", @@ -2359,7 +2642,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7d7c3adf3db8bf61e92eb90cb659c8e8b734593a8f7c8e12a843c7ddba24b87e" dependencies = [ - "arrow", + "arrow 58.2.0", "datafusion-common", "indexmap 2.14.0", "itertools 0.14.0", @@ -2372,8 +2655,8 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f28aa4e10384e782774b10e72aca4d93ef7b31aa653095d9d4536b0a3dbc51b6" dependencies = [ - "arrow", - "arrow-buffer", + "arrow 58.2.0", + "arrow-buffer 58.2.0", "base64", "blake2", "blake3", @@ -2405,7 +2688,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "00aa6217e56098ba84e0a338176fe52f0a84cca398021512c6c8c5eff806d0ad" dependencies = [ "ahash", - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -2427,7 +2710,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b511250349407db7c43832ab2de63f5557b19a20dfd236b39ca2c04468b50d47" dependencies = [ "ahash", - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-expr-common", "datafusion-physical-expr-common", @@ -2439,8 +2722,8 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ef13a858e20d50f0a9bb5e96e7ac82b4e7597f247515bccca4fdd2992df0212a" dependencies = [ - "arrow", - "arrow-ord", + "arrow 58.2.0", + "arrow-ord 58.2.0", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -2464,7 +2747,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "72b40d3f5bbb3905f9ccb1ce9485a9595c77b69758a7c24d3ba79e334ff51e7e" dependencies = [ - "arrow", + "arrow 58.2.0", "async-trait", "datafusion-catalog", "datafusion-common", @@ -2480,7 +2763,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d4e88ec9d57c9b685d02f58bfee7be62d72610430ddcedb82a08e5d9925dbfb6" dependencies = [ - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-doc", "datafusion-expr", @@ -2519,7 +2802,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e929015451a67f77d9d8b727b2bf3a40c4445fdef6cdc53281d7d97c76888ace" dependencies = [ - "arrow", + "arrow 58.2.0", "chrono", "datafusion-common", "datafusion-expr", @@ -2539,7 +2822,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b1e68aba7a4b350401cfdf25a3d6f989ad898a7410164afe9ca52080244cb59" dependencies = [ "ahash", - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-expr", "datafusion-expr-common", @@ -2561,7 +2844,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ea22315f33cf2e0adc104e8ec42e285f6ed93998d565c65e82fec6a9ee9f9db4" dependencies = [ - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-expr", "datafusion-functions", @@ -2577,7 +2860,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b04b45ea8ad3ac2d78f2ea2a76053e06591c9629c7a603eda16c10649ecf4362" dependencies = [ "ahash", - "arrow", + "arrow 58.2.0", "chrono", "datafusion-common", "datafusion-expr-common", @@ -2593,7 +2876,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7cb13397809a425918f608dfe8653f332015a3e330004ab191b4404187238b95" dependencies = [ - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-execution", "datafusion-expr", @@ -2612,9 +2895,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5edc023675791af9d5fb4cc4c24abf5f7bd3bd4dcf9e5bd90ea1eff6976dcc79" dependencies = [ "ahash", - "arrow", - "arrow-ord", - "arrow-schema", + "arrow 58.2.0", + "arrow-ord 58.2.0", + "arrow-schema 58.2.0", "async-trait", "datafusion-common", "datafusion-common-runtime", @@ -2643,7 +2926,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ac8c76860e355616555081cab5968cec1af7a80701ff374510860bcd567e365a" dependencies = [ - "arrow", + "arrow 58.2.0", "datafusion-common", "datafusion-datasource", "datafusion-expr-common", @@ -2674,7 +2957,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e059dcf8544da0d6598d0235be3cc29c209094a5976b2e4822e4a2cf91c2b5c5" dependencies = [ - "arrow", + "arrow 58.2.0", "bigdecimal", "chrono", "crc32fast", @@ -2701,7 +2984,7 @@ version = "53.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fa0d133ddf8b9b3b872acac900157f783e7b879fe9a6bccf389abebbfac45ec1" dependencies = [ - "arrow", + "arrow 58.2.0", "bigdecimal", "chrono", "datafusion-common", @@ -2722,6 +3005,48 @@ dependencies = [ "uuid", ] +[[package]] +name = "delta_kernel" +version = "0.19.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06f7fc164b1557731fcc68a198e813811a000efade0f112d4f0a002e65042b83" +dependencies = [ + "arrow 57.3.1", + "bytes", + "chrono", + "comfy-table", + "crc", + "delta_kernel_derive", + "futures", + "indexmap 2.14.0", + "itertools 0.14.0", + "object_store 0.12.5", + "parquet 57.3.1", + "reqwest 0.12.28", + "roaring 0.11.3", + "rustc_version", + "serde", + "serde_json", + "strum", + "thiserror 2.0.18", + "tokio", + "tracing", + "url", + "uuid", + "z85", +] + +[[package]] +name = "delta_kernel_derive" +version = "0.19.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86815a2c475835751ffa9b8d9ac8ed86cf86294304c42bedd1103d54f25ecbfe" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.117", +] + [[package]] name = "der" version = "0.7.10" @@ -2852,6 +3177,15 @@ dependencies = [ "const-random", ] +[[package]] +name = "document-features" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4b8a88685455ed29a21542a33abd9cb6510b6b129abadabdcef0f4c55bc8f61" +dependencies = [ + "litrs", +] + [[package]] name = "dtor" version = "0.1.1" @@ -3573,14 +3907,14 @@ dependencies = [ "anyhow", "apache-avro", "array-init", - "arrow-arith", - "arrow-array", - "arrow-buffer", - "arrow-cast", - "arrow-ord", - "arrow-schema", - "arrow-select", - "arrow-string", + "arrow-arith 58.2.0", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-cast 58.2.0", + "arrow-ord 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", + "arrow-string 58.2.0", "as-any", "async-trait", "backon", @@ -3599,10 +3933,10 @@ dependencies = [ "murmur3", "once_cell", "ordered-float 4.6.0", - "parquet", + "parquet 58.1.0", "rand 0.9.4", "reqwest 0.12.28", - "roaring", + "roaring 0.11.3", "serde", "serde_bytes", "serde_derive", @@ -4203,6 +4537,12 @@ version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "92daf443525c4cce67b150400bc2316076100ce0b3686209eb8cf3c31612e6f0" +[[package]] +name = "litrs" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "11d3d7f243d5c5a8b9bb5d6dd2b1602c0cb0b9db1621bafc7ed66e35ff9fe092" + [[package]] name = "lock_api" version = "0.4.14" @@ -4263,6 +4603,15 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" +[[package]] +name = "lz4_flex" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90071f8077f8e40adfc4b7fe9cd495ce316263f19e75c2211eeff3fdf475a3d9" +dependencies = [ + "twox-hash", +] + [[package]] name = "lz4_flex" version = "0.13.0" @@ -4517,6 +4866,44 @@ dependencies = [ "memchr", ] +[[package]] +name = "object_store" +version = "0.12.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbfbfff40aeccab00ec8a910b57ca8ecf4319b335c542f2edcd19dd25a1e2a00" +dependencies = [ + "async-trait", + "base64", + "bytes", + "chrono", + "form_urlencoded", + "futures", + "http 1.4.0", + "http-body-util", + "httparse", + "humantime", + "hyper", + "itertools 0.14.0", + "md-5", + "parking_lot", + "percent-encoding", + "quick-xml 0.38.4", + "rand 0.9.4", + "reqwest 0.12.28", + "ring", + "rustls-pemfile", + "serde", + "serde_json", + "serde_urlencoded", + "thiserror 2.0.18", + "tokio", + "tracing", + "url", + "walkdir", + "wasm-bindgen-futures", + "web-time", +] + [[package]] name = "object_store" version = "0.13.2" @@ -4568,7 +4955,7 @@ dependencies = [ "chrono", "futures", "mea", - "object_store", + "object_store 0.13.2", "opendal 0.56.0", "pin-project", "tokio", @@ -4797,6 +5184,43 @@ dependencies = [ "windows-link", ] +[[package]] +name = "parquet" +version = "57.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e832c6aa20310fc6de7ea5a3f4e20d34fd83e3b43229d32b81ffe5c14d74692" +dependencies = [ + "ahash", + "arrow-array 57.3.1", + "arrow-buffer 57.3.1", + "arrow-cast 57.3.1", + "arrow-data 57.3.1", + "arrow-ipc 57.3.1", + "arrow-schema 57.3.1", + "arrow-select 57.3.1", + "base64", + "brotli", + "bytes", + "chrono", + "flate2", + "futures", + "half", + "hashbrown 0.16.1", + "lz4_flex 0.12.2", + "num-bigint", + "num-integer", + "num-traits", + "object_store 0.12.5", + "paste", + "seq-macro", + "simdutf8", + "snap", + "thrift", + "tokio", + "twox-hash", + "zstd", +] + [[package]] name = "parquet" version = "58.1.0" @@ -4804,12 +5228,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7d3f9f2205199603564127932b89695f52b62322f541d0fc7179d57c2e1c9877" dependencies = [ "ahash", - "arrow-array", - "arrow-buffer", - "arrow-data", - "arrow-ipc", - "arrow-schema", - "arrow-select", + "arrow-array 58.2.0", + "arrow-buffer 58.2.0", + "arrow-data 58.2.0", + "arrow-ipc 58.2.0", + "arrow-schema 58.2.0", + "arrow-select 58.2.0", "base64", "brotli", "bytes", @@ -4818,11 +5242,11 @@ dependencies = [ "futures", "half", "hashbrown 0.16.1", - "lz4_flex", + "lz4_flex 0.13.0", "num-bigint", "num-integer", "num-traits", - "object_store", + "object_store 0.13.2", "parquet-variant", "parquet-variant-compute", "parquet-variant-json", @@ -4843,7 +5267,7 @@ version = "58.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2bf493f3c9ddd984d0efb019f67343e4aa4bab893931f6a14b82083065dc3d28" dependencies = [ - "arrow-schema", + "arrow-schema 58.2.0", "chrono", "half", "indexmap 2.14.0", @@ -4857,8 +5281,8 @@ version = "58.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6ac038d46a503a7d563b4f5df5802c4315d5343d009feab195d15ac512b4cb27" dependencies = [ - "arrow", - "arrow-schema", + "arrow 58.2.0", + "arrow-schema 58.2.0", "chrono", "half", "indexmap 2.14.0", @@ -4874,7 +5298,7 @@ version = "58.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "015a09c2ffe5108766c7c1235c307b8a3c2ea64eca38455ba1a7f3a7f32f16e2" dependencies = [ - "arrow-schema", + "arrow-schema 58.2.0", "base64", "chrono", "parquet-variant", @@ -5695,6 +6119,16 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "roaring" +version = "0.10.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19e8d2cfa184d94d0726d650a9f4a1be7f9b76ac9fdb954219878dc00c1c1e7b" +dependencies = [ + "bytemuck", + "byteorder", +] + [[package]] name = "roaring" version = "0.11.3" @@ -5816,6 +6250,15 @@ dependencies = [ "security-framework", ] +[[package]] +name = "rustls-pemfile" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dce314e5fee3f39953d46bb63bb8a46d40c2f8fb7cc5a3b6cab2bde9721d6e50" +dependencies = [ + "rustls-pki-types", +] + [[package]] name = "rustls-pki-types" version = "1.14.0" @@ -6703,6 +7146,7 @@ version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "63e71662fa4b2a2c3a26f570f037eb95bb1f85397f3cd8076caed2f026a6d100" dependencies = [ + "log", "pin-project-lite", "tracing-attributes", "tracing-core", @@ -6901,6 +7345,7 @@ checksum = "ddd74a9687298c6858e9b88ec8935ec45d22e8fd5e6394fa1bd4e99a87789c76" dependencies = [ "getrandom 0.4.2", "js-sys", + "rand 0.10.1", "serde_core", "wasm-bindgen", ] @@ -7575,6 +8020,12 @@ dependencies = [ "synstructure", ] +[[package]] +name = "z85" +version = "3.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c6e61e59a957b7ccee15d2049f86e8bfd6f66968fcd88f018950662d9b86e675" + [[package]] name = "zerocopy" version = "0.8.48" diff --git a/native/Cargo.toml b/native/Cargo.toml index d1b5c74af9..4923a75008 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -18,6 +18,10 @@ [workspace] default-members = ["core", "spark-expr", "common", "proto", "jni-bridge", "shuffle"] members = ["core", "spark-expr", "common", "proto", "jni-bridge", "shuffle", "hdfs", "fs-hdfs"] +# The contrib crate at ../contrib/delta/native is intentionally NOT a workspace member +# (workspace members must live hierarchically under the workspace root). It's pulled in +# as a path dep by `core/Cargo.toml` when the `contrib-delta` feature is enabled. +exclude = ["../contrib"] resolver = "2" [workspace.package] diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4fb3ed4c5d..e5cacd1b09 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -69,6 +69,9 @@ aws-config = { workspace = true } aws-credential-types = { workspace = true } parking_lot = "0.12.5" datafusion-comet-objectstore-hdfs = { path = "../hdfs", optional = true, default-features = false, features = ["hdfs"] } +# Optional Delta Lake contrib (enabled by the `contrib-delta` feature). Source lives +# under `contrib/delta/native/` so non-Delta committers can ignore it. +comet-contrib-delta = { path = "../../contrib/delta/native", optional = true } reqwest = { version = "0.12", default-features = false, features = ["rustls-tls-native-roots", "http2"] } object_store_opendal = { version = "0.56.0", optional = true } hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} @@ -99,6 +102,10 @@ default = ["hdfs-opendal"] hdfs = ["datafusion-comet-objectstore-hdfs"] hdfs-opendal = ["opendal", "object_store_opendal", "hdfs-sys"] jemalloc = ["tikv-jemallocator", "tikv-jemalloc-ctl"] +# Delta Lake integration. When enabled, links the `comet-contrib-delta` crate +# into `libcomet` and activates the `OpStruct::DeltaScan` dispatcher arm. +# Default builds carry zero Delta surface. +contrib-delta = ["dep:comet-contrib-delta"] # exclude optional packages from cargo machete verifications [package.metadata.cargo-machete] diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index f5b04cc51d..f3f538004f 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -232,6 +232,7 @@ fn op_name(op: &OpStruct) -> &'static str { OpStruct::Explode(_) => "Explode", OpStruct::CsvScan(_) => "CsvScan", OpStruct::ShuffleScan(_) => "ShuffleScan", + OpStruct::DeltaScan(_) => "DeltaScan", } } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index b00f140026..50651c5c6c 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -21,6 +21,9 @@ pub mod expression_registry; pub mod macros; pub mod operator_registry; +#[cfg(feature = "contrib-delta")] +mod contrib_delta_scan; + use crate::execution::operators::init_csv_datasource_exec; use crate::execution::operators::IcebergScanExec; use crate::execution::{ @@ -1376,6 +1379,10 @@ impl PhysicalPlanner { common.encryption_enabled, common.use_field_id, common.ignore_missing_field_id, + // ignore_missing_files: not exposed through the native_datafusion + // scan proto today. Contribs (e.g. Delta) wire this through directly + // when they call init_datasource_exec. + false, )?; Ok(( vec![], @@ -1492,6 +1499,26 @@ impl PhysicalPlanner { )), )) } + OpStruct::DeltaScan(scan) => { + // Delta Lake scan -- handled by the optional `contrib/delta/` integration. + // The dispatcher arm exists unconditionally so a default build that receives + // a Delta-shaped plan from a misconfigured driver gets a clear error instead + // of a "no match" decode failure. + #[cfg(not(feature = "contrib-delta"))] + { + let _ = scan; + Err(GeneralError( + "Received a DeltaScan operator but core was built without the \ + `contrib-delta` Cargo feature. Rebuild with \ + `cargo build --features contrib-delta` to enable Delta Lake support." + .into(), + )) + } + #[cfg(feature = "contrib-delta")] + { + self.plan_delta_scan(spark_plan, scan) + } + } OpStruct::ShuffleWriter(writer) => { assert_eq!(children.len(), 1); let (scans, shuffle_scans, child) = diff --git a/native/core/src/execution/planner/contrib_delta_scan.rs b/native/core/src/execution/planner/contrib_delta_scan.rs new file mode 100644 index 0000000000..42a2b528b8 --- /dev/null +++ b/native/core/src/execution/planner/contrib_delta_scan.rs @@ -0,0 +1,271 @@ +// 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. + +//! `OpStruct::DeltaScan` planner body, feature-gated behind `contrib-delta`. +//! +//! This module mirrors the size and shape of the `OpStruct::IcebergScan` arm in +//! `super::planner` -- the arm itself stays tiny (just dispatches here), and the +//! Delta-specific algorithmic pieces (DV filter exec wrapping, column-mapping +//! rename projection, partition value parsing) live in the +//! [`comet_contrib_delta`] crate. + +#![cfg(feature = "contrib-delta")] + +use std::collections::HashMap; +use std::sync::Arc; + +use comet_contrib_delta::planner::{ + build_delta_partitioned_files, ColumnMappingFilterRewriter, +}; +use comet_contrib_delta::DeltaDvFilterExec; +use datafusion::arrow::datatypes::{Field, Schema, SchemaRef}; +use datafusion::common::tree_node::{TransformedResult, TreeNode}; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::empty::EmptyExec; +use datafusion::physical_plan::expressions::Column; +use datafusion::physical_plan::projection::ProjectionExec; +use datafusion_comet_proto::spark_operator::{DeltaScan, Operator}; + +use crate::execution::operators::ExecutionError; +use crate::execution::operators::ExecutionError::GeneralError; +use crate::execution::planner::convert_spark_types_to_arrow_schema; +use crate::execution::planner::PhysicalPlanner; +use crate::execution::planner::PlanCreationResult; +use crate::execution::spark_plan::SparkPlan; +use crate::parquet::parquet_exec::init_datasource_exec; +use crate::parquet::parquet_support::prepare_object_store_with_configs; + +impl PhysicalPlanner { + pub(crate) fn plan_delta_scan( + &self, + spark_plan: &Operator, + scan: &DeltaScan, + ) -> PlanCreationResult { + let common = scan + .common + .as_ref() + .ok_or_else(|| GeneralError("DeltaScan missing common data".into()))?; + + let required_schema: SchemaRef = + convert_spark_types_to_arrow_schema(common.required_schema.as_slice()); + let mut data_schema: SchemaRef = + convert_spark_types_to_arrow_schema(common.data_schema.as_slice()); + let partition_schema: SchemaRef = + convert_spark_types_to_arrow_schema(common.partition_schema.as_slice()); + + // Column mapping: substitute physical names into data_schema so ParquetSource + // projects by the names actually in the file. A rename projection on top maps + // physical names back to the logical names upstream operators expect. + let logical_to_physical: HashMap = common + .column_mappings + .iter() + .map(|cm| (cm.logical_name.clone(), cm.physical_name.clone())) + .collect(); + let has_column_mapping = !logical_to_physical.is_empty(); + if has_column_mapping { + let new_fields: Vec<_> = data_schema + .fields() + .iter() + .map(|f| { + if let Some(physical) = logical_to_physical.get(f.name()) { + Arc::new(Field::new( + physical, + f.data_type().clone(), + f.is_nullable(), + )) + } else { + Arc::clone(f) + } + }) + .collect(); + data_schema = Arc::new(Schema::new(new_fields)); + } + let projection_vector: Vec = common + .projection_vector + .iter() + .map(|offset| *offset as usize) + .collect(); + + // Empty-partition fast path. + if scan.tasks.is_empty() { + return Ok(( + vec![], + vec![], + Arc::new(SparkPlan::new( + spark_plan.plan_id, + Arc::new(EmptyExec::new(required_schema)), + vec![], + )), + )); + } + + // Build pushed-down data filters, rewriting Column refs to physical names when + // column mapping is active. + let data_filters: Result>, ExecutionError> = common + .data_filters + .iter() + .map(|expr| { + let filter = self + .create_expr(expr, Arc::clone(&required_schema)) + .map_err(|e| GeneralError(format!("DeltaScan filter: {e}")))?; + if has_column_mapping { + let mut rewriter = ColumnMappingFilterRewriter { + logical_to_physical: &logical_to_physical, + data_schema: &data_schema, + }; + filter + .rewrite(&mut rewriter) + .data() + .map_err(|e| GeneralError(format!("ColumnMappingFilterRewriter: {e}"))) + } else { + Ok(filter) + } + }) + .collect(); + + let object_store_options: HashMap = common + .object_store_options + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + + // Build PartitionedFiles. Kernel has already resolved each file path to an + // absolute URL on the driver, so we thread them straight through. Delta stores + // TIMESTAMP partition values in the JVM default TZ; pass the session TZ so + // partition-value parsing produces the correct instant. + let files = build_delta_partitioned_files( + &scan.tasks, + partition_schema.as_ref(), + common.session_timezone.as_str(), + ) + .map_err(GeneralError)?; + + // Split files by DV presence -- each DV'd file becomes its own FileGroup so the + // DeltaDvFilterExec's per-partition mapping is 1:1 with one physical parquet + // file. All non-DV files go in a single combined group. + let mut file_groups: Vec> = Vec::new(); + let mut deleted_indexes_per_group: Vec> = Vec::new(); + let mut non_dv_files: Vec = Vec::new(); + for (file, task) in files.into_iter().zip(scan.tasks.iter()) { + if task.deleted_row_indexes.is_empty() { + non_dv_files.push(file); + } else { + file_groups.push(vec![file]); + deleted_indexes_per_group.push(task.deleted_row_indexes.clone()); + } + } + if !non_dv_files.is_empty() { + file_groups.push(non_dv_files); + deleted_indexes_per_group.push(Vec::new()); + } + + // Pick any one file to register the object store (they all share the same root). + let one_file = scan + .tasks + .first() + .map(|t| t.file_path.clone()) + .ok_or_else(|| { + GeneralError("DeltaScan has no tasks after split-mode injection".into()) + })?; + let url = url::Url::parse(&one_file) + .map_err(|e| GeneralError(format!("DeltaScan invalid file URL: {e}")))?; + let (object_store_url, _root_path) = prepare_object_store_with_configs( + self.session_ctx().runtime_env(), + url.to_string(), + &object_store_options, + ) + .map_err(|e| GeneralError(format!("prepare_object_store_with_configs: {e}")))?; + + let delta_exec = init_datasource_exec( + Arc::clone(&required_schema), + Some(data_schema), + Some(partition_schema), + object_store_url, + file_groups, + Some(projection_vector), + Some(data_filters?), + None, // default_values + common.session_timezone.as_str(), + common.case_sensitive, + false, // return_null_struct_if_all_fields_missing + self.session_ctx(), + false, // encryption_enabled (Delta tables we natively support are unencrypted) + false, // use_field_id + false, // ignore_missing_field_id + common.ignore_missing_files, + )?; + + // Wrap in a DV filter when any partition has a DV. Skip the wrapper otherwise + // to avoid the per-batch pass-through cost in the common "no DVs" case. + let final_exec: Arc = + if deleted_indexes_per_group.iter().any(|v| !v.is_empty()) { + Arc::new( + DeltaDvFilterExec::new(delta_exec, deleted_indexes_per_group) + .map_err(|e| GeneralError(format!("DeltaDvFilterExec: {e}")))?, + ) + } else { + delta_exec + }; + + // When column mapping is active, the scan's output schema carries PHYSICAL + // column names. Upstream operators reference columns by LOGICAL name, so add a + // ProjectionExec aliasing each physical column back to its logical name. + let scan_out = final_exec.schema(); + let needs_rename = has_column_mapping + && required_schema.fields().len() == scan_out.fields().len() + && required_schema + .fields() + .iter() + .zip(scan_out.fields().iter()) + .any(|(req, phys)| req.name() != phys.name()); + let with_rename: Arc = if needs_rename { + let phys_to_logical: HashMap<&str, &str> = scan_out + .fields() + .iter() + .zip(required_schema.fields().iter()) + .map(|(phys, req)| (phys.name().as_str(), req.name().as_str())) + .collect(); + let projections: Vec<(Arc, String)> = scan_out + .fields() + .iter() + .enumerate() + .map(|(idx, phys_field)| { + let col: Arc = + Arc::new(Column::new(phys_field.name(), idx)); + let alias = phys_to_logical + .get(phys_field.name().as_str()) + .map(|s| s.to_string()) + .unwrap_or_else(|| phys_field.name().clone()); + (col, alias) + }) + .collect(); + Arc::new( + ProjectionExec::try_new(projections, final_exec) + .map_err(|e| GeneralError(format!("rename ProjectionExec: {e}")))?, + ) + } else { + final_exec + }; + + Ok(( + vec![], + vec![], + Arc::new(SparkPlan::new(spark_plan.plan_id, with_rename, vec![])), + )) + } +} diff --git a/native/core/src/execution/planner/operator_registry.rs b/native/core/src/execution/planner/operator_registry.rs index eb31184461..316ada0a77 100644 --- a/native/core/src/execution/planner/operator_registry.rs +++ b/native/core/src/execution/planner/operator_registry.rs @@ -151,5 +151,6 @@ fn get_operator_type(spark_operator: &Operator) -> Option { OpStruct::Explode(_) => None, // Not yet in OperatorType enum OpStruct::CsvScan(_) => Some(OperatorType::CsvScan), OpStruct::ShuffleScan(_) => None, // Not yet in OperatorType enum + OpStruct::DeltaScan(_) => None, // Optional contrib; not in OperatorType enum } } diff --git a/native/core/src/parquet/missing_file_tolerant.rs b/native/core/src/parquet/missing_file_tolerant.rs new file mode 100644 index 0000000000..6432c9bd00 --- /dev/null +++ b/native/core/src/parquet/missing_file_tolerant.rs @@ -0,0 +1,232 @@ +// 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. + +//! Tolerant FileSource / FileOpener decorators for honouring Spark's +//! `spark.sql.files.ignoreMissingFiles`. +//! +//! DataFusion's `DataSourceExec` constructs its `FileStream` with the default +//! `OnError::Fail` and provides no public knob to flip it to `OnError::Skip`, +//! so Spark's "silently skip files that disappeared between planning and +//! execution" semantics cannot be opted into via a config. Instead we wrap the +//! inner `FileOpener`: when the opener's future resolves to a NotFound error +//! we return an empty `BoxStream` so the file contributes zero batches and the +//! `FileStream` simply moves to the next file. +//! +//! All other `FileSource` trait methods delegate to the inner source verbatim +//! so optimizer hooks (`try_pushdown_filters`, `repartitioned`, ...) keep +//! their normal behaviour. + +use arrow::array::RecordBatch; +use datafusion::common::Result; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileScanConfig, FileSource}; +use datafusion::physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr, PhysicalSortExpr}; +use datafusion::physical_plan::filter_pushdown::FilterPushdownPropagation; +use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion::physical_plan::projection::ProjectionExprs; +use datafusion::physical_plan::sort_pushdown::SortOrderPushdownResult; +use datafusion::physical_plan::DisplayFormatType; +use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; +use datafusion_datasource::table_schema::TableSchema; +use futures::{stream, FutureExt, StreamExt}; +use object_store::ObjectStore; +use std::any::Any; +use std::fmt::{self, Formatter}; +use std::sync::Arc; + +/// FileOpener decorator that converts NotFound errors from the wrapped +/// opener's future into an empty `BoxStream`. Any other error is propagated +/// unchanged so we don't paper over real corruption / IO problems. +pub(crate) struct IgnoreMissingFileOpener { + inner: Arc, +} + +impl IgnoreMissingFileOpener { + pub(crate) fn new(inner: Arc) -> Self { + Self { inner } + } +} + +fn is_not_found(err: &datafusion::error::DataFusionError) -> bool { + use datafusion::error::DataFusionError; + // object_store wraps NotFound into ObjectStore; parquet's reader may surface it via External. + // We inspect both the variant and its display chain for the literal "not found" / + // "NotFound" token because the precise DataFusion wrapping differs by error path. + let mut current: &(dyn std::error::Error + 'static) = err; + loop { + if let Some(os_err) = current.downcast_ref::() { + if matches!(os_err, object_store::Error::NotFound { .. }) { + return true; + } + } + if let Some(io_err) = current.downcast_ref::() { + if io_err.kind() == std::io::ErrorKind::NotFound { + return true; + } + } + match current.source() { + Some(src) => current = src, + None => break, + } + } + // Display-based fallback for adapters that erase the underlying type + // (e.g. parquet's ParquetError -> DataFusionError::External). Anchored to + // recognised NotFound phrasings only -- a loose substring match on "not found" + // would silently swallow unrelated parquet messages like "row group statistics + // not found" or "page index not found" and produce wrong empty results. + let msg = err.to_string(); + matches!(err, DataFusionError::External(_) | DataFusionError::ObjectStore(_)) + && (msg.contains("Object at location") + || msg.contains("Generic NotFound") + || msg.contains("NoSuchKey") + || msg.contains("NoSuchFile") + || msg.contains("No such file or directory")) +} + +impl FileOpener for IgnoreMissingFileOpener { + fn open(&self, partitioned_file: PartitionedFile) -> Result { + let inner_future = self.inner.open(partitioned_file)?; + Ok(Box::pin(inner_future.map(|opened| match opened { + Ok(stream) => Ok(stream), + Err(e) if is_not_found(&e) => { + let empty = stream::empty::>(); + Ok(empty.boxed()) + } + Err(e) => Err(e), + }))) + } +} + +/// FileSource decorator that wraps the inner source's `FileOpener` in +/// `IgnoreMissingFileOpener`. All other methods delegate verbatim. +#[derive(Clone)] +pub(crate) struct IgnoreMissingFileSource { + inner: Arc, +} + +impl IgnoreMissingFileSource { + pub(crate) fn new(inner: Arc) -> Arc { + Arc::new(Self { inner }) + } +} + +impl FileSource for IgnoreMissingFileSource { + fn create_file_opener( + &self, + object_store: Arc, + base_config: &FileScanConfig, + partition: usize, + ) -> Result> { + let inner_opener = + self.inner + .create_file_opener(object_store, base_config, partition)?; + Ok(Arc::new(IgnoreMissingFileOpener::new(inner_opener))) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn table_schema(&self) -> &TableSchema { + self.inner.table_schema() + } + + fn with_batch_size(&self, batch_size: usize) -> Arc { + // Re-wrap so the batch-sized clone still skips missing files. + IgnoreMissingFileSource::new(self.inner.with_batch_size(batch_size)) + } + + fn filter(&self) -> Option> { + self.inner.filter() + } + + fn projection(&self) -> Option<&ProjectionExprs> { + self.inner.projection() + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + self.inner.metrics() + } + + fn file_type(&self) -> &str { + self.inner.file_type() + } + + fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + self.inner.fmt_extra(t, f) + } + + fn supports_repartitioning(&self) -> bool { + self.inner.supports_repartitioning() + } + + fn repartitioned( + &self, + target_partitions: usize, + repartition_file_min_size: usize, + output_ordering: Option, + config: &FileScanConfig, + ) -> Result> { + self.inner.repartitioned( + target_partitions, + repartition_file_min_size, + output_ordering, + config, + ) + } + + fn try_pushdown_filters( + &self, + filters: Vec>, + config: &datafusion::config::ConfigOptions, + ) -> Result>> { + let prop = self.inner.try_pushdown_filters(filters, config)?; + // Re-wrap the updated_node so the post-pushdown FileSource keeps the + // missing-file tolerance. + Ok(FilterPushdownPropagation { + filters: prop.filters, + updated_node: prop.updated_node.map(IgnoreMissingFileSource::new), + }) + } + + fn try_pushdown_sort( + &self, + order: &[PhysicalSortExpr], + eq_properties: &EquivalenceProperties, + ) -> Result>> { + match self.inner.try_pushdown_sort(order, eq_properties)? { + SortOrderPushdownResult::Exact { inner } => Ok(SortOrderPushdownResult::Exact { + inner: IgnoreMissingFileSource::new(inner), + }), + SortOrderPushdownResult::Inexact { inner } => Ok(SortOrderPushdownResult::Inexact { + inner: IgnoreMissingFileSource::new(inner), + }), + SortOrderPushdownResult::Unsupported => Ok(SortOrderPushdownResult::Unsupported), + } + } + + fn try_pushdown_projection( + &self, + projection: &ProjectionExprs, + ) -> Result>> { + Ok(self + .inner + .try_pushdown_projection(projection)? + .map(IgnoreMissingFileSource::new)) + } +} + diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index 5de14aa610..58e55ca132 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -29,6 +29,7 @@ pub mod read; pub mod schema_adapter; mod cast_column; +mod missing_file_tolerant; mod objectstore; use std::collections::HashMap; @@ -519,6 +520,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBat // so the native side does not need to do field-ID matching here. false, false, + // ignore_missing_files: iceberg-compat doesn't surface Spark's flag today. + false, )?; let partition_index: usize = 0; diff --git a/native/core/src/parquet/parquet_exec.rs b/native/core/src/parquet/parquet_exec.rs index 38a0755658..f497a2ac7b 100644 --- a/native/core/src/parquet/parquet_exec.rs +++ b/native/core/src/parquet/parquet_exec.rs @@ -17,6 +17,7 @@ use crate::execution::operators::ExecutionError; use crate::parquet::encryption_support::{CometEncryptionConfig, ENCRYPTION_FACTORY_ID}; +use crate::parquet::missing_file_tolerant::IgnoreMissingFileSource; use crate::parquet::parquet_read_cached_factory::CachingParquetReaderFactory; use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; @@ -74,6 +75,10 @@ pub(crate) fn init_datasource_exec( encryption_enabled: bool, use_field_id: bool, ignore_missing_field_id: bool, + // When true, files that fail to open with NotFound errors are silently skipped + // (matches Spark's `spark.sql.files.ignoreMissingFiles=true`). Wired through to + // a `FileSource` decorator that swallows the error as an empty stream. + ignore_missing_files: bool, ) -> Result, ExecutionError> { let (table_parquet_options, mut spark_parquet_options) = get_options( session_timezone, @@ -168,6 +173,15 @@ pub(crate) fn init_datasource_exec( _ => Arc::new(parquet_source), }; + // Honour Spark's `ignoreMissingFiles` by wrapping the final FileSource so its + // FileOpener swallows object-store NotFound errors as empty streams. Wrapped + // after pushdown to preserve all optimizer behaviour. + let file_source: Arc = if ignore_missing_files { + IgnoreMissingFileSource::new(file_source) + } else { + file_source + }; + let expr_adapter_factory: Arc = Arc::new( SparkPhysicalExprAdapterFactory::new(spark_parquet_options, default_values), ); diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 7cefe06da7..61f3bd7062 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -53,6 +53,10 @@ message Operator { Explode explode = 114; CsvScan csv_scan = 115; ShuffleScan shuffle_scan = 116; + // Delta Lake scan. Wire format used by `contrib/delta/`. Only decoded when + // core is built with `--features contrib-delta`; in default builds the + // dispatcher arm is `#[cfg]`-stubbed out so the contrib has zero runtime cost. + DeltaScan delta_scan = 117; } } @@ -265,6 +269,84 @@ message IcebergDeleteFile { uint64 file_size_in_bytes = 5; } +// ===================================================================================== +// Delta Lake scan messages -- consumed by the optional `contrib/delta/` integration. +// Field numbers must remain stable; older serialised plans encode them on the wire. +// ===================================================================================== + +// Per-scan invariants. Lives at the head of every Delta scan operator payload. +message DeltaScanCommon { + // Required schema (projection applied to data schema). + repeated SparkStructField required_schema = 1; + // Full data schema of the table (on-disk parquet schema). + repeated SparkStructField data_schema = 2; + // Partition columns (stored in add actions, not in file paths). + repeated SparkStructField partition_schema = 3; + // Pushed-down data filters (Catalyst translated). + repeated spark.spark_expression.Expr data_filters = 4; + // Column projection indices into (data_schema ++ partition_schema). + repeated int64 projection_vector = 5; + string session_timezone = 6; + bool case_sensitive = 7; + // Cloud storage credentials / options, keyed by Hadoop-style names. + map object_store_options = 8; + // Root path of the Delta table (file:// or s3://...). + string table_root = 9; + // Optional snapshot version; unset = latest. + optional uint64 snapshot_version = 10; + // Per-file parallelism knob within a single Spark task. + uint32 data_file_concurrency_limit = 11; + // Informational label for debug output. + string source = 12; + // Logical->physical column mapping (column_mapping_mode = id or name). + repeated DeltaColumnMapping column_mappings = 13; + optional string materialized_row_id_column_name = 14; + optional string materialized_row_commit_version_column_name = 15; + // When true the DataSourceExec swallows object-store NotFound errors as empty + // streams; matches Spark's `spark.sql.files.ignoreMissingFiles=true` semantics. + bool ignore_missing_files = 16; +} + +// Operator payload for a Delta scan. +message DeltaScan { + DeltaScanCommon common = 1; + repeated DeltaScanTask tasks = 2; +} + +message DeltaScanTask { + string file_path = 1; + uint64 file_size = 2; + optional uint64 record_count = 3; + repeated DeltaPartitionValue partition_values = 4; + // Pre-materialised deleted row indexes (kernel resolves the DV bitmap on the driver). + // Empty = no DV on this file. + repeated uint64 deleted_row_indexes = 5; + optional int64 base_row_id = 6; + optional int64 default_row_commit_version = 7; + // File-splitting byte range. Both unset = read whole file. + optional uint64 byte_range_start = 8; + optional uint64 byte_range_end = 9; +} + +message DeltaPartitionValue { + string name = 1; + optional string value = 2; +} + +// JNI return value from planDeltaScan -- the list of tasks for a given snapshot. +message DeltaScanTaskList { + uint64 snapshot_version = 1; + string table_root = 2; + repeated DeltaScanTask tasks = 3; + repeated string unsupported_features = 4; + repeated DeltaColumnMapping column_mappings = 5; +} + +message DeltaColumnMapping { + string logical_name = 1; + string physical_name = 2; +} + message Projection { repeated spark.spark_expression.Expr project_list = 1; } diff --git a/native/shuffle/src/spark_unsafe/unsafe_object.rs b/native/shuffle/src/spark_unsafe/unsafe_object.rs index f32ea8c23b..b810b7f771 100644 --- a/native/shuffle/src/spark_unsafe/unsafe_object.rs +++ b/native/shuffle/src/spark_unsafe/unsafe_object.rs @@ -19,7 +19,6 @@ use super::list::SparkUnsafeArray; use super::map::SparkUnsafeMap; use super::row::SparkUnsafeRow; use datafusion_comet_common::bytes_to_i128; -use std::str::from_utf8; const MAX_LONG_DIGITS: u8 = 18; @@ -75,19 +74,25 @@ pub trait SparkUnsafeObject { } /// Returns string value at the given index of the object. + /// + /// Spark's `UnsafeRow.getUTF8String` wraps the bytes via `UTF8String.fromAddress` + /// with no UTF-8 validation, and Spark's `cast(BinaryType -> StringType)` is a + /// zero-copy reinterpret that can leave arbitrary bytes in a `StringType` column + /// (e.g. Delta's Z-Order `interleave_bits(...).cast(StringType)`). Strict validation + /// here panics on those rows even though Spark itself treats them as opaque. The + /// bytes flow directly into Arrow's `StringBuilder::append_value`, which copies them + /// without introspection. fn get_string(&self, index: usize) -> &str { let (offset, len) = self.get_offset_and_len(index); let addr = self.get_row_addr() + offset as i64; - // SAFETY: addr points to valid UTF-8 string data within the variable-length region. - // Offset and length are read from the fixed-length portion of the row/array. debug_assert!(addr != 0, "get_string: null address at index {index}"); debug_assert!( len >= 0, "get_string: negative length {len} at index {index}" ); let slice: &[u8] = unsafe { std::slice::from_raw_parts(addr as *const u8, len as usize) }; - - from_utf8(slice).unwrap() + // SAFETY: matches Spark's `UTF8String.fromAddress` no-validate semantics. See doc above. + unsafe { std::str::from_utf8_unchecked(slice) } } /// Returns binary value at the given index of the object. diff --git a/spark/pom.xml b/spark/pom.xml index d3c18ccf87..bb4fb308fe 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -323,6 +323,51 @@ under the License. + + + contrib-delta + + 4.1.0 + + + + io.delta + delta-spark_${scala.binary.version} + ${delta.version} + provided + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-contrib-delta-source + generate-sources + + add-source + + + + ${project.parent.basedir}/contrib/delta/src/main/scala + + + + + + + + + generate-docs diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index 6140eca553..c221e346af 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -68,7 +68,8 @@ class CometExecIterator( partitionIndex: Int, broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, encryptedFilePaths: Seq[String] = Seq.empty, - shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty) + shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty, + taskFilePaths: Seq[String] = Seq.empty) extends Iterator[ColumnarBatch] with Logging { @@ -177,18 +178,19 @@ class CometExecIterator( // threw the exception, so we log the exception with taskAttemptId here logError(s"Native execution for task $taskAttemptId failed", e) + // `(?s)` so `.` matches across newlines -- native parquet errors sometimes + // span multiple lines (footer corruption messages include a hex dump), and a + // single-line regex would let those slip past the wrap and surface as bare + // CometNativeException to the user. val parquetError: scala.util.matching.Regex = - """^Parquet error: (?:.*)$""".r + """(?s)^Parquet error: (?:.*)$""".r e.getMessage match { case parquetError() => - // See org.apache.spark.sql.errors.QueryExecutionErrors.failedToReadDataError - // See org.apache.parquet.hadoop.ParquetFileReader for error message. - // _LEGACY_ERROR_TEMP_2254 has no message placeholders; Spark 4 strict-checks - // parameters and raises INTERNAL_ERROR if any are passed. - throw new SparkException( - errorClass = "_LEGACY_ERROR_TEMP_2254", - messageParameters = Map.empty, - cause = new SparkException("File is not a Parquet file.", e)) + // Wrap in the FAILED_READ_FILE.NO_HINT SparkException Spark produces when + // its own parquet reader fails. The shim accesses spark-private APIs + // (InputFileBlockHolder, QueryExecutionErrors) from a Spark-package class. + throw org.apache.spark.sql.comet.shims.ShimSparkErrorConverter + .wrapNativeParquetError(e, taskFilePaths) case _ => throw e } diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 72c2bea9e4..7266dfa706 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -261,6 +261,14 @@ case class CometExecRule(session: SparkSession) case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => convertToComet(scan, CometNativeScan).getOrElse(scan) + // Delta scan marker stamped by the optional contrib/delta integration. + // The handler is resolved via reflection (no compile-time dependency + // on the contrib) -- present only when -Pcontrib-delta was activated. + case scan: CometScanExec if scan.scanImpl == DeltaIntegration.DeltaScanImpl => + DeltaIntegration.scanHandler + .flatMap(handler => convertToComet(scan, handler)) + .getOrElse(scan) + // Fully native Iceberg scan for V2 (iceberg-rust path) // Only handle scans with native metadata; other scans fall through to isCometScan // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 64b69be1e9..9355252d71 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -163,9 +163,35 @@ case class CometScanRule(session: SparkSession) scanExec.relation match { case r: HadoopFsRelation => + // Try the optional Delta contrib first. When this build wasn't compiled with + // `-Pcontrib-delta`, the bridge returns None and we fall through to the + // vanilla scan path. When the Delta classes are on the classpath, the contrib + // either claims the scan (returning a CometScanExec marker) or declines via + // its own `withInfo` fallback message. + DeltaIntegration.transformV1IfDelta(plan, session, scanExec, r) match { + case Some(handled) => return handled + case None => // proceed with vanilla logic + } if (!CometScanExec.isFileFormatSupported(r.fileFormat)) { return withInfo(scanExec, s"Unsupported file format ${r.fileFormat}") } + // Filesystem scheme allowlist. Comet's native readers go through object_store, + // which only understands a fixed set of URL schemes. Custom Hadoop FileSystems + // (e.g. a test FakeFileSystem registered via spark.hadoop.fs..impl) would + // surface at execution time as `Generic URL error: Unable to recognise URL "..."`. + // Decline here so Spark's reader -- which goes through the Hadoop FS API and can + // resolve custom schemes -- handles the scan. + val supportedSchemes = + Set("file", "s3", "s3a", "gs", "gcs", "oss", "abfss", "abfs", "wasbs", "wasb") + val unsupportedFsSchemes = r.location.rootPaths + .map(p => p.toUri.getScheme) + .filter(s => s != null && !supportedSchemes.contains(s)) + .toSet + if (unsupportedFsSchemes.nonEmpty) { + return withInfo( + scanExec, + s"Unsupported filesystem schemes: ${unsupportedFsSchemes.mkString(", ")}") + } val hadoopConf = r.sparkSession.sessionState.newHadoopConfWithOptions(r.options) // TODO is this restriction valid for all native scan types? diff --git a/spark/src/main/scala/org/apache/comet/rules/DeltaIntegration.scala b/spark/src/main/scala/org/apache/comet/rules/DeltaIntegration.scala new file mode 100644 index 0000000000..0e80a6fb74 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/rules/DeltaIntegration.scala @@ -0,0 +1,151 @@ +/* + * 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. + */ + +package org.apache.comet.rules + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.comet.CometScanExec +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.HadoopFsRelation + +import org.apache.comet.serde.CometOperatorSerde + +/** + * Reflection-based bridge to the optional `contrib/delta/` integration. + * + * On default builds the contrib classes don't exist on the classpath, so the reflective class + * lookups fail and every method here returns the "not handled" sentinel. On builds compiled with + * `-Pcontrib-delta` (Maven) + `--features contrib-delta` (Cargo), the contrib classes are present + * and the lookups resolve, dispatching the call into the contrib helpers. + * + * Keeping this bridge as one small file in core lets the Delta detection block in `CometScanRule` + * and the serde dispatch in `CometExecRule` stay ~10 lines each -- exactly the shape Parth's + * review on #4339 asked for. + * + * No `SPI`, no `ServiceLoader`, no registry: the contrib provides its own static helper objects + * with stable names; this bridge just calls them. + */ +object DeltaIntegration { + + private val ScanRuleClass = "org.apache.comet.contrib.delta.DeltaScanRule" + private val SerdeClass = "org.apache.comet.contrib.delta.CometDeltaNativeScan" + + /** scanImpl tag the contrib stamps on CometScanExec markers it produces. */ + val DeltaScanImpl: String = "native_delta_compat" + + // Lazy class lookups -- single reflection cost per JVM, cached either as the + // class handle or as the empty option if the contrib wasn't bundled. + @volatile private var scanRuleLookup: Option[Option[Class[AnyRef]]] = None + @volatile private var serdeLookup: Option[Option[Class[AnyRef]]] = None + + private def scanRuleCls: Option[Class[AnyRef]] = + scanRuleLookup.getOrElse { + val cls = + try { + // scalastyle:off classforname + Some(Class.forName(ScanRuleClass).asInstanceOf[Class[AnyRef]]) + // scalastyle:on classforname + } catch { case _: ClassNotFoundException => None } + scanRuleLookup = Some(cls) + cls + } + + private def serdeCls: Option[Class[AnyRef]] = + serdeLookup.getOrElse { + val cls = + try { + // scalastyle:off classforname + Some(Class.forName(SerdeClass).asInstanceOf[Class[AnyRef]]) + // scalastyle:on classforname + } catch { case _: ClassNotFoundException => None } + serdeLookup = Some(cls) + cls + } + + /** True when the Delta contrib was bundled into this build. */ + def isAvailable: Boolean = scanRuleCls.isDefined + + /** + * Delegate the V1 scan transform to the Delta contrib when both (a) the contrib is on the + * classpath, AND (b) the relation's file format is `DeltaParquetFileFormat`. + * + * Returns `Some(plan)` if the contrib handled the scan (either with a transformed + * `CometScanExec` marker or by explicitly declining via the `withInfo` path); `None` to + * indicate "not a Delta scan, proceed with the vanilla CometScanRule path". + */ + // Cached reflective binding: resolved once per JVM. The contrib's + // `transformV1IfDelta` is invoked for every V1 scan in every plan, even + // non-Delta ones; resolving the Method on each call would be a per-scan + // reflection round-trip just to find we don't apply. + @volatile private var transformV1IfDeltaBindingCache + : Option[Option[(AnyRef, java.lang.reflect.Method)]] = None + + private def transformV1IfDeltaBinding: Option[(AnyRef, java.lang.reflect.Method)] = + transformV1IfDeltaBindingCache.getOrElse { + val binding = scanRuleCls.flatMap { cls => + try { + val module = cls.getField("MODULE$").get(null) + val m = cls.getMethod( + "transformV1IfDelta", + classOf[SparkPlan], + classOf[SparkSession], + classOf[FileSourceScanExec], + classOf[HadoopFsRelation]) + Some((module, m)) + } catch { + case _: Exception => None + } + } + transformV1IfDeltaBindingCache = Some(binding) + binding + } + + def transformV1IfDelta( + plan: SparkPlan, + session: SparkSession, + scanExec: FileSourceScanExec, + relation: HadoopFsRelation): Option[SparkPlan] = { + transformV1IfDeltaBinding.flatMap { case (module, m) => + try { + Option(m.invoke(module, plan, session, scanExec, relation)) + .map(_.asInstanceOf[Option[SparkPlan]]) + .flatten + } catch { + // scalastyle:off + case _: Exception => None + // scalastyle:on + } + } + } + + /** + * The Delta scan handler, resolved via reflection from the contrib's `CometDeltaNativeScan` + * companion object. Returns `None` when the contrib isn't bundled into this build. + * `CometExecRule` calls this and passes the result through the standard `convertToComet(scan, + * handler)` path so the Delta scan flows through the same code as `CometNativeScan` etc. + */ + def scanHandler: Option[CometOperatorSerde[CometScanExec]] = serdeCls.flatMap { cls => + try { + val module = cls.getField("MODULE$").get(null) + Some(module.asInstanceOf[CometOperatorSerde[CometScanExec]]) + } catch { + case _: Exception => None + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/arrays.scala b/spark/src/main/scala/org/apache/comet/serde/arrays.scala index 5edc08840a..fdbb5b206a 100644 --- a/spark/src/main/scala/org/apache/comet/serde/arrays.scala +++ b/spark/src/main/scala/org/apache/comet/serde/arrays.scala @@ -480,6 +480,26 @@ object CometCreateArray extends CometExpressionSerde[CreateArray] { return exprToProtoInternal(emptyArrayLiteral, inputs, binding) } + // DataFusion's `make_array` asserts strict element-type equality (down to nested + // field nullability) via `MutableArrayData::with_capacities`. Spark's CreateArray + // is more permissive: children may share the same surface type (e.g. all + // `Struct`) but differ only in nested field nullability when + // the analyzer didn't insert coercion casts -- Delta's CDC write path builds + // these `array(struct(...), struct(...))` plans manually (one struct per change + // type) and can leave `_change_type` non-nullable in one arm and nullable in + // another. Native execution then panics inside `make_array_inner`. Decline here + // when any pair of children disagree on data type so Spark's JVM evaluator + // (which doesn't have this strictness) handles it. + val childTypes = children.map(_.dataType) + if (childTypes.distinct.size > 1) { + withInfo( + expr, + "CreateArray children have mismatched data types: " + + childTypes.distinct.mkString(", "), + children: _*) + return None + } + val childExprs = children.map(exprToProtoInternal(_, inputs, binding)) if (childExprs.forall(_.isDefined)) { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala index 47eda98a11..321e433457 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala @@ -36,7 +36,8 @@ import org.apache.comet.serde.OperatorOuterClass private[spark] class CometExecPartition( override val index: Int, val inputPartitions: Array[Partition], - val planDataByKey: Map[String, Array[Byte]]) + val planDataByKey: Map[String, Array[Byte]], + val filePaths: Seq[String] = Seq.empty) extends Partition /** @@ -66,7 +67,8 @@ private[spark] class CometExecRDD( subqueries: Seq[ScalarSubquery], broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, encryptedFilePaths: Seq[String] = Seq.empty, - shuffleScanIndices: Set[Int] = Set.empty) + shuffleScanIndices: Set[Int] = Set.empty, + @transient perPartitionFilePaths: Array[Seq[String]] = Array.empty) extends RDD[ColumnarBatch](sc, inputRDDs.map(rdd => new OneToOneDependency(rdd))) { // Determine partition count: from inputs if available, otherwise from parameter @@ -90,13 +92,33 @@ private[spark] class CometExecRDD( (0 until numPartitions).map { idx => val inputParts = inputRDDs.map(_.partitions(idx)).toArray val planData = perPartitionByKey.map { case (key, arr) => key -> arr(idx) } - new CometExecPartition(idx, inputParts, planData) + val fp = + if (perPartitionFilePaths.length > idx) perPartitionFilePaths(idx) else Seq.empty[String] + new CometExecPartition(idx, inputParts, planData, fp) }.toArray } override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { val partition = split.asInstanceOf[CometExecPartition] + // Populate Spark's `InputFileBlockHolder` thread-local so `input_file_name()` and + // friends return the correct path for this task's data. Comet's native scan does + // not go through Spark's `FileScanRDD` (which is what normally maintains this + // thread-local), so without this hook Delta's UPDATE/DELETE/MERGE flows -- which + // rely on `input_file_name()` to identify touched files -- silently see an empty + // path. Set to the partition's first file: Delta forces one-task-per-partition + // when `input_file_name()` is referenced (see DeltaScanRule), so there is exactly + // one file in that case. Registers an unset on task completion to avoid leaking + // across tasks on the same executor thread. + if (partition.filePaths.nonEmpty) { + org.apache.spark.rdd.InputFileBlockHolder.set(partition.filePaths.head, 0L, 0L) + Option(context).foreach { ctx => + ctx.addTaskCompletionListener[Unit] { _ => + org.apache.spark.rdd.InputFileBlockHolder.unset() + } + } + } + val inputs = inputRDDs.zip(partition.inputPartitions).map { case (rdd, part) => rdd.iterator(part, context) } @@ -130,7 +152,8 @@ private[spark] class CometExecRDD( partition.index, broadcastedHadoopConfForEncryption, encryptedFilePaths, - shuffleBlockIters) + shuffleBlockIters, + taskFilePaths = partition.filePaths) // Register ScalarSubqueries so native code can look them up subqueries.foreach(sub => CometScalarSubquery.setSubquery(it.id, sub)) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala index f4f0b1fb74..6247d8727a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeScanExec.scala @@ -156,7 +156,8 @@ case class CometNativeScanExec( * all files for all partitions in the driver, we serialize only common metadata (once) and each * partition's files (lazily, as tasks are scheduled). */ - @transient private lazy val serializedPartitionData: (Array[Byte], Array[Array[Byte]]) = { + @transient private lazy val serializedPartitionData + : (Array[Byte], Array[Array[Byte]], Array[Seq[String]]) = { // Outer partitionFilters (wrapper) DPP is resolved by Spark's standard // prepare -> waitForSubqueries lifecycle, triggered explicitly via // CometLeafExec.ensureSubqueriesResolved called from @@ -227,13 +228,21 @@ case class CometNativeScanExec( partitionNativeScan.toByteArray }.toArray - (commonBytes, perPartitionBytes) + // File paths per partition -- threaded through CometExecRDD to CometExecIterator + // so wrapNativeParquetError can populate FAILED_READ_FILE.NO_HINT exceptions with + // the actual file path. CometNativeScanExec bypasses Spark's FileScanRDD, so the + // standard InputFileBlockHolder thread-local isn't set. + val perPartitionPaths = filePartitions.map(_.files.map(_.filePath.toString).toSeq).toArray + + (commonBytes, perPartitionBytes, perPartitionPaths) } def commonData: Array[Byte] = serializedPartitionData._1 def perPartitionData: Array[Array[Byte]] = serializedPartitionData._2 + def perPartitionFilePaths: Array[Seq[String]] = serializedPartitionData._3 + override def doExecuteColumnar(): RDD[ColumnarBatch] = { val nativeMetrics = CometMetricNode.fromCometPlan(this) val serializedPlan = CometExec.serializeNativePlan(nativeOp) @@ -261,7 +270,8 @@ case class CometNativeScanExec( nativeMetrics, Seq.empty, broadcastedHadoopConfForEncryption, - encryptedFilePaths) { + encryptedFilePaths, + perPartitionFilePaths = perPartitionFilePaths) { override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { val res = super.compute(split, context) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index f315aae6e2..838214eb23 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -66,6 +66,12 @@ import org.apache.comet.serde.operator.CometSink */ private[comet] trait PlanDataInjector { + /** + * Which `OpStructCase` this injector cares about. Used by `injectPlanData` for an O(1) + * pre-filter so we don't run every injector's `canInject` against every operator in the tree. + */ + def opStructCase: Operator.OpStructCase + /** Check if this injector can handle the given operator. */ def canInject(op: Operator): Boolean @@ -81,12 +87,32 @@ private[comet] trait PlanDataInjector { */ private[comet] object PlanDataInjector { - // Registry of injectors for different operator types - private val injectors: Seq[PlanDataInjector] = Seq( - IcebergPlanDataInjector, - NativeScanPlanDataInjector - // Future: DeltaPlanDataInjector, HudiPlanDataInjector, etc. - ) + // Registry of injectors for different operator types. The contrib/delta integration's + // DeltaPlanDataInjector is appended via one reflective class lookup -- present only when + // the contrib was bundled (i.e. -Pcontrib-delta on the Maven build). Default builds get + // the empty Option and an unmodified injectors list, so there's zero contrib surface at + // runtime on default builds. + private val injectors: Seq[PlanDataInjector] = { + val builtin: Seq[PlanDataInjector] = Seq(IcebergPlanDataInjector, NativeScanPlanDataInjector) + val deltaOpt: Option[PlanDataInjector] = + try { + // scalastyle:off classforname + val cls = Class.forName("org.apache.spark.sql.comet.DeltaPlanDataInjector") + // scalastyle:on classforname + Some(cls.getField("MODULE$").get(null).asInstanceOf[PlanDataInjector]) + } catch { + case _: ClassNotFoundException => None + case _: Exception => None + } + builtin ++ deltaOpt + } + + // O(1) lookup by op kind: most operators in any tree don't match any injector, so the + // per-op `for (injector <- injectors if injector.canInject(op))` walk was paying N*M + // canInject calls (N operators, M injectors) just to find no match. Keying by + // `OpStructCase` lets us skip the iteration entirely for non-scan operators. + private val injectorsByKind: Map[Operator.OpStructCase, PlanDataInjector] = + injectors.map(i => i.opStructCase -> i).toMap /** * Injects planning data into an Operator tree by finding nodes that need injection and applying @@ -101,21 +127,24 @@ private[comet] object PlanDataInjector { partitionByKey: Map[String, Array[Byte]]): Operator = { val builder = op.toBuilder - // Try each injector to see if it can handle this operator - for (injector <- injectors if injector.canInject(op)) { - injector.getKey(op) match { - case Some(key) => - (commonByKey.get(key), partitionByKey.get(key)) match { - case (Some(commonBytes), Some(partitionBytes)) => - val injectedOp = injector.inject(op, commonBytes, partitionBytes) - // Copy the injected operator's fields to our builder - builder.clear() - builder.mergeFrom(injectedOp) - case _ => - throw new CometRuntimeException(s"Missing planning data for key: $key") - } - case None => - } + // O(1) by op kind, then a canInject confirm (which may inspect detail fields like + // `hasCommon` / `!hasFilePartition`). Most operators in any tree are non-scan and skip. + injectorsByKind.get(op.getOpStructCase) match { + case Some(injector) if injector.canInject(op) => + injector.getKey(op) match { + case Some(key) => + (commonByKey.get(key), partitionByKey.get(key)) match { + case (Some(commonBytes), Some(partitionBytes)) => + val injectedOp = injector.inject(op, commonBytes, partitionBytes) + // Copy the injected operator's fields to our builder + builder.clear() + builder.mergeFrom(injectedOp) + case _ => + throw new CometRuntimeException(s"Missing planning data for key: $key") + } + case None => + } + case _ => } // Recursively process children @@ -159,6 +188,8 @@ private[comet] object IcebergPlanDataInjector extends PlanDataInjector { } }) + override val opStructCase: Operator.OpStructCase = Operator.OpStructCase.ICEBERG_SCAN + override def canInject(op: Operator): Boolean = op.hasIcebergScan && op.getIcebergScan.getFileScanTasksCount == 0 && @@ -198,6 +229,8 @@ private[comet] object IcebergPlanDataInjector extends PlanDataInjector { */ private[comet] object NativeScanPlanDataInjector extends PlanDataInjector { + override val opStructCase: Operator.OpStructCase = Operator.OpStructCase.NATIVE_SCAN + override def canInject(op: Operator): Boolean = op.hasNativeScan && op.getNativeScan.hasCommon && @@ -560,6 +593,20 @@ abstract class CometNativeExec extends CometExec { // Unified RDD creation - CometExecRDD handles all cases val subqueries = collectSubqueries(this) val hasScanInput = sparkPlans.exists(_.isInstanceOf[CometNativeScanExec]) + // Collect per-partition file paths from any CometNativeScanExec leaves so + // CometExecIterator can populate FAILED_READ_FILE.NO_HINT exceptions with + // the actual path. Multiple scans (joins) get concatenated per partition. + val perPartitionFilePaths: Array[Seq[String]] = { + val scans = sparkPlans.collect { case s: CometNativeScanExec => s } + if (scans.isEmpty) Array.empty[Seq[String]] + else { + val perScan = scans.map(_.perPartitionFilePaths) + val n = firstNonBroadcastPlanNumPartitions + (0 until n).map { idx => + perScan.flatMap { arr => if (arr.length > idx) arr(idx) else Seq.empty }.toSeq + }.toArray + } + } new CometExecRDD( sparkContext, inputs.toSeq, @@ -572,7 +619,8 @@ abstract class CometNativeExec extends CometExec { subqueries, broadcastedHadoopConfForEncryption, encryptedFilePaths, - shuffleScanIndices) { + shuffleScanIndices, + perPartitionFilePaths = perPartitionFilePaths) { override def compute( split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { diff --git a/spark/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimSparkErrorConverter.scala b/spark/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimSparkErrorConverter.scala index ad5481c377..fb99cb1de4 100644 --- a/spark/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimSparkErrorConverter.scala +++ b/spark/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimSparkErrorConverter.scala @@ -34,6 +34,39 @@ import org.apache.comet.CometSparkSessionExtensions.isSpark41Plus object ShimSparkErrorConverter { val ObjectLocationPattern: Regex = "Object at location (.+?) not found".r + + /** + * Wrap a native parquet failure (e.g. "Parquet error: Invalid Parquet file. Corrupt footer") in + * the FAILED_READ_FILE.NO_HINT SparkException Spark itself produces when its own parquet reader + * fails. Matches what Spark/Delta tests assert on (e.g. SnapshotManagementSuite "should not + * recover when the current checkpoint is broken" asserts the message contains "Encountered + * error while reading file" + the path). + * + * File path is queried from Spark's `InputFileBlockHolder` thread-local. Falls back to an empty + * string when not set; the static phrasing still passes the assertion. + */ + def wrapNativeParquetError( + cause: Throwable, + taskFilePaths: Seq[String] = Seq.empty): Throwable = { + // Prefer the per-task file list threaded in from CometExecIterator (set by + // CometExecRDD via CometNativeExec/CometNativeScanExec collecting file paths from + // any scan leaves in the plan). Comet's native scan path does NOT go through + // Spark's FileScanRDD, so InputFileBlockHolder is typically not populated. Fall + // back to InputFileBlockHolder for any path that does set it. Tests like + // SnapshotManagementSuite assert the error message contains the file path, so an + // accurate path here is load-bearing. + val filePath = if (taskFilePaths.nonEmpty) { + taskFilePaths.mkString(",") + } else { + try { + val p = org.apache.spark.rdd.InputFileBlockHolder.getInputFilePath + if (p == null) "" else p.toString + } catch { + case _: Throwable => "" + } + } + QueryExecutionErrors.cannotReadFilesError(cause, filePath) + } } /**