diff --git a/.claude/skills/wire-datafusion-function/SKILL.md b/.claude/skills/wire-datafusion-function/SKILL.md new file mode 100644 index 0000000000..c03f0c4d80 --- /dev/null +++ b/.claude/skills/wire-datafusion-function/SKILL.md @@ -0,0 +1,131 @@ +--- +name: wire-datafusion-function +description: Use when wiring an existing DataFusion or datafusion-spark function into Comet for a Spark expression. Identifies the right wiring pattern (one-line passthrough, datafusion-spark UDF registration, or custom serde with input massaging / restrictions), applies the Scala serde, registers the UDF in jni_api when needed, and adds SQL file tests. Assumes the function already exists upstream — if not, switch to `implement-comet-expression`. +argument-hint: +--- + +Wire Comet support for the `$ARGUMENTS` Spark expression by reusing an existing DataFusion or `datafusion-spark` function. **No native Rust is written here** — if upstream coverage is missing, stop and run `implement-comet-expression`. + +## Wiring patterns + +Pick the lightest one that satisfies the Spark contract. + +| Pattern | When | What to change | +| ------------------------------ | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| **A** — passthrough | DF built-in (e.g. `acos`), or `datafusion-spark` UDF already registered in `register_datafusion_spark_function` | one line in the right map of `QueryPlanSerde.scala`: `classOf[Foo] -> CometScalarFunction("foo")` | +| **B** — register + passthrough | `datafusion-spark` UDF, _not_ yet registered, semantics already match Spark | Pattern A line **plus** `session_ctx.register_udf(ScalarUDF::new_from_impl(SparkFoo::default()));` in `native/core/src/execution/jni_api.rs::register_datafusion_spark_function` | +| **C** — custom serde | Inputs need preprocessing (cast, `nullIfNegative`, `+0.0` flip), or you need to set return type / `failOnError`, restrict input types via `getSupportLevel`, enforce foldable-only args, or attach `getCompatibleNotes`/`getIncompatibleReasons`/`getUnsupportedReasons` | new `CometXxx` object in the topic file (`math.scala`, `strings.scala`, …); see `CometCeil`, `CometAtan2`, `CometLog`, `CometSha2`, `CometAbs` | + +Function names must match: the string passed to `CometScalarFunction("xyz")` equals `SparkXyz::name()` upstream. + +## Workflow + +### 1. Study the Spark contract + +Find the `case class $ARGUMENTS` (PascalCase). Prefer the user's local Spark clone (`CLAUDE.md` / project memory); fall back to `git clone --depth 1 https://github.com/apache/spark.git /tmp/spark-master`. If the clone is sandbox-blocked and no local clone is found, **stop and ask** — Spark is the ground truth. + +Note `inputTypes`, `dataType`, `eval` / `nullSafeEval`, `require` guards, ANSI mode branches, and foldable-only arguments. These define what Comet must reproduce. + +### 2. Find the upstream function (at the pinned version) + +The pinned versions in `native/Cargo.toml` are the ground truth — a function on upstream `main` may not exist in the version Comet depends on. Resolve versions first, then search sources matching those exact versions. **Pick one source path; do not mix.** + +```bash +REPO_ROOT=$(git rev-parse --show-toplevel) +# Portable regex — BSD awk on macOS does not support \b. +DF_SPARK_VER=$(awk -F'"' '/^datafusion-spark[ =]/ {print $2; exit}' "$REPO_ROOT/native/Cargo.toml") +DF_FUNCS_VER=$(awk -F'"' '/^datafusion[ =]/ {print $2; exit}' "$REPO_ROOT/native/Cargo.toml") +[ -z "$DF_SPARK_VER" ] || [ -z "$DF_FUNCS_VER" ] && { echo "ERROR: version extraction failed"; exit 1; } +EXPR='$ARGUMENTS' +``` + +**Option A — cargo registry (preferred):** + +```bash +DF_SPARK=$(ls -d ~/.cargo/registry/src/*/datafusion-spark-${DF_SPARK_VER}/ 2>/dev/null | head -1) +DF_FUNCS=$(ls -d ~/.cargo/registry/src/*/datafusion-functions-${DF_FUNCS_VER}/ 2>/dev/null | head -1) +# If empty, run `cargo fetch` from native/. +grep -rin "fn name" "$DF_SPARK/src/function/" 2>/dev/null | grep -i "$EXPR" +grep -rin "fn name" "$DF_FUNCS/src/" 2>/dev/null | grep -i "$EXPR" +``` + +**Option B — local DataFusion clone** (only if `CLAUDE.md` / memory points at one). Grep at the pinned tag — DataFusion uses lightweight tags (``, no `v` prefix), so use `tag -l` not `rev-parse --verify ^{tag}`: + +```bash +DF_CLONE= +[ -z "$(git -C "$DF_CLONE" tag -l "$DF_SPARK_VER")" ] && echo "WARNING: tag missing — git fetch --tags or use Option A" +git -C "$DF_CLONE" grep -in "fn name" "$DF_SPARK_VER" -- 'datafusion/spark/src/function/' | grep -i "$EXPR" +git -C "$DF_CLONE" grep -in "fn name" "$DF_FUNCS_VER" -- 'datafusion/functions/src/' | grep -i "$EXPR" +``` + +Never grep the clone's working tree directly — it may be on `main`. + +### 2a. Decision gate — confirm the source crate + +- **Found in `datafusion-spark`** → proceed (Spark-tuned by design). If listed in `register_datafusion_spark_function` → Pattern A; otherwise Pattern B. +- **Found only in `datafusion-functions`** → **STOP and `AskUserQuestion`**. Pure DataFusion follows standard SQL semantics and frequently diverges from Spark (NULL vs error, negatives, overflow, return type, NaN). Surface what you found in each crate and the divergences you've already identified, then offer: + 1. Wire from `datafusion-functions` with Pattern C bridging — list the divergences and the masking/casting that closes them. + 2. Stop and switch to `implement-comet-expression` to port to `datafusion-spark` upstream first. +- **Found in neither** → stop and run `implement-comet-expression`. + +If semantics diverge in a way Pattern C can't bridge, escalate. + +### 3. Apply the Scala wiring + +Add the entry to the matching map in `spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala`: `mathExpressions`, `stringExpressions`, `arrayExpressions`, `mapExpressions`, `structExpressions`, `predicateExpressions`, `conditionalExpressions`, `bitwiseExpressions`, `temporalExpressions`, `hashExpressions`, `conversionExpressions`, `miscExpressions`. + +Spark expression classes are in scope via `import org.apache.spark.sql.catalyst.expressions._`. For Pattern C, the topic file uses **explicit** imports — extend the existing `import …expressions.{…}` line and place the new object alongside its peers. + +Helpers from `QueryPlanSerde`: + +- `scalarFunctionExprToProto(name, args*)` — return type inferred, `failOnError = false`. +- `scalarFunctionExprToProtoWithReturnType(name, returnType, failOnError, args*)` — explicit type / fail-on-error. +- `optExprWithInfo(optExpr, expr, children*)` — wrap final result; propagates "why we couldn't convert" tags. +- `withInfo(expr, "reason")` — tag a fallback when returning `None`. + +`getSupportLevel` returning `Incompatible(Some("…"))` gates behind `spark.comet.expr..allowIncompatible=true`. `Unsupported(…)` always falls back. + +### 4. Register the UDF (Pattern B only) + +In `native/core/src/execution/jni_api.rs::register_datafusion_spark_function`, add the `use` and the `register_udf` line, grouped with neighbors. Patterns A and C never touch native code. + +### 5. Add SQL file tests + +Create `spark/src/test/resources/sql-tests/expressions//$ARGUMENTS.sql`. Mandatory rules: + +1. **Test only the directly supported input types** — read `inputTypes` from the Spark source. Do **not** add columns for implicit-cast widening (`ImplicitCastInputTypes`); the cast path has its own tests. E.g. `inputTypes = Seq(IntegerType)` (`Factorial`) → one `int` column, not byte/short/long/float/double/decimal. For `TypeCollection(A, B)`, add one column per member. +2. **All `SELECT`s read from a parquet table** — no literal-only queries (the constant folder skips the read path and column-vector kernel). +3. **Mix `NULL` into every column** — exercises the validity bitmap. +4. **Per type, cover the edge-case set:** + - Float / Double: `NaN`, `+0.0`, `-0.0`, `±Infinity` via `cast('NaN' as double)` etc. + - Integer / decimal: each type's max and min (`127`/`-128`, `2147483647`/`-2147483648`, `9223372036854775807`/`-9223372036854775808`, `Decimal(38, 0)` boundary) for overflow / wrap. + - String / binary: empty, multi-byte UTF-8 (`'é'`, `'日本'`), embedded NULs if relevant. +5. **Cover the Spark-specific edges** found in step 1: in-range boundaries, just-out-of-range values, ANSI error paths, foldable-only args. + +Use `query expect_fallback(...)` for inputs where the serde returns `None`. Format: `docs/source/contributor-guide/sql-file-tests.md`. + +### 6. Update docs + +Hand-curated (`make` does not regenerate these): + +- `docs/source/user-guide/latest/expressions.md` — add `| | \`\` |` to the matching category table, alphabetical. +- `docs/source/contributor-guide/spark_expressions_support.md` — flip `- [ ] ` to `- [x] `. + +Per-expression compatibility pages under `compatibility/expressions/*.md` are auto-generated from `getCompatibleNotes` / `getIncompatibleReasons` / `getUnsupportedReasons` — do not hand-edit. + +### 7. Build, audit, finalize + +```bash +make +cd native && cargo clippy --all-targets --workspace -- -D warnings +``` + +Then run `audit-comet-expression` on `$ARGUMENTS` to compare against Spark 3.4.3 / 3.5.8 / 4.0.1 and surface coverage gaps; iterate on tests. + +The user generally runs tests themselves. If asked for a smoke test: + +```bash +./mvnw test -Dsuites="org.apache.comet.CometSqlFileTestSuite $ARGUMENTS" -Dtest=none +``` + +PR: fill in `.github/pull_request_template.md`; under "What changes are included", note the skill was used and which upstream function (`datafusion::…` built-in or `datafusion-spark::function::…`) is being wired. diff --git a/.github/actions/java-test/action.yaml b/.github/actions/java-test/action.yaml index 687703c7cf..7ec586bd10 100644 --- a/.github/actions/java-test/action.yaml +++ b/.github/actions/java-test/action.yaml @@ -29,10 +29,6 @@ inputs: description: 'Maven options passed to the mvn command' required: false default: '' - scan_impl: - description: 'The default Parquet scan implementation' - required: false - default: 'auto' upload-test-reports: description: 'Whether to upload test results including coverage to GitHub' required: false @@ -72,7 +68,6 @@ runs: shell: bash if: ${{ inputs.suites == '' }} env: - COMET_PARQUET_SCAN_IMPL: ${{ inputs.scan_impl }} SPARK_LOCAL_HOSTNAME: "localhost" SPARK_LOCAL_IP: "127.0.0.1" run: | @@ -81,7 +76,6 @@ runs: shell: bash if: ${{ inputs.suites != '' }} env: - COMET_PARQUET_SCAN_IMPL: ${{ inputs.scan_impl }} SPARK_LOCAL_HOSTNAME: "localhost" SPARK_LOCAL_IP: "127.0.0.1" run: | diff --git a/.github/actions/setup-spark-builder/action.yaml b/.github/actions/setup-spark-builder/action.yaml index 8f2882efe4..454a30fb57 100644 --- a/.github/actions/setup-spark-builder/action.yaml +++ b/.github/actions/setup-spark-builder/action.yaml @@ -51,7 +51,7 @@ runs: path: | ~/.m2/repository /root/.m2/repository - key: ${{ runner.os }}-spark-sql-${{ hashFiles('spark/**/pom.xml', 'common/**/pom.xml') }} + key: ${{ runner.os }}-spark-sql-${{ hashFiles('common/**/pom.xml', 'spark/**/pom.xml') }} restore-keys: | ${{ runner.os }}-spark-sql- diff --git a/.github/workflows/iceberg_spark_test.yml b/.github/workflows/iceberg_spark_test.yml index 2159af9ad4..712c3d5178 100644 --- a/.github/workflows/iceberg_spark_test.yml +++ b/.github/workflows/iceberg_spark_test.yml @@ -30,11 +30,9 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/**" - - "common/src/test/**" - "spark/src/main/scala/org/apache/comet/GenerateDocs.scala" - "spark-integration/**" pull_request: @@ -43,11 +41,9 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/**" - - "common/src/test/**" - "spark/src/main/scala/org/apache/comet/GenerateDocs.scala" - "spark-integration/**" # manual trigger diff --git a/.github/workflows/miri.yml b/.github/workflows/miri.yml index 438b48f15c..4efe8806a7 100644 --- a/.github/workflows/miri.yml +++ b/.github/workflows/miri.yml @@ -29,7 +29,6 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" @@ -39,7 +38,6 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index dd5377ef54..a65ded05fb 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -30,7 +30,6 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" @@ -41,7 +40,6 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" @@ -191,7 +189,7 @@ jobs: uses: ./.github/actions/setup-builder with: rust-version: ${{ env.RUST_VERSION }} - jdk-version: 17 # JDK only needed for common module proto generation + jdk-version: 17 # JDK only needed for JVM module proto generation - name: Restore Cargo cache uses: actions/cache/restore@v5 @@ -281,27 +279,22 @@ jobs: - name: "Spark 3.4, JDK 11, Scala 2.12" java_version: "11" maven_opts: "-Pspark-3.4 -Pscala-2.12" - scan_impl: "auto" - name: "Spark 3.5, JDK 17, Scala 2.13" java_version: "17" maven_opts: "-Pspark-3.5 -Pscala-2.13" - scan_impl: "native_iceberg_compat" - name: "Spark 4.0, JDK 21" java_version: "21" maven_opts: "-Pspark-4.0" - scan_impl: "auto" - name: "Spark 4.1, JDK 17" java_version: "17" maven_opts: "-Pspark-4.1" - scan_impl: "auto" - name: "Spark 4.2, JDK 17" java_version: "17" maven_opts: "-Pspark-4.2" - scan_impl: "auto" suite: - name: "fuzz" value: | @@ -309,6 +302,7 @@ jobs: org.apache.comet.CometFuzzAggregateSuite org.apache.comet.CometFuzzIcebergSuite org.apache.comet.CometFuzzMathSuite + org.apache.comet.CometCodegenFuzzSuite org.apache.comet.DataGeneratorSuite - name: "shuffle" value: | @@ -387,12 +381,15 @@ jobs: org.apache.comet.expressions.conditional.CometIfSuite org.apache.comet.expressions.conditional.CometCoalesceSuite org.apache.comet.expressions.conditional.CometCaseWhenSuite + org.apache.comet.CometCodegenSuite + org.apache.comet.CometCodegenSourceSuite + org.apache.comet.CometCodegenHOFSuite - name: "sql" value: | org.apache.spark.sql.CometToPrettyStringSuite org.apache.spark.sql.CometCollationSuite fail-fast: false - name: ${{ matrix.profile.name }}/${{ matrix.profile.scan_impl }} [${{ matrix.suite.name }}] + name: ${{ matrix.profile.name }} [${{ matrix.suite.name }}] runs-on: ubuntu-24.04 container: image: amd64/rust @@ -429,10 +426,9 @@ jobs: - name: Java test steps uses: ./.github/actions/java-test with: - artifact_name: ${{ matrix.profile.name }}-${{ matrix.suite.name }}-${{ github.run_id }}-${{ github.run_number }}-${{ github.run_attempt }}-${{ matrix.profile.scan_impl }} + artifact_name: ${{ matrix.profile.name }}-${{ matrix.suite.name }}-${{ github.run_id }}-${{ github.run_number }}-${{ github.run_attempt }} suites: ${{ matrix.suite.name == 'sql' && matrix.profile.name == 'Spark 3.4, JDK 11, Scala 2.12' && '' || matrix.suite.value }} maven_opts: ${{ matrix.profile.maven_opts }} - scan_impl: ${{ matrix.profile.scan_impl }} upload-test-reports: true skip-native-build: true diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 8abaa1c776..a83d70f380 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -30,7 +30,6 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" @@ -41,7 +40,6 @@ on: - "doc/**" - "docs/**" - "**.md" - - "dev/changelog/*.md" - "native/core/benches/**" - "native/spark-expr/benches/**" - "spark/src/test/scala/org/apache/spark/sql/benchmark/**" @@ -157,6 +155,7 @@ jobs: org.apache.comet.CometFuzzAggregateSuite org.apache.comet.CometFuzzIcebergSuite org.apache.comet.CometFuzzMathSuite + org.apache.comet.CometCodegenFuzzSuite org.apache.comet.DataGeneratorSuite - name: "shuffle" value: | @@ -234,6 +233,9 @@ jobs: org.apache.comet.expressions.conditional.CometIfSuite org.apache.comet.expressions.conditional.CometCoalesceSuite org.apache.comet.expressions.conditional.CometCaseWhenSuite + org.apache.comet.CometCodegenSuite + org.apache.comet.CometCodegenSourceSuite + org.apache.comet.CometCodegenHOFSuite - name: "sql" value: | org.apache.spark.sql.CometToPrettyStringSuite diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 51b8edc23b..b4f249d62d 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -142,15 +142,13 @@ jobs: - {name: "sql_hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} - {name: "sql_hive-2", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest"} - {name: "sql_hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} - # Since 4f5eaf0, auto mode uses native_datafusion for V1 scans, - # so we only need to test with auto. config: - - {spark-short: '3.4', spark-full: '3.4.3', java: 11, scan-impl: 'auto'} - - {spark-short: '3.5', spark-full: '3.5.8', java: 11, scan-impl: 'auto'} - - {spark-short: '4.0', spark-full: '4.0.2', java: 21, scan-impl: 'auto'} - - {spark-short: '4.1', spark-full: '4.1.1', java: 17, scan-impl: 'auto'} + - {spark-short: '3.4', spark-full: '3.4.3', java: 11} + - {spark-short: '3.5', spark-full: '3.5.8', java: 11} + - {spark-short: '4.0', spark-full: '4.0.2', java: 21} + - {spark-short: '4.1', spark-full: '4.1.1', java: 17} fail-fast: false - name: spark-sql-${{ matrix.config.scan-impl }}-${{ matrix.module.name }}/spark-${{ matrix.config.spark-full }}-jdk${{ matrix.config.java }} + name: spark-sql-${{ matrix.module.name }}/spark-${{ matrix.config.spark-full }}-jdk${{ matrix.config.java }} runs-on: ubuntu-24.04 container: image: amd64/rust @@ -176,8 +174,24 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true COMET_PARQUET_SCAN_IMPL=${{ matrix.config.scan-impl }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ - build/sbt -Dsbt.log.noformat=true -mem $SBT_MEM ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" + # SERIAL_SBT_TESTS gates SparkParallelTestGrouping in + # project/SparkBuild.scala. For Spark 4.0.2 on JDK 21 we + # leave it unset so the grouping is installed and + # DEDICATED_JVM_SBT_TESTS below actually forks a dedicated + # JVM per listed suite, working around the V1/V2 Parquet and + # Orc source-suite cross-suite file-stream leak under JDK 21 + # (issue #4327). For other rows we keep it set to reduce + # peak memory on standard 7 GB runners. + if [ "${{ matrix.config.spark-short }}" != "4.0" ] || [ "${{ matrix.config.java }}" != "21" ]; then + export SERIAL_SBT_TESTS=1 + fi + # Cap parallel forked test JVMs at 1 so that even when + # SparkParallelTestGrouping is enabled we don't blow the + # 7 GB runner budget (each forked test JVM has -Xmx2g). + NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ + build/sbt -Dsbt.log.noformat=true -mem $SBT_MEM \ + 'set Global / concurrentRestrictions := Seq(Tags.limit(Tags.ForkedTestGroup, 1))' \ + ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" if [ "${{ github.event.inputs.collect-fallback-logs }}" = "true" ]; then find . -type f -name "unit-tests.log" -print0 | xargs -0 grep -h "Comet cannot accelerate" | sed 's/.*Comet cannot accelerate/Comet cannot accelerate/' | sort -u > fallback.log fi @@ -186,9 +200,6 @@ jobs: # Standard GitHub runners have 7 GB RAM; cap SBT heap so forked test # JVMs fit alongside it. SBT_MEM: "3072" - # Disable parallel test execution to reduce peak memory usage — - # mirrors what apache/spark does on GitHub Actions. - SERIAL_SBT_TESTS: "1" # Mirror Spark's own JDK 21 / 25 CI workaround. apache/spark's # build_java21.yml and build_java25.yml set this same env var to # process-isolate the V1/V2 Parquet and Orc source suites because @@ -201,7 +212,7 @@ jobs: if: ${{ github.event.inputs.collect-fallback-logs == 'true' }} uses: actions/upload-artifact@v7 with: - name: fallback-log-spark-sql-${{ matrix.config.scan-impl }}-${{ matrix.module.name }}-spark-${{ matrix.config.spark-full }}-jdk${{ matrix.config.java }} + name: fallback-log-spark-sql-${{ matrix.module.name }}-spark-${{ matrix.config.spark-full }}-jdk${{ matrix.config.java }} path: "**/fallback.log" merge-fallback-logs: diff --git a/.github/workflows/spark_sql_test_native_iceberg_compat.yml b/.github/workflows/spark_sql_test_native_iceberg_compat.yml deleted file mode 100644 index af84e8556d..0000000000 --- a/.github/workflows/spark_sql_test_native_iceberg_compat.yml +++ /dev/null @@ -1,72 +0,0 @@ -# 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. - -name: Spark SQL Tests (native_iceberg_compat) - -concurrency: - group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} - cancel-in-progress: true - -on: - # manual trigger - # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow - workflow_dispatch: - -env: - RUST_VERSION: stable - RUST_BACKTRACE: 1 - -jobs: - spark-sql-catalyst-native-iceberg-compat: - strategy: - matrix: - os: [ubuntu-24.04] - java-version: [11] - spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.8'}] - module: - - {name: "catalyst", args1: "catalyst/test", args2: ""} - - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} - - {name: "sql/core-2", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.ExtendedSQLTest"} - - {name: "sql/core-3", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.SlowSQLTest"} - - {name: "sql/hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} - - {name: "sql/hive-2", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest"} - - {name: "sql/hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} - fail-fast: false - name: spark-sql-native-iceberg-compat-${{ matrix.module.name }}/${{ matrix.os }}/spark-${{ matrix.spark-version.full }}/java-${{ matrix.java-version }} - runs-on: ${{ matrix.os }} - container: - image: amd64/rust - steps: - - uses: actions/checkout@v6 - - name: Setup Rust & Java toolchain - uses: ./.github/actions/setup-builder - with: - rust-version: ${{env.RUST_VERSION}} - jdk-version: ${{ matrix.java-version }} - - name: Setup Spark - uses: ./.github/actions/setup-spark-builder - with: - spark-version: ${{ matrix.spark-version.full }} - spark-short-version: ${{ matrix.spark-version.short }} - - name: Run Spark tests - run: | - cd apache-spark - rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - ENABLE_COMET=true ENABLE_COMET_ONHEAP=true COMET_PARQUET_SCAN_IMPL=native_iceberg_compat build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" - env: - LC_ALL: "C.UTF-8" - diff --git a/CHANGELOG.md b/CHANGELOG.md index c9959d5f20..76183dbf12 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -19,4 +19,4 @@ under the License. # Apache DataFusion Comet Changelog -Comprehensive changelogs for each release are available [here](dev/changelog). +Comprehensive changelogs for each release are available [here](docs/source/changelog). diff --git a/Makefile b/Makefile index 60ea51bf0c..8685d171be 100644 --- a/Makefile +++ b/Makefile @@ -68,28 +68,28 @@ endif core-amd64: rustup target add x86_64-apple-darwin cd native && RUSTFLAGS="-Ctarget-cpu=skylake" CC=o64-clang CXX=o64-clang++ cargo build --target x86_64-apple-darwin --release $(FEATURES_ARG) - mkdir -p common/target/classes/org/apache/comet/darwin/x86_64 - cp native/target/x86_64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/x86_64 + mkdir -p spark/target/classes/org/apache/comet/darwin/x86_64 + cp native/target/x86_64-apple-darwin/release/libcomet.dylib spark/target/classes/org/apache/comet/darwin/x86_64 cd native && RUSTFLAGS="-Ctarget-cpu=x86-64-v3" cargo build --release $(FEATURES_ARG) - mkdir -p common/target/classes/org/apache/comet/linux/amd64 - cp native/target/release/libcomet.so common/target/classes/org/apache/comet/linux/amd64 - jar -cf common/target/comet-native-x86_64.jar \ - -C common/target/classes/org/apache/comet darwin \ - -C common/target/classes/org/apache/comet linux - ./dev/deploy-file common/target/comet-native-x86_64.jar comet-native-x86_64${COMET_CLASSIFIER} jar + mkdir -p spark/target/classes/org/apache/comet/linux/amd64 + cp native/target/release/libcomet.so spark/target/classes/org/apache/comet/linux/amd64 + jar -cf spark/target/comet-native-x86_64.jar \ + -C spark/target/classes/org/apache/comet darwin \ + -C spark/target/classes/org/apache/comet linux + ./dev/deploy-file spark/target/comet-native-x86_64.jar comet-native-x86_64${COMET_CLASSIFIER} jar core-arm64: rustup target add aarch64-apple-darwin cd native && RUSTFLAGS="-Ctarget-cpu=apple-m1" CC=arm64-apple-darwin21.4-clang CXX=arm64-apple-darwin21.4-clang++ CARGO_FEATURE_NEON=1 cargo build --target aarch64-apple-darwin --release $(FEATURES_ARG) - mkdir -p common/target/classes/org/apache/comet/darwin/aarch64 - cp native/target/aarch64-apple-darwin/release/libcomet.dylib common/target/classes/org/apache/comet/darwin/aarch64 + mkdir -p spark/target/classes/org/apache/comet/darwin/aarch64 + cp native/target/aarch64-apple-darwin/release/libcomet.dylib spark/target/classes/org/apache/comet/darwin/aarch64 cd native && RUSTFLAGS="-Ctarget-cpu=neoverse-n1" cargo build --release $(FEATURES_ARG) - mkdir -p common/target/classes/org/apache/comet/linux/aarch64 - cp native/target/release/libcomet.so common/target/classes/org/apache/comet/linux/aarch64 - jar -cf common/target/comet-native-aarch64.jar \ - -C common/target/classes/org/apache/comet darwin \ - -C common/target/classes/org/apache/comet linux - ./dev/deploy-file common/target/comet-native-aarch64.jar comet-native-aarch64${COMET_CLASSIFIER} jar + mkdir -p spark/target/classes/org/apache/comet/linux/aarch64 + cp native/target/release/libcomet.so spark/target/classes/org/apache/comet/linux/aarch64 + jar -cf spark/target/comet-native-aarch64.jar \ + -C spark/target/classes/org/apache/comet darwin \ + -C spark/target/classes/org/apache/comet linux + ./dev/deploy-file spark/target/comet-native-aarch64.jar comet-native-aarch64${COMET_CLASSIFIER} jar release-linux: clean rustup target add aarch64-apple-darwin x86_64-apple-darwin diff --git a/benchmarks/tpc/engines/comet-hashjoin.toml b/benchmarks/tpc/engines/comet-hashjoin.toml index 1aa4957241..202dcad914 100644 --- a/benchmarks/tpc/engines/comet-hashjoin.toml +++ b/benchmarks/tpc/engines/comet-hashjoin.toml @@ -30,6 +30,5 @@ driver_class_path = ["$COMET_JAR"] "spark.executor.extraClassPath" = "$COMET_JAR" "spark.plugins" = "org.apache.spark.CometPlugin" "spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" -"spark.comet.scan.impl" = "native_datafusion" "spark.comet.exec.replaceSortMergeJoin" = "true" "spark.comet.expression.Cast.allowIncompatible" = "true" diff --git a/benchmarks/tpc/engines/comet.toml b/benchmarks/tpc/engines/comet.toml index 05b2cb22ba..30ffa20d22 100644 --- a/benchmarks/tpc/engines/comet.toml +++ b/benchmarks/tpc/engines/comet.toml @@ -30,5 +30,4 @@ driver_class_path = ["$COMET_JAR"] "spark.executor.extraClassPath" = "$COMET_JAR" "spark.plugins" = "org.apache.spark.CometPlugin" "spark.shuffle.manager" = "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager" -"spark.comet.scan.impl" = "native_datafusion" "spark.comet.expression.Cast.allowIncompatible" = "true" diff --git a/common/pom.xml b/common/pom.xml index 4556f6a334..43c5033dd1 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -33,196 +33,38 @@ under the License. comet-common-spark${spark.version.short}_${scala.binary.version} comet-common + + false - - - org.apache.spark - spark-sql_${scala.binary.version} - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - org.apache.parquet - parquet-format-structures - - - org.apache.arrow - arrow-vector - - - org.apache.arrow - arrow-memory-unsafe - - - org.apache.arrow - arrow-c-data - - - org.scala-lang.modules - scala-collection-compat_${scala.binary.version} - - - junit - junit - test - - - org.assertj - assertj-core - test - - - - - io.github.git-commit-id - git-commit-id-maven-plugin - ${git-commit-id-maven-plugin.version} - - - get-the-git-infos - - revision - - initialize - - - - true - ${project.build.outputDirectory}/comet-git-info.properties - full - - ^git.branch$ - ^git.build.*$ - ^git.commit.id.(abbrev|full)$ - ^git.remote.*$ - - - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - true - true - false - true - - - - org.apache.arrow:* - - - - - *:* - - **/*.thrift - git.properties - log4j.properties - log4j2.properties - arrow-git.properties - - - - org.apache.arrow:arrow-vector - - - codegen/** - - - - - - org.apache.arrow - ${comet.shade.packageName}.arrow - - - org/apache/arrow/c/jni/JniWrapper - org/apache/arrow/c/jni/PrivateData - org/apache/arrow/c/jni/CDataJniException - - org/apache/arrow/c/ArrayStreamExporter$ExportedArrayStreamPrivateData - - - - - - - - + net.alchim31.maven scala-maven-plugin + - org.codehaus.mojo - build-helper-maven-plugin - - - add-shim-source - generate-sources - - add-source - - - - src/main/${shims.majorVerSrc} - src/main/${shims.minorVerSrc} - - - - + io.github.evis + scalafix-maven-plugin_${scala.binary.version} + ${scalafix-maven-plugin.version} + + true + - - - ${project.basedir}/src/main/resources - - - ${project.basedir}/../native/target/x86_64-apple-darwin/release - - libcomet.dylib - - org/apache/comet/darwin/x86_64 - - - ${project.basedir}/../native/target/aarch64-apple-darwin/release - - libcomet.dylib - - org/apache/comet/darwin/aarch64 - - - ${jni.dir} - - libcomet.dylib - libcomet.so - comet.dll - - org/apache/comet/${platform}/${arch} - - diff --git a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java b/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java deleted file mode 100644 index 49ce92a408..0000000000 --- a/common/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.arrow.c; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.types.pojo.Field; - -import org.apache.comet.IcebergApi; - -/** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */ -public abstract class AbstractCometSchemaImporter { - private final BufferAllocator allocator; - private final SchemaImporter importer; - private final CDataDictionaryProvider provider = new CDataDictionaryProvider(); - - public AbstractCometSchemaImporter(BufferAllocator allocator) { - this.allocator = allocator; - this.importer = new SchemaImporter(allocator); - } - - public BufferAllocator getAllocator() { - return allocator; - } - - public CDataDictionaryProvider getProvider() { - return provider; - } - - public Field importField(ArrowSchema schema) { - try { - return importer.importField(schema, provider); - } finally { - schema.release(); - schema.close(); - } - } - - /** - * Imports data from ArrowArray/ArrowSchema into a FieldVector. This is basically the same as Java - * Arrow `Data.importVector`. `Data.importVector` initiates `SchemaImporter` internally which is - * used to fill dictionary ids for dictionary encoded vectors. Every call to `importVector` will - * begin with dictionary ids starting from 0. So, separate calls to `importVector` will overwrite - * dictionary ids. To avoid this, we need to use the same `SchemaImporter` instance for all calls - * to `importVector`. - */ - public FieldVector importVector(ArrowArray array, ArrowSchema schema) { - Field field = importField(schema); - FieldVector vector = field.createVector(allocator); - Data.importIntoVector(allocator, array, vector, provider); - - return vector; - } - - @IcebergApi - public void close() { - provider.close(); - } -} diff --git a/common/src/main/java/org/apache/comet/IcebergApi.java b/common/src/main/java/org/apache/comet/IcebergApi.java deleted file mode 100644 index 915fd87848..0000000000 --- a/common/src/main/java/org/apache/comet/IcebergApi.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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; - -import java.lang.annotation.Documented; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -/** - * Indicates that the annotated element is part of the public API used by Apache Iceberg. - * - *

This annotation marks classes, methods, constructors, and fields that form the contract - * between Comet and Iceberg. Changes to these APIs may break Iceberg's Comet integration, so - * contributors should exercise caution and consider backward compatibility when modifying annotated - * elements. - * - *

The Iceberg integration uses Comet's native Parquet reader for accelerated vectorized reads. - * See the contributor guide documentation for details on how Iceberg uses these APIs. - * - * @see Apache Iceberg - */ -@Documented -@Retention(RetentionPolicy.RUNTIME) -@Target({ElementType.TYPE, ElementType.METHOD, ElementType.CONSTRUCTOR, ElementType.FIELD}) -public @interface IcebergApi {} diff --git a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java deleted file mode 100644 index f8385f41b6..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.parquet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.Type; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.TimestampNTZType$; - -import org.apache.comet.CometConf; -import org.apache.comet.IcebergApi; -import org.apache.comet.vector.CometVector; - -/** Base class for Comet Parquet column reader implementations. */ -@IcebergApi -public abstract class AbstractColumnReader implements AutoCloseable { - protected static final Logger LOG = LoggerFactory.getLogger(AbstractColumnReader.class); - - /** The Spark data type. */ - protected final DataType type; - - /** The Spark data type. */ - protected final Type fieldType; - - /** Parquet column descriptor. */ - protected final ColumnDescriptor descriptor; - - /** - * Whether to always return 128 bit decimals, regardless of its precision. If false, this will - * return 32, 64 or 128 bit decimals depending on the precision. - */ - protected final boolean useDecimal128; - - /** - * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) - * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them - * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when - * seeing these dates/timestamps. - */ - protected final boolean useLegacyDateTimestamp; - - /** The size of one batch, gets updated by 'readBatch' */ - protected int batchSize; - - /** A pointer to the native implementation of ColumnReader. */ - @IcebergApi protected long nativeHandle; - - AbstractColumnReader( - DataType type, - Type fieldType, - ColumnDescriptor descriptor, - boolean useDecimal128, - boolean useLegacyDateTimestamp) { - this.type = type; - this.fieldType = fieldType; - this.descriptor = descriptor; - this.useDecimal128 = useDecimal128; - this.useLegacyDateTimestamp = useLegacyDateTimestamp; - } - - AbstractColumnReader( - DataType type, - ColumnDescriptor descriptor, - boolean useDecimal128, - boolean useLegacyDateTimestamp) { - this(type, null, descriptor, useDecimal128, useLegacyDateTimestamp); - TypeUtil.checkParquetType(descriptor, type); - } - - ColumnDescriptor getDescriptor() { - return descriptor; - } - - String getPath() { - return String.join(".", this.descriptor.getPath()); - } - - /** - * Set the batch size of this reader to be 'batchSize'. Also initializes the native column reader. - */ - @IcebergApi - public void setBatchSize(int batchSize) { - assert nativeHandle == 0 - : "Native column reader shouldn't be initialized before " + "'setBatchSize' is called"; - this.batchSize = batchSize; - initNative(); - } - - /** - * Reads a batch of 'total' new rows. - * - * @param total the total number of rows to read - */ - public abstract void readBatch(int total); - - /** Returns the {@link CometVector} read by this reader. */ - public abstract CometVector currentBatch(); - - @IcebergApi - @Override - public void close() { - if (nativeHandle != 0) { - LOG.debug("Closing the column reader"); - Native.closeColumnReader(nativeHandle); - nativeHandle = 0; - } - } - - protected void initNative() { - LOG.debug("initializing the native column reader"); - DataType readType = (boolean) CometConf.COMET_SCHEMA_EVOLUTION_ENABLED().get() ? type : null; - boolean useLegacyDateTimestampOrNTZ = - useLegacyDateTimestamp || type == TimestampNTZType$.MODULE$; - nativeHandle = - Utils.initColumnReader( - descriptor, readType, batchSize, useDecimal128, useLegacyDateTimestampOrNTZ); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java deleted file mode 100644 index 521eb4aa5b..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java +++ /dev/null @@ -1,255 +0,0 @@ -/* - * 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.parquet; - -import java.math.BigDecimal; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.vector.*; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns; -import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.types.UTF8String; - -import org.apache.comet.vector.CometPlainVector; -import org.apache.comet.vector.CometVector; - -/** - * A column reader that returns constant vectors using Arrow Java vectors directly (no native - * mutable buffers). Used for partition columns and missing columns in the native_iceberg_compat - * scan path. - * - *

The vector is filled with the constant value repeated for every row in the batch. This is - * necessary because the underlying Arrow vector's buffers must be large enough to match the - * reported value count — otherwise variable-width types (strings, binary) would have undersized - * offset buffers, causing out-of-bounds reads on the native side. - */ -public class ArrowConstantColumnReader extends AbstractColumnReader { - private final BufferAllocator allocator = new RootAllocator(); - - private boolean isNull; - private Object value; - private FieldVector fieldVector; - private CometPlainVector vector; - private int currentSize; - - /** Constructor for missing columns (default values from schema). */ - ArrowConstantColumnReader(StructField field, int batchSize, boolean useDecimal128) { - super(field.dataType(), TypeUtil.convertToParquet(field), useDecimal128, false); - this.batchSize = batchSize; - this.value = - ResolveDefaultColumns.getExistenceDefaultValues(new StructType(new StructField[] {field}))[ - 0]; - initVector(value, batchSize); - } - - /** Constructor for partition columns with values from a row. */ - ArrowConstantColumnReader( - StructField field, int batchSize, InternalRow values, int index, boolean useDecimal128) { - super(field.dataType(), TypeUtil.convertToParquet(field), useDecimal128, false); - this.batchSize = batchSize; - Object v = values.get(index, field.dataType()); - this.value = v; - initVector(v, batchSize); - } - - @Override - public void setBatchSize(int batchSize) { - close(); - this.batchSize = batchSize; - initVector(value, batchSize); - } - - @Override - public void readBatch(int total) { - if (total != currentSize) { - close(); - initVector(value, total); - } - } - - @Override - public CometVector currentBatch() { - return vector; - } - - @Override - public void close() { - if (vector != null) { - vector.close(); - vector = null; - } - if (fieldVector != null) { - fieldVector.close(); - fieldVector = null; - } - } - - private void initVector(Object value, int count) { - currentSize = count; - if (value == null) { - isNull = true; - fieldVector = createNullVector(count); - } else { - isNull = false; - fieldVector = createFilledVector(value, count); - } - vector = new CometPlainVector(fieldVector, useDecimal128, false, true); - } - - /** Creates a vector of the correct type with {@code count} null values. */ - private FieldVector createNullVector(int count) { - String name = "constant"; - FieldVector v; - if (type == DataTypes.BooleanType) { - v = new BitVector(name, allocator); - } else if (type == DataTypes.ByteType) { - v = new TinyIntVector(name, allocator); - } else if (type == DataTypes.ShortType) { - v = new SmallIntVector(name, allocator); - } else if (type == DataTypes.IntegerType || type == DataTypes.DateType) { - v = new IntVector(name, allocator); - } else if (type == DataTypes.LongType - || type == DataTypes.TimestampType - || type == TimestampNTZType$.MODULE$) { - v = new BigIntVector(name, allocator); - } else if (type == DataTypes.FloatType) { - v = new Float4Vector(name, allocator); - } else if (type == DataTypes.DoubleType) { - v = new Float8Vector(name, allocator); - } else if (type == DataTypes.BinaryType) { - v = new VarBinaryVector(name, allocator); - } else if (type == DataTypes.StringType) { - v = new VarCharVector(name, allocator); - } else if (type instanceof DecimalType) { - DecimalType dt = (DecimalType) type; - if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { - v = new IntVector(name, allocator); - } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { - v = new BigIntVector(name, allocator); - } else { - v = new DecimalVector(name, allocator, dt.precision(), dt.scale()); - } - } else { - throw new UnsupportedOperationException("Unsupported Spark type: " + type); - } - v.setValueCount(count); - return v; - } - - /** Creates a vector filled with {@code count} copies of the given value. */ - private FieldVector createFilledVector(Object value, int count) { - String name = "constant"; - if (type == DataTypes.BooleanType) { - BitVector v = new BitVector(name, allocator); - v.allocateNew(count); - int bit = (boolean) value ? 1 : 0; - for (int i = 0; i < count; i++) v.setSafe(i, bit); - v.setValueCount(count); - return v; - } else if (type == DataTypes.ByteType) { - TinyIntVector v = new TinyIntVector(name, allocator); - v.allocateNew(count); - byte val = (byte) value; - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (type == DataTypes.ShortType) { - SmallIntVector v = new SmallIntVector(name, allocator); - v.allocateNew(count); - short val = (short) value; - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (type == DataTypes.IntegerType || type == DataTypes.DateType) { - IntVector v = new IntVector(name, allocator); - v.allocateNew(count); - int val = (int) value; - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (type == DataTypes.LongType - || type == DataTypes.TimestampType - || type == TimestampNTZType$.MODULE$) { - BigIntVector v = new BigIntVector(name, allocator); - v.allocateNew(count); - long val = (long) value; - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (type == DataTypes.FloatType) { - Float4Vector v = new Float4Vector(name, allocator); - v.allocateNew(count); - float val = (float) value; - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (type == DataTypes.DoubleType) { - Float8Vector v = new Float8Vector(name, allocator); - v.allocateNew(count); - double val = (double) value; - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (type == DataTypes.BinaryType) { - VarBinaryVector v = new VarBinaryVector(name, allocator); - v.allocateNew(count); - byte[] bytes = (byte[]) value; - for (int i = 0; i < count; i++) v.setSafe(i, bytes, 0, bytes.length); - v.setValueCount(count); - return v; - } else if (type == DataTypes.StringType) { - VarCharVector v = new VarCharVector(name, allocator); - v.allocateNew(count); - byte[] bytes = ((UTF8String) value).getBytes(); - for (int i = 0; i < count; i++) v.setSafe(i, bytes, 0, bytes.length); - v.setValueCount(count); - return v; - } else if (type instanceof DecimalType) { - DecimalType dt = (DecimalType) type; - Decimal d = (Decimal) value; - if (!useDecimal128 && dt.precision() <= Decimal.MAX_INT_DIGITS()) { - IntVector v = new IntVector(name, allocator); - v.allocateNew(count); - int val = (int) d.toUnscaledLong(); - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else if (!useDecimal128 && dt.precision() <= Decimal.MAX_LONG_DIGITS()) { - BigIntVector v = new BigIntVector(name, allocator); - v.allocateNew(count); - long val = d.toUnscaledLong(); - for (int i = 0; i < count; i++) v.setSafe(i, val); - v.setValueCount(count); - return v; - } else { - DecimalVector v = new DecimalVector(name, allocator, dt.precision(), dt.scale()); - v.allocateNew(count); - BigDecimal bd = d.toJavaBigDecimal(); - for (int i = 0; i < count; i++) v.setSafe(i, bd); - v.setValueCount(count); - return v; - } - } else { - throw new UnsupportedOperationException("Unsupported Spark type: " + type); - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java deleted file mode 100644 index 7d17e551df..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * 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.parquet; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.vector.BigIntVector; -import org.apache.spark.sql.types.*; - -import org.apache.comet.vector.CometPlainVector; -import org.apache.comet.vector.CometVector; - -/** - * A column reader that computes row indices in Java and creates Arrow BigIntVectors directly (no - * native mutable buffers). Used for the row index metadata column in the native_iceberg_compat scan - * path. - * - *

The {@code indices} array contains alternating pairs of (start_index, count) representing - * ranges of sequential row indices within each row group. - */ -public class ArrowRowIndexColumnReader extends AbstractColumnReader { - private final BufferAllocator allocator = new RootAllocator(); - - /** Alternating (start_index, count) pairs from row groups. */ - private final long[] indices; - - /** Number of row indices consumed so far across batches. */ - private long offset; - - private BigIntVector fieldVector; - private CometPlainVector vector; - - public ArrowRowIndexColumnReader(StructField field, int batchSize, long[] indices) { - super(field.dataType(), TypeUtil.convertToParquet(field), false, false); - this.indices = indices; - this.batchSize = batchSize; - } - - @Override - public void setBatchSize(int batchSize) { - close(); - this.batchSize = batchSize; - } - - @Override - public void readBatch(int total) { - close(); - - fieldVector = new BigIntVector("row_index", allocator); - fieldVector.allocateNew(total); - - // Port of Rust set_indices: iterate (start, count) pairs, skip offset rows, fill up to total. - long skipped = 0; - int filled = 0; - for (int i = 0; i < indices.length && filled < total; i += 2) { - long index = indices[i]; - long count = indices[i + 1]; - long skip = Math.min(count, offset - skipped); - skipped += skip; - if (count == skip) { - continue; - } - long remaining = Math.min(count - skip, total - filled); - for (long j = 0; j < remaining; j++) { - fieldVector.setSafe(filled, index + skip + j); - filled++; - } - } - offset += filled; - - fieldVector.setValueCount(filled); - vector = new CometPlainVector(fieldVector, false, false, false); - vector.setNumValues(filled); - } - - @Override - public CometVector currentBatch() { - return vector; - } - - @Override - public void close() { - if (vector != null) { - vector.close(); - vector = null; - } - if (fieldVector != null) { - fieldVector.close(); - fieldVector = null; - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java b/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java deleted file mode 100644 index a23216c7f5..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/BloomFilterReader.java +++ /dev/null @@ -1,253 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; -import org.apache.parquet.column.values.bloomfilter.BloomFilter; -import org.apache.parquet.crypto.AesCipher; -import org.apache.parquet.crypto.InternalColumnDecryptionSetup; -import org.apache.parquet.crypto.InternalFileDecryptor; -import org.apache.parquet.crypto.ModuleCipherFactory; -import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.apache.parquet.filter2.predicate.FilterPredicate; -import org.apache.parquet.filter2.predicate.Operators; -import org.apache.parquet.filter2.predicate.UserDefinedPredicate; -import org.apache.parquet.format.BlockCipher; -import org.apache.parquet.format.BloomFilterHeader; -import org.apache.parquet.format.Util; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.parquet.io.SeekableInputStream; - -public class BloomFilterReader implements FilterPredicate.Visitor { - private static final Logger LOG = LoggerFactory.getLogger(BloomFilterReader.class); - private static final boolean BLOCK_MIGHT_MATCH = false; - private static final boolean BLOCK_CANNOT_MATCH = true; - - private final Map columns; - private final Map cache = new HashMap<>(); - private final InternalFileDecryptor fileDecryptor; - private final SeekableInputStream inputStream; - - BloomFilterReader( - BlockMetaData block, InternalFileDecryptor fileDecryptor, SeekableInputStream inputStream) { - this.columns = new HashMap<>(); - for (ColumnChunkMetaData column : block.getColumns()) { - columns.put(column.getPath(), column); - } - this.fileDecryptor = fileDecryptor; - this.inputStream = inputStream; - } - - @Override - public > Boolean visit(Operators.Eq eq) { - T value = eq.getValue(); - - if (value == null) { - // the bloom filter bitset contains only non-null values so isn't helpful. this - // could check the column stats, but the StatisticsFilter is responsible - return BLOCK_MIGHT_MATCH; - } - - Operators.Column filterColumn = eq.getColumn(); - ColumnChunkMetaData meta = columns.get(filterColumn.getColumnPath()); - if (meta == null) { - // the column isn't in this file so all values are null, but the value - // must be non-null because of the above check. - return BLOCK_CANNOT_MATCH; - } - - try { - BloomFilter bloomFilter = readBloomFilter(meta); - if (bloomFilter != null && !bloomFilter.findHash(bloomFilter.hash(value))) { - return BLOCK_CANNOT_MATCH; - } - } catch (RuntimeException e) { - LOG.warn(e.getMessage()); - return BLOCK_MIGHT_MATCH; - } - - return BLOCK_MIGHT_MATCH; - } - - @Override - public > Boolean visit(Operators.NotEq notEq) { - return BLOCK_MIGHT_MATCH; - } - - @Override - public > Boolean visit(Operators.Lt lt) { - return BLOCK_MIGHT_MATCH; - } - - @Override - public > Boolean visit(Operators.LtEq ltEq) { - return BLOCK_MIGHT_MATCH; - } - - @Override - public > Boolean visit(Operators.Gt gt) { - return BLOCK_MIGHT_MATCH; - } - - @Override - public > Boolean visit(Operators.GtEq gtEq) { - return BLOCK_MIGHT_MATCH; - } - - @Override - public Boolean visit(Operators.And and) { - return and.getLeft().accept(this) || and.getRight().accept(this); - } - - @Override - public Boolean visit(Operators.Or or) { - return or.getLeft().accept(this) && or.getRight().accept(this); - } - - @Override - public Boolean visit(Operators.Not not) { - throw new IllegalArgumentException( - "This predicate " - + not - + " contains a not! Did you forget" - + " to run this predicate through LogicalInverseRewriter?"); - } - - @Override - public , U extends UserDefinedPredicate> Boolean visit( - Operators.UserDefined udp) { - return visit(udp, false); - } - - @Override - public , U extends UserDefinedPredicate> Boolean visit( - Operators.LogicalNotUserDefined udp) { - return visit(udp.getUserDefined(), true); - } - - private , U extends UserDefinedPredicate> Boolean visit( - Operators.UserDefined ud, boolean inverted) { - return BLOCK_MIGHT_MATCH; - } - - BloomFilter readBloomFilter(ColumnChunkMetaData meta) { - if (cache.containsKey(meta.getPath())) { - return cache.get(meta.getPath()); - } - try { - if (!cache.containsKey(meta.getPath())) { - BloomFilter bloomFilter = readBloomFilterInternal(meta); - if (bloomFilter == null) { - return null; - } - - cache.put(meta.getPath(), bloomFilter); - } - return cache.get(meta.getPath()); - } catch (IOException e) { - LOG.error("Failed to read Bloom filter data", e); - } - - return null; - } - - private BloomFilter readBloomFilterInternal(ColumnChunkMetaData meta) throws IOException { - long bloomFilterOffset = meta.getBloomFilterOffset(); - if (bloomFilterOffset < 0) { - return null; - } - - // Prepare to decrypt Bloom filter (for encrypted columns) - BlockCipher.Decryptor bloomFilterDecryptor = null; - byte[] bloomFilterHeaderAAD = null; - byte[] bloomFilterBitsetAAD = null; - if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { - InternalColumnDecryptionSetup columnDecryptionSetup = - fileDecryptor.getColumnSetup(meta.getPath()); - if (columnDecryptionSetup.isEncrypted()) { - bloomFilterDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); - bloomFilterHeaderAAD = - AesCipher.createModuleAAD( - fileDecryptor.getFileAAD(), - ModuleCipherFactory.ModuleType.BloomFilterHeader, - meta.getRowGroupOrdinal(), - columnDecryptionSetup.getOrdinal(), - -1); - bloomFilterBitsetAAD = - AesCipher.createModuleAAD( - fileDecryptor.getFileAAD(), - ModuleCipherFactory.ModuleType.BloomFilterBitset, - meta.getRowGroupOrdinal(), - columnDecryptionSetup.getOrdinal(), - -1); - } - } - - // Read Bloom filter data header. - inputStream.seek(bloomFilterOffset); - BloomFilterHeader bloomFilterHeader; - try { - bloomFilterHeader = - Util.readBloomFilterHeader(inputStream, bloomFilterDecryptor, bloomFilterHeaderAAD); - } catch (IOException e) { - LOG.warn("read no bloom filter"); - return null; - } - - int numBytes = bloomFilterHeader.getNumBytes(); - if (numBytes <= 0 || numBytes > BlockSplitBloomFilter.UPPER_BOUND_BYTES) { - LOG.warn("the read bloom filter size is wrong, size is {}", bloomFilterHeader.getNumBytes()); - return null; - } - - if (!bloomFilterHeader.getHash().isSetXXHASH() - || !bloomFilterHeader.getAlgorithm().isSetBLOCK() - || !bloomFilterHeader.getCompression().isSetUNCOMPRESSED()) { - LOG.warn( - "the read bloom filter is not supported yet, algorithm = {}, hash = {}, " - + "compression = {}", - bloomFilterHeader.getAlgorithm(), - bloomFilterHeader.getHash(), - bloomFilterHeader.getCompression()); - return null; - } - - byte[] bitset; - if (null == bloomFilterDecryptor) { - bitset = new byte[numBytes]; - inputStream.readFully(bitset); - } else { - bitset = bloomFilterDecryptor.decrypt(inputStream, bloomFilterBitsetAAD); - if (bitset.length != numBytes) { - throw new ParquetCryptoRuntimeException("Wrong length of decrypted bloom filter bitset"); - } - } - return new BlockSplitBloomFilter(bitset); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java deleted file mode 100644 index 4e4f6ba0db..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java +++ /dev/null @@ -1,230 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.parquet.crypto.AesCipher; -import org.apache.parquet.crypto.InternalColumnDecryptionSetup; -import org.apache.parquet.crypto.InternalFileDecryptor; -import org.apache.parquet.crypto.ModuleCipherFactory; -import org.apache.parquet.format.BlockCipher; -import org.apache.parquet.format.Util; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.parquet.internal.column.columnindex.ColumnIndex; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; -import org.apache.parquet.internal.hadoop.metadata.IndexReference; -import org.apache.parquet.io.SeekableInputStream; - -class ColumnIndexReader implements ColumnIndexStore { - private static final Logger LOG = LoggerFactory.getLogger(ColumnIndexReader.class); - - // Used for columns are not in this parquet file - private static final IndexStore MISSING_INDEX_STORE = - new IndexStore() { - @Override - public ColumnIndex getColumnIndex() { - return null; - } - - @Override - public OffsetIndex getOffsetIndex() { - return null; - } - }; - - private static final ColumnIndexReader EMPTY = - new ColumnIndexReader(new BlockMetaData(), Collections.emptySet(), null, null) { - @Override - public ColumnIndex getColumnIndex(ColumnPath column) { - return null; - } - - @Override - public OffsetIndex getOffsetIndex(ColumnPath column) { - throw new MissingOffsetIndexException(column); - } - }; - - private final InternalFileDecryptor fileDecryptor; - private final SeekableInputStream inputStream; - private final Map store; - - /** - * Creates a column index store which lazily reads column/offset indexes for the columns in paths. - * Paths are the set of columns used for the projection. - */ - static ColumnIndexReader create( - BlockMetaData block, - Set paths, - InternalFileDecryptor fileDecryptor, - SeekableInputStream inputStream) { - try { - return new ColumnIndexReader(block, paths, fileDecryptor, inputStream); - } catch (MissingOffsetIndexException e) { - return EMPTY; - } - } - - private ColumnIndexReader( - BlockMetaData block, - Set paths, - InternalFileDecryptor fileDecryptor, - SeekableInputStream inputStream) { - this.fileDecryptor = fileDecryptor; - this.inputStream = inputStream; - Map store = new HashMap<>(); - for (ColumnChunkMetaData column : block.getColumns()) { - ColumnPath path = column.getPath(); - if (paths.contains(path)) { - store.put(path, new IndexStoreImpl(column)); - } - } - this.store = store; - } - - @Override - public ColumnIndex getColumnIndex(ColumnPath column) { - return store.getOrDefault(column, MISSING_INDEX_STORE).getColumnIndex(); - } - - @Override - public OffsetIndex getOffsetIndex(ColumnPath column) { - return store.getOrDefault(column, MISSING_INDEX_STORE).getOffsetIndex(); - } - - private interface IndexStore { - ColumnIndex getColumnIndex(); - - OffsetIndex getOffsetIndex(); - } - - private class IndexStoreImpl implements IndexStore { - private final ColumnChunkMetaData meta; - private ColumnIndex columnIndex; - private boolean columnIndexRead; - private final OffsetIndex offsetIndex; - - IndexStoreImpl(ColumnChunkMetaData meta) { - this.meta = meta; - OffsetIndex oi; - try { - oi = readOffsetIndex(meta); - } catch (IOException e) { - // If the I/O issue still stands it will fail the reading later; - // otherwise we fail the filtering only with a missing offset index. - LOG.warn("Unable to read offset index for column {}", meta.getPath(), e); - oi = null; - } - if (oi == null) { - throw new MissingOffsetIndexException(meta.getPath()); - } - offsetIndex = oi; - } - - @Override - public ColumnIndex getColumnIndex() { - if (!columnIndexRead) { - try { - columnIndex = readColumnIndex(meta); - } catch (IOException e) { - // If the I/O issue still stands it will fail the reading later; - // otherwise we fail the filtering only with a missing column index. - LOG.warn("Unable to read column index for column {}", meta.getPath(), e); - } - columnIndexRead = true; - } - return columnIndex; - } - - @Override - public OffsetIndex getOffsetIndex() { - return offsetIndex; - } - } - - // Visible for testing - ColumnIndex readColumnIndex(ColumnChunkMetaData column) throws IOException { - IndexReference ref = column.getColumnIndexReference(); - if (ref == null) { - return null; - } - inputStream.seek(ref.getOffset()); - - BlockCipher.Decryptor columnIndexDecryptor = null; - byte[] columnIndexAAD = null; - if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { - InternalColumnDecryptionSetup columnDecryptionSetup = - fileDecryptor.getColumnSetup(column.getPath()); - if (columnDecryptionSetup.isEncrypted()) { - columnIndexDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); - columnIndexAAD = - AesCipher.createModuleAAD( - fileDecryptor.getFileAAD(), - ModuleCipherFactory.ModuleType.ColumnIndex, - column.getRowGroupOrdinal(), - columnDecryptionSetup.getOrdinal(), - -1); - } - } - return ParquetMetadataConverter.fromParquetColumnIndex( - column.getPrimitiveType(), - Util.readColumnIndex(inputStream, columnIndexDecryptor, columnIndexAAD)); - } - - // Visible for testing - OffsetIndex readOffsetIndex(ColumnChunkMetaData column) throws IOException { - IndexReference ref = column.getOffsetIndexReference(); - if (ref == null) { - return null; - } - inputStream.seek(ref.getOffset()); - - BlockCipher.Decryptor offsetIndexDecryptor = null; - byte[] offsetIndexAAD = null; - if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { - InternalColumnDecryptionSetup columnDecryptionSetup = - fileDecryptor.getColumnSetup(column.getPath()); - if (columnDecryptionSetup.isEncrypted()) { - offsetIndexDecryptor = columnDecryptionSetup.getMetaDataDecryptor(); - offsetIndexAAD = - AesCipher.createModuleAAD( - fileDecryptor.getFileAAD(), - ModuleCipherFactory.ModuleType.OffsetIndex, - column.getRowGroupOrdinal(), - columnDecryptionSetup.getOrdinal(), - -1); - } - } - return ParquetMetadataConverter.fromParquetOffsetIndex( - Util.readOffsetIndex(inputStream, offsetIndexDecryptor, offsetIndexAAD)); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java deleted file mode 100644 index 744d128304..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ColumnPageReader.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.List; -import java.util.Queue; - -import org.apache.parquet.bytes.BytesInput; -import org.apache.parquet.column.page.DataPage; -import org.apache.parquet.column.page.DataPageV1; -import org.apache.parquet.column.page.DataPageV2; -import org.apache.parquet.column.page.DictionaryPage; -import org.apache.parquet.column.page.PageReader; -import org.apache.parquet.compression.CompressionCodecFactory; -import org.apache.parquet.crypto.AesCipher; -import org.apache.parquet.crypto.ModuleCipherFactory; -import org.apache.parquet.format.BlockCipher; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.io.ParquetDecodingException; - -public class ColumnPageReader implements PageReader { - private final CompressionCodecFactory.BytesInputDecompressor decompressor; - private final long valueCount; - private final Queue compressedPages; - private final DictionaryPage compressedDictionaryPage; - - private final OffsetIndex offsetIndex; - private final long rowCount; - private int pageIndex = 0; - - private final BlockCipher.Decryptor blockDecryptor; - private final byte[] dataPageAAD; - private final byte[] dictionaryPageAAD; - - ColumnPageReader( - CompressionCodecFactory.BytesInputDecompressor decompressor, - List compressedPages, - DictionaryPage compressedDictionaryPage, - OffsetIndex offsetIndex, - long rowCount, - BlockCipher.Decryptor blockDecryptor, - byte[] fileAAD, - int rowGroupOrdinal, - int columnOrdinal) { - this.decompressor = decompressor; - this.compressedPages = new ArrayDeque<>(compressedPages); - this.compressedDictionaryPage = compressedDictionaryPage; - long count = 0; - for (DataPage p : compressedPages) { - count += p.getValueCount(); - } - this.valueCount = count; - this.offsetIndex = offsetIndex; - this.rowCount = rowCount; - this.blockDecryptor = blockDecryptor; - - if (blockDecryptor != null) { - dataPageAAD = - AesCipher.createModuleAAD( - fileAAD, ModuleCipherFactory.ModuleType.DataPage, rowGroupOrdinal, columnOrdinal, 0); - dictionaryPageAAD = - AesCipher.createModuleAAD( - fileAAD, - ModuleCipherFactory.ModuleType.DictionaryPage, - rowGroupOrdinal, - columnOrdinal, - -1); - } else { - dataPageAAD = null; - dictionaryPageAAD = null; - } - } - - @Override - public long getTotalValueCount() { - return valueCount; - } - - /** Returns the total value count of the current page. */ - public int getPageValueCount() { - return compressedPages.element().getValueCount(); - } - - /** Skips the current page so it won't be returned by {@link #readPage()} */ - public void skipPage() { - compressedPages.poll(); - pageIndex++; - } - - @Override - public DataPage readPage() { - final DataPage compressedPage = compressedPages.poll(); - if (compressedPage == null) { - return null; - } - final int currentPageIndex = pageIndex++; - - if (null != blockDecryptor) { - AesCipher.quickUpdatePageAAD(dataPageAAD, getPageOrdinal(currentPageIndex)); - } - - return compressedPage.accept( - new DataPage.Visitor() { - @Override - public DataPage visit(DataPageV1 dataPageV1) { - try { - BytesInput bytes = dataPageV1.getBytes(); - if (null != blockDecryptor) { - bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD)); - } - BytesInput decompressed = - decompressor.decompress(bytes, dataPageV1.getUncompressedSize()); - - final DataPageV1 decompressedPage; - if (offsetIndex == null) { - decompressedPage = - new DataPageV1( - decompressed, - dataPageV1.getValueCount(), - dataPageV1.getUncompressedSize(), - dataPageV1.getStatistics(), - dataPageV1.getRlEncoding(), - dataPageV1.getDlEncoding(), - dataPageV1.getValueEncoding()); - } else { - long firstRowIndex = offsetIndex.getFirstRowIndex(currentPageIndex); - decompressedPage = - new DataPageV1( - decompressed, - dataPageV1.getValueCount(), - dataPageV1.getUncompressedSize(), - firstRowIndex, - Math.toIntExact( - offsetIndex.getLastRowIndex(currentPageIndex, rowCount) - - firstRowIndex - + 1), - dataPageV1.getStatistics(), - dataPageV1.getRlEncoding(), - dataPageV1.getDlEncoding(), - dataPageV1.getValueEncoding()); - } - if (dataPageV1.getCrc().isPresent()) { - decompressedPage.setCrc(dataPageV1.getCrc().getAsInt()); - } - return decompressedPage; - } catch (IOException e) { - throw new ParquetDecodingException("could not decompress page", e); - } - } - - @Override - public DataPage visit(DataPageV2 dataPageV2) { - if (!dataPageV2.isCompressed() && offsetIndex == null && null == blockDecryptor) { - return dataPageV2; - } - BytesInput pageBytes = dataPageV2.getData(); - - if (null != blockDecryptor) { - try { - pageBytes = - BytesInput.from(blockDecryptor.decrypt(pageBytes.toByteArray(), dataPageAAD)); - } catch (IOException e) { - throw new ParquetDecodingException( - "could not convert page ByteInput to byte array", e); - } - } - if (dataPageV2.isCompressed()) { - int uncompressedSize = - Math.toIntExact( - dataPageV2.getUncompressedSize() - - dataPageV2.getDefinitionLevels().size() - - dataPageV2.getRepetitionLevels().size()); - try { - pageBytes = decompressor.decompress(pageBytes, uncompressedSize); - } catch (IOException e) { - throw new ParquetDecodingException("could not decompress page", e); - } - } - - if (offsetIndex == null) { - return DataPageV2.uncompressed( - dataPageV2.getRowCount(), - dataPageV2.getNullCount(), - dataPageV2.getValueCount(), - dataPageV2.getRepetitionLevels(), - dataPageV2.getDefinitionLevels(), - dataPageV2.getDataEncoding(), - pageBytes, - dataPageV2.getStatistics()); - } else { - return DataPageV2.uncompressed( - dataPageV2.getRowCount(), - dataPageV2.getNullCount(), - dataPageV2.getValueCount(), - offsetIndex.getFirstRowIndex(currentPageIndex), - dataPageV2.getRepetitionLevels(), - dataPageV2.getDefinitionLevels(), - dataPageV2.getDataEncoding(), - pageBytes, - dataPageV2.getStatistics()); - } - } - }); - } - - @Override - public DictionaryPage readDictionaryPage() { - if (compressedDictionaryPage == null) { - return null; - } - try { - BytesInput bytes = compressedDictionaryPage.getBytes(); - if (null != blockDecryptor) { - bytes = BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dictionaryPageAAD)); - } - DictionaryPage decompressedPage = - new DictionaryPage( - decompressor.decompress(bytes, compressedDictionaryPage.getUncompressedSize()), - compressedDictionaryPage.getDictionarySize(), - compressedDictionaryPage.getEncoding()); - if (compressedDictionaryPage.getCrc().isPresent()) { - decompressedPage.setCrc(compressedDictionaryPage.getCrc().getAsInt()); - } - return decompressedPage; - } catch (IOException e) { - throw new ParquetDecodingException("Could not decompress dictionary page", e); - } - } - - private int getPageOrdinal(int currentPageIndex) { - return offsetIndex == null ? currentPageIndex : offsetIndex.getPageOrdinal(currentPageIndex); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java deleted file mode 100644 index 6ef3bbf73c..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ /dev/null @@ -1,314 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.arrow.c.ArrowArray; -import org.apache.arrow.c.ArrowSchema; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.vector.FieldVector; -import org.apache.arrow.vector.dictionary.Dictionary; -import org.apache.arrow.vector.types.pojo.DictionaryEncoding; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.Encoding; -import org.apache.parquet.column.page.DataPage; -import org.apache.parquet.column.page.DataPageV1; -import org.apache.parquet.column.page.DataPageV2; -import org.apache.parquet.column.page.DictionaryPage; -import org.apache.parquet.column.page.PageReader; -import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.spark.sql.types.DataType; - -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.IcebergApi; -import org.apache.comet.vector.CometDecodedVector; -import org.apache.comet.vector.CometDictionary; -import org.apache.comet.vector.CometDictionaryVector; -import org.apache.comet.vector.CometPlainVector; -import org.apache.comet.vector.CometVector; - -@IcebergApi -public class ColumnReader extends AbstractColumnReader { - protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); - protected final BufferAllocator ALLOCATOR = new RootAllocator(); - - /** - * The current Comet vector holding all the values read by this column reader. Owned by this - * reader and MUST be closed after use. - */ - private CometDecodedVector currentVector; - - /** Dictionary values for this column. Only set if the column is using dictionary encoding. */ - protected CometDictionary dictionary; - - /** Reader for dictionary & data pages in the current column chunk. */ - protected PageReader pageReader; - - /** Whether the first data page has been loaded. */ - private boolean firstPageLoaded = false; - - /** - * The number of nulls in the current batch, used when we are skipping importing of Arrow vectors, - * in which case we'll simply update the null count of the existing vectors. - */ - int currentNumNulls; - - /** - * The number of values in the current batch, used when we are skipping importing of Arrow - * vectors, in which case we'll simply update the null count of the existing vectors. - */ - int currentNumValues; - - /** - * Whether the last loaded vector contains any null value. This is used to determine if we can - * skip vector reloading. If the flag is false, Arrow C API will skip to import the validity - * buffer, and therefore we cannot skip vector reloading. - */ - boolean hadNull; - - private final CometSchemaImporter importer; - - private ArrowArray array = null; - private ArrowSchema schema = null; - - ColumnReader( - DataType type, - ColumnDescriptor descriptor, - CometSchemaImporter importer, - int batchSize, - boolean useDecimal128, - boolean useLegacyDateTimestamp) { - super(type, descriptor, useDecimal128, useLegacyDateTimestamp); - assert batchSize > 0 : "Batch size must be positive, found " + batchSize; - this.batchSize = batchSize; - this.importer = importer; - initNative(); - } - - /** - * Set the page reader for a new column chunk to read. Expects to call `readBatch` after this. - * - * @param pageReader the page reader for the new column chunk - * @see Comet Issue #2079 - */ - @IcebergApi - public void setPageReader(PageReader pageReader) throws IOException { - this.pageReader = pageReader; - - DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); - if (dictionaryPage != null) { - LOG.debug("dictionary page encoding = {}", dictionaryPage.getEncoding()); - Native.setDictionaryPage( - nativeHandle, - dictionaryPage.getDictionarySize(), - dictionaryPage.getBytes().toByteArray(), - dictionaryPage.getEncoding().ordinal()); - } - } - - /** This method is called from Apache Iceberg. */ - @IcebergApi - public void setRowGroupReader(RowGroupReader rowGroupReader, ParquetColumnSpec columnSpec) - throws IOException { - ColumnDescriptor descriptor = Utils.buildColumnDescriptor(columnSpec); - setPageReader(rowGroupReader.getPageReader(descriptor)); - } - - @Override - public void readBatch(int total) { - LOG.debug("Start to batch of size = " + total); - - if (!firstPageLoaded) { - readPage(); - firstPageLoaded = true; - } - - // Now first reset the current columnar batch so that it can be used to fill in a new batch - // of values. Then, keep reading more data pages (via 'readBatch') until the current batch is - // full, or we have read 'total' number of values. - Native.resetBatch(nativeHandle); - - int left = total, nullsRead = 0; - while (left > 0) { - int[] array = Native.readBatch(nativeHandle, left); - int valuesRead = array[0]; - nullsRead += array[1]; - if (valuesRead < left) { - readPage(); - } - left -= valuesRead; - } - - this.currentNumValues = total; - this.currentNumNulls = nullsRead; - } - - /** Returns the {@link CometVector} read by this reader. */ - @Override - public CometVector currentBatch() { - return loadVector(); - } - - @Override - public void close() { - if (currentVector != null) { - currentVector.close(); - currentVector = null; - } - super.close(); - } - - /** Returns a decoded {@link CometDecodedVector Comet vector}. */ - public CometDecodedVector loadVector() { - LOG.debug("Reloading vector"); - - // Close the previous vector first to release struct memory allocated to import Arrow array & - // schema from native side, through the C data interface - if (currentVector != null) { - currentVector.close(); - } - - LogicalTypeAnnotation logicalTypeAnnotation = - descriptor.getPrimitiveType().getLogicalTypeAnnotation(); - boolean isUuid = - logicalTypeAnnotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation; - - array = ArrowArray.allocateNew(ALLOCATOR); - schema = ArrowSchema.allocateNew(ALLOCATOR); - - long arrayAddr = array.memoryAddress(); - long schemaAddr = schema.memoryAddress(); - - Native.currentBatch(nativeHandle, arrayAddr, schemaAddr); - - FieldVector vector = importer.importVector(array, schema); - - DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); - - CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); - - // Update whether the current vector contains any null values. This is used in the following - // batch(s) to determine whether we can skip loading the native vector. - hadNull = cometVector.hasNull(); - - if (dictionaryEncoding == null) { - if (dictionary != null) { - // This means the column was using dictionary encoding but now has fall-back to plain - // encoding, on the native side. Setting 'dictionary' to null here, so we can use it as - // a condition to check if we can re-use vector later. - dictionary = null; - } - // Either the column is not dictionary encoded, or it was using dictionary encoding but - // a new data page has switched back to use plain encoding. For both cases we should - // return plain vector. - currentVector = cometVector; - return currentVector; - } - - // We should already re-initiate `CometDictionary` here because `Data.importVector` API will - // release the previous dictionary vector and create a new one. - Dictionary arrowDictionary = importer.getProvider().lookup(dictionaryEncoding.getId()); - CometPlainVector dictionaryVector = - new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); - if (dictionary != null) { - dictionary.setDictionaryVector(dictionaryVector); - } else { - dictionary = new CometDictionary(dictionaryVector); - } - - currentVector = - new CometDictionaryVector( - cometVector, dictionary, importer.getProvider(), useDecimal128, false, isUuid); - - currentVector = - new CometDictionaryVector(cometVector, dictionary, importer.getProvider(), useDecimal128); - return currentVector; - } - - protected void readPage() { - DataPage page = pageReader.readPage(); - if (page == null) { - throw new RuntimeException("overreading: returned DataPage is null"); - } - ; - int pageValueCount = page.getValueCount(); - page.accept( - new DataPage.Visitor() { - @Override - public Void visit(DataPageV1 dataPageV1) { - LOG.debug("data page encoding = {}", dataPageV1.getValueEncoding()); - if (dataPageV1.getDlEncoding() != Encoding.RLE - && descriptor.getMaxDefinitionLevel() != 0) { - throw new UnsupportedOperationException( - "Unsupported encoding: " + dataPageV1.getDlEncoding()); - } - if (!isValidValueEncoding(dataPageV1.getValueEncoding())) { - throw new UnsupportedOperationException( - "Unsupported value encoding: " + dataPageV1.getValueEncoding()); - } - try { - byte[] array = dataPageV1.getBytes().toByteArray(); - Native.setPageV1( - nativeHandle, pageValueCount, array, dataPageV1.getValueEncoding().ordinal()); - } catch (IOException e) { - throw new RuntimeException(e); - } - return null; - } - - @Override - public Void visit(DataPageV2 dataPageV2) { - if (!isValidValueEncoding(dataPageV2.getDataEncoding())) { - throw new UnsupportedOperationException( - "Unsupported encoding: " + dataPageV2.getDataEncoding()); - } - try { - Native.setPageV2( - nativeHandle, - pageValueCount, - dataPageV2.getDefinitionLevels().toByteArray(), - dataPageV2.getRepetitionLevels().toByteArray(), - dataPageV2.getData().toByteArray(), - dataPageV2.getDataEncoding().ordinal()); - } catch (IOException e) { - throw new RuntimeException(e); - } - return null; - } - }); - } - - @SuppressWarnings("deprecation") - private boolean isValidValueEncoding(Encoding encoding) { - switch (encoding) { - case PLAIN: - case RLE_DICTIONARY: - case PLAIN_DICTIONARY: - return true; - default: - return false; - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/CometInputFile.java b/common/src/main/java/org/apache/comet/parquet/CometInputFile.java deleted file mode 100644 index eb54d1a724..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/CometInputFile.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FutureDataInputStreamBuilder; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.util.VersionInfo; -import org.apache.parquet.hadoop.util.HadoopStreams; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.SeekableInputStream; - -/** - * A Parquet {@link InputFile} implementation that's similar to {@link - * org.apache.parquet.hadoop.util.HadoopInputFile}, but with optimizations introduced in Hadoop 3.x, - * for S3 specifically. - */ -public class CometInputFile implements InputFile { - private static final String MAJOR_MINOR_REGEX = "^(\\d+)\\.(\\d+)(\\..*)?$"; - private static final Pattern VERSION_MATCHER = Pattern.compile(MAJOR_MINOR_REGEX); - - private final FileSystem fs; - private final FileStatus stat; - private final Configuration conf; - - public static CometInputFile fromPath(Path path, Configuration conf) throws IOException { - FileSystem fs = path.getFileSystem(conf); - return new CometInputFile(fs, fs.getFileStatus(path), conf); - } - - private CometInputFile(FileSystem fs, FileStatus stat, Configuration conf) { - this.fs = fs; - this.stat = stat; - this.conf = conf; - } - - @Override - public long getLength() { - return stat.getLen(); - } - - public Configuration getConf() { - return this.conf; - } - - public FileSystem getFileSystem() { - return this.fs; - } - - public Path getPath() { - return stat.getPath(); - } - - @Override - public SeekableInputStream newStream() throws IOException { - FSDataInputStream stream; - try { - if (isAtLeastHadoop33()) { - // If Hadoop version is >= 3.3.x, we'll use the 'openFile' API which can save a - // HEAD request from cloud storages like S3 - FutureDataInputStreamBuilder inputStreamBuilder = - fs.openFile(stat.getPath()).withFileStatus(stat); - - if (stat.getPath().toString().startsWith("s3a")) { - // Switch to random S3 input policy so that we don't do sequential read on the entire - // S3 object. By default, the policy is normal which does sequential read until a back - // seek happens, which in our case will never happen. - inputStreamBuilder = - inputStreamBuilder.opt("fs.s3a.experimental.input.fadvise", "random"); - } - stream = inputStreamBuilder.build().get(); - } else { - stream = fs.open(stat.getPath()); - } - } catch (Exception e) { - throw new IOException("Error when opening file " + stat.getPath(), e); - } - return HadoopStreams.wrap(stream); - } - - public SeekableInputStream newStream(long offset, long length) throws IOException { - try { - FSDataInputStream stream; - if (isAtLeastHadoop33()) { - FutureDataInputStreamBuilder inputStreamBuilder = - fs.openFile(stat.getPath()).withFileStatus(stat); - - if (stat.getPath().toString().startsWith("s3a")) { - // Switch to random S3 input policy so that we don't do sequential read on the entire - // S3 object. By default, the policy is normal which does sequential read until a back - // seek happens, which in our case will never happen. - // - // Also set read ahead length equal to the column chunk length so we don't have to open - // multiple S3 http connections. - inputStreamBuilder = - inputStreamBuilder - .opt("fs.s3a.experimental.input.fadvise", "random") - .opt("fs.s3a.readahead.range", Long.toString(length)); - } - - stream = inputStreamBuilder.build().get(); - } else { - stream = fs.open(stat.getPath()); - } - return HadoopStreams.wrap(stream); - } catch (Exception e) { - throw new IOException( - "Error when opening file " + stat.getPath() + ", offset=" + offset + ", length=" + length, - e); - } - } - - @Override - public String toString() { - return stat.getPath().toString(); - } - - private static boolean isAtLeastHadoop33() { - String version = VersionInfo.getVersion(); - return CometInputFile.isAtLeastHadoop33(version); - } - - static boolean isAtLeastHadoop33(String version) { - Matcher matcher = VERSION_MATCHER.matcher(version); - if (matcher.matches()) { - if (matcher.group(1).equals("3")) { - int minorVersion = Integer.parseInt(matcher.group(2)); - return minorVersion >= 3; - } - } - return false; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java b/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java deleted file mode 100644 index b4b5a8fc96..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.bytes.BytesInput; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.DictionaryPage; -import org.apache.parquet.column.page.DictionaryPageReadStore; -import org.apache.parquet.compression.CompressionCodecFactory; -import org.apache.parquet.crypto.AesCipher; -import org.apache.parquet.crypto.InternalColumnDecryptionSetup; -import org.apache.parquet.crypto.InternalFileDecryptor; -import org.apache.parquet.crypto.ModuleCipherFactory; -import org.apache.parquet.format.BlockCipher; -import org.apache.parquet.format.DictionaryPageHeader; -import org.apache.parquet.format.PageHeader; -import org.apache.parquet.format.Util; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.io.ParquetDecodingException; -import org.apache.parquet.io.SeekableInputStream; - -public class DictionaryPageReader implements DictionaryPageReadStore { - private final Map> cache; - private final InternalFileDecryptor fileDecryptor; - private final SeekableInputStream inputStream; - private final ParquetReadOptions options; - private final Map columns; - - DictionaryPageReader( - BlockMetaData block, - InternalFileDecryptor fileDecryptor, - SeekableInputStream inputStream, - ParquetReadOptions options) { - this.columns = new HashMap<>(); - this.cache = new ConcurrentHashMap<>(); - this.fileDecryptor = fileDecryptor; - this.inputStream = inputStream; - this.options = options; - - for (ColumnChunkMetaData column : block.getColumns()) { - columns.put(column.getPath().toDotString(), column); - } - } - - @Override - public DictionaryPage readDictionaryPage(ColumnDescriptor descriptor) { - String dotPath = String.join(".", descriptor.getPath()); - ColumnChunkMetaData column = columns.get(dotPath); - - if (column == null) { - throw new ParquetDecodingException("Failed to load dictionary, unknown column: " + dotPath); - } - - return cache - .computeIfAbsent( - dotPath, - key -> { - try { - final DictionaryPage dict = - column.hasDictionaryPage() ? readDictionary(column) : null; - - // Copy the dictionary to ensure it can be reused if it is returned - // more than once. This can happen when a DictionaryFilter has two or - // more predicates for the same column. Cache misses as well. - return (dict != null) ? Optional.of(reusableCopy(dict)) : Optional.empty(); - } catch (IOException e) { - throw new ParquetDecodingException("Failed to read dictionary", e); - } - }) - .orElse(null); - } - - DictionaryPage readDictionary(ColumnChunkMetaData meta) throws IOException { - if (!meta.hasDictionaryPage()) { - return null; - } - - if (inputStream.getPos() != meta.getStartingPos()) { - inputStream.seek(meta.getStartingPos()); - } - - boolean encryptedColumn = false; - InternalColumnDecryptionSetup columnDecryptionSetup = null; - byte[] dictionaryPageAAD = null; - BlockCipher.Decryptor pageDecryptor = null; - if (null != fileDecryptor && !fileDecryptor.plaintextFile()) { - columnDecryptionSetup = fileDecryptor.getColumnSetup(meta.getPath()); - if (columnDecryptionSetup.isEncrypted()) { - encryptedColumn = true; - } - } - - PageHeader pageHeader; - if (!encryptedColumn) { - pageHeader = Util.readPageHeader(inputStream); - } else { - byte[] dictionaryPageHeaderAAD = - AesCipher.createModuleAAD( - fileDecryptor.getFileAAD(), - ModuleCipherFactory.ModuleType.DictionaryPageHeader, - meta.getRowGroupOrdinal(), - columnDecryptionSetup.getOrdinal(), - -1); - pageHeader = - Util.readPageHeader( - inputStream, columnDecryptionSetup.getMetaDataDecryptor(), dictionaryPageHeaderAAD); - dictionaryPageAAD = - AesCipher.createModuleAAD( - fileDecryptor.getFileAAD(), - ModuleCipherFactory.ModuleType.DictionaryPage, - meta.getRowGroupOrdinal(), - columnDecryptionSetup.getOrdinal(), - -1); - pageDecryptor = columnDecryptionSetup.getDataDecryptor(); - } - - if (!pageHeader.isSetDictionary_page_header()) { - return null; - } - - DictionaryPage compressedPage = - readCompressedDictionary(pageHeader, inputStream, pageDecryptor, dictionaryPageAAD); - CompressionCodecFactory.BytesInputDecompressor decompressor = - options.getCodecFactory().getDecompressor(meta.getCodec()); - - return new DictionaryPage( - decompressor.decompress(compressedPage.getBytes(), compressedPage.getUncompressedSize()), - compressedPage.getDictionarySize(), - compressedPage.getEncoding()); - } - - private DictionaryPage readCompressedDictionary( - PageHeader pageHeader, - SeekableInputStream fin, - BlockCipher.Decryptor pageDecryptor, - byte[] dictionaryPageAAD) - throws IOException { - DictionaryPageHeader dictHeader = pageHeader.getDictionary_page_header(); - - int uncompressedPageSize = pageHeader.getUncompressed_page_size(); - int compressedPageSize = pageHeader.getCompressed_page_size(); - - byte[] dictPageBytes = new byte[compressedPageSize]; - fin.readFully(dictPageBytes); - - BytesInput bin = BytesInput.from(dictPageBytes); - - if (null != pageDecryptor) { - bin = BytesInput.from(pageDecryptor.decrypt(bin.toByteArray(), dictionaryPageAAD)); - } - - return new DictionaryPage( - bin, - uncompressedPageSize, - dictHeader.getNum_values(), - org.apache.parquet.column.Encoding.valueOf(dictHeader.getEncoding().name())); - } - - private static DictionaryPage reusableCopy(DictionaryPage dict) throws IOException { - return new DictionaryPage( - BytesInput.from(dict.getBytes().toByteArray()), - dict.getDictionarySize(), - dict.getEncoding()); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/FileReader.java b/common/src/main/java/org/apache/comet/parquet/FileReader.java deleted file mode 100644 index 80c214fc7c..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/FileReader.java +++ /dev/null @@ -1,1308 +0,0 @@ -/* - * 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.parquet; - -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.lang.reflect.Method; -import java.net.URI; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.zip.CRC32; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.Preconditions; -import org.apache.parquet.bytes.ByteBufferInputStream; -import org.apache.parquet.bytes.BytesInput; -import org.apache.parquet.bytes.BytesUtils; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.DataPage; -import org.apache.parquet.column.page.DataPageV1; -import org.apache.parquet.column.page.DataPageV2; -import org.apache.parquet.column.page.DictionaryPage; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.compression.CompressionCodecFactory; -import org.apache.parquet.crypto.AesCipher; -import org.apache.parquet.crypto.EncryptionPropertiesFactory; -import org.apache.parquet.crypto.FileDecryptionProperties; -import org.apache.parquet.crypto.InternalColumnDecryptionSetup; -import org.apache.parquet.crypto.InternalFileDecryptor; -import org.apache.parquet.crypto.ModuleCipherFactory; -import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.apache.parquet.filter2.compat.FilterCompat; -import org.apache.parquet.format.BlockCipher; -import org.apache.parquet.format.DataPageHeader; -import org.apache.parquet.format.DataPageHeaderV2; -import org.apache.parquet.format.DictionaryPageHeader; -import org.apache.parquet.format.FileCryptoMetaData; -import org.apache.parquet.format.PageHeader; -import org.apache.parquet.format.Util; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.ParquetInputFormat; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.parquet.hadoop.metadata.FileMetaData; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.util.counters.BenchmarkCounter; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.internal.filter2.columnindex.ColumnIndexFilter; -import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore; -import org.apache.parquet.internal.filter2.columnindex.RowRanges; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.ParquetDecodingException; -import org.apache.parquet.io.SeekableInputStream; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.spark.sql.execution.metric.SQLMetric; - -import org.apache.comet.IcebergApi; - -import static org.apache.parquet.hadoop.ParquetFileWriter.EFMAGIC; -import static org.apache.parquet.hadoop.ParquetFileWriter.MAGIC; - -import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.BLOOMFILTER; -import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.DICTIONARY; -import static org.apache.comet.parquet.RowGroupFilter.FilterLevel.STATISTICS; - -/** - * A Parquet file reader. Mostly followed {@code ParquetFileReader} in {@code parquet-mr}, but with - * customizations & optimizations for Comet. - */ -@IcebergApi -public class FileReader implements Closeable { - private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); - - private final ParquetMetadataConverter converter; - private final SeekableInputStream f; - private final InputFile file; - private final Map metrics; - private final Map paths = new HashMap<>(); - private final FileMetaData fileMetaData; // may be null - private final List blocks; - private final List blockIndexStores; - private final List blockRowRanges; - private final CRC32 crc; - private final ParquetMetadata footer; - - /** - * Read configurations come from two options: - options: these are options defined & specified - * from 'parquet-mr' library - cometOptions: these are Comet-specific options, for the features - * introduced in Comet's Parquet implementation - */ - private final ParquetReadOptions options; - - private final ReadOptions cometOptions; - - private int currentBlock = 0; - private RowGroupReader currentRowGroup = null; - private InternalFileDecryptor fileDecryptor; - - FileReader(InputFile file, ParquetReadOptions options, ReadOptions cometOptions) - throws IOException { - this(file, null, options, cometOptions, null); - } - - /** This constructor is called from Apache Iceberg. */ - @IcebergApi - public FileReader( - WrappedInputFile file, - ReadOptions cometOptions, - Map properties, - Long start, - Long length, - byte[] fileEncryptionKey, - byte[] fileAADPrefix) - throws IOException { - ParquetReadOptions options = - buildParquetReadOptions( - new Configuration(), properties, start, length, fileEncryptionKey, fileAADPrefix); - this.converter = new ParquetMetadataConverter(options); - this.file = file; - this.f = file.newStream(); - this.options = options; - this.cometOptions = cometOptions; - this.metrics = null; - try { - this.footer = readFooter(file, options, f, converter); - } catch (Exception e) { - // In case that reading footer throws an exception in the constructor, the new stream - // should be closed. Otherwise, there's no way to close this outside. - f.close(); - throw e; - } - this.fileMetaData = footer.getFileMetaData(); - this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called before filterRowGroups! - if (null != fileDecryptor && fileDecryptor.plaintextFile()) { - this.fileDecryptor = null; // Plaintext file. No need in decryptor - } - - this.blocks = footer.getBlocks(); // filter row group in iceberg - this.blockIndexStores = listWithNulls(this.blocks.size()); - this.blockRowRanges = listWithNulls(this.blocks.size()); - for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) { - paths.put(ColumnPath.get(col.getPath()), col); - } - this.crc = options.usePageChecksumVerification() ? new CRC32() : null; - } - - FileReader( - InputFile file, - ParquetReadOptions options, - ReadOptions cometOptions, - Map metrics) - throws IOException { - this(file, null, options, cometOptions, metrics); - } - - FileReader( - InputFile file, - ParquetMetadata footer, - ParquetReadOptions options, - ReadOptions cometOptions, - Map metrics) - throws IOException { - this.converter = new ParquetMetadataConverter(options); - this.file = file; - this.f = file.newStream(); - this.options = options; - this.cometOptions = cometOptions; - this.metrics = metrics; - if (footer == null) { - try { - footer = readFooter(file, options, f, converter); - } catch (Exception e) { - // In case that reading footer throws an exception in the constructor, the new stream - // should be closed. Otherwise, there's no way to close this outside. - f.close(); - throw e; - } - } - this.footer = footer; - this.fileMetaData = footer.getFileMetaData(); - this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called before filterRowGroups! - if (null != fileDecryptor && fileDecryptor.plaintextFile()) { - this.fileDecryptor = null; // Plaintext file. No need in decryptor - } - - this.blocks = filterRowGroups(footer.getBlocks()); - this.blockIndexStores = listWithNulls(this.blocks.size()); - this.blockRowRanges = listWithNulls(this.blocks.size()); - for (ColumnDescriptor col : footer.getFileMetaData().getSchema().getColumns()) { - paths.put(ColumnPath.get(col.getPath()), col); - } - this.crc = options.usePageChecksumVerification() ? new CRC32() : null; - } - - /** Returns the footer of the Parquet file being read. */ - ParquetMetadata getFooter() { - return this.footer; - } - - /** Returns the metadata of the Parquet file being read. */ - FileMetaData getFileMetaData() { - return this.fileMetaData; - } - - /** Returns the input stream of the Parquet file being read. */ - public SeekableInputStream getInputStream() { - return this.f; - } - - /** Returns the Parquet options for reading the file. */ - public ParquetReadOptions getOptions() { - return this.options; - } - - /** Returns all the row groups of this reader (after applying row group filtering). */ - public List getRowGroups() { - return blocks; - } - - /** Sets the projected columns to be read later via {@link #readNextRowGroup()} */ - public void setRequestedSchema(List projection) { - paths.clear(); - for (ColumnDescriptor col : projection) { - paths.put(ColumnPath.get(col.getPath()), col); - } - } - - /** This method is called from Apache Iceberg. */ - @IcebergApi - public void setRequestedSchemaFromSpecs(List specList) { - paths.clear(); - for (ParquetColumnSpec colSpec : specList) { - ColumnDescriptor descriptor = Utils.buildColumnDescriptor(colSpec); - paths.put(ColumnPath.get(colSpec.getPath()), descriptor); - } - } - - private static ParquetReadOptions buildParquetReadOptions( - Configuration conf, - Map properties, - Long start, - Long length, - byte[] fileEncryptionKey, - byte[] fileAADPrefix) { - - // Iceberg remove these read properties when building the ParquetReadOptions. - // We want build the exact same ParquetReadOptions as Iceberg's. - Collection readPropertiesToRemove = - Set.of( - ParquetInputFormat.UNBOUND_RECORD_FILTER, - ParquetInputFormat.FILTER_PREDICATE, - ParquetInputFormat.READ_SUPPORT_CLASS, - EncryptionPropertiesFactory.CRYPTO_FACTORY_CLASS_PROPERTY_NAME); - - for (String property : readPropertiesToRemove) { - conf.unset(property); - } - - ParquetReadOptions.Builder optionsBuilder = HadoopReadOptions.builder(conf); - for (Map.Entry entry : properties.entrySet()) { - optionsBuilder.set(entry.getKey(), entry.getValue()); - } - - if (start != null && length != null) { - optionsBuilder.withRange(start, start + length); - } - - if (fileEncryptionKey != null) { - FileDecryptionProperties fileDecryptionProperties = - FileDecryptionProperties.builder() - .withFooterKey(fileEncryptionKey) - .withAADPrefix(fileAADPrefix) - .build(); - optionsBuilder.withDecryption(fileDecryptionProperties); - } - - return optionsBuilder.build(); - } - - /** - * Gets the total number of records across all row groups (after applying row group filtering). - */ - public long getRecordCount() { - long total = 0; - for (BlockMetaData block : blocks) { - total += block.getRowCount(); - } - return total; - } - - /** - * Gets the total number of records across all row groups (after applying both row group filtering - * and page-level column index filtering). - */ - public long getFilteredRecordCount() { - if (!options.useColumnIndexFilter() - || !FilterCompat.isFilteringRequired(options.getRecordFilter())) { - return getRecordCount(); - } - long total = 0; - for (int i = 0, n = blocks.size(); i < n; ++i) { - total += getRowRanges(i).rowCount(); - } - return total; - } - - /** Skips the next row group. Returns false if there's no row group to skip. Otherwise, true. */ - @IcebergApi - public boolean skipNextRowGroup() { - return advanceToNextBlock(); - } - - /** - * Returns the next row group to read (after applying row group filtering), or null if there's no - * more row group. - */ - @IcebergApi - public RowGroupReader readNextRowGroup() throws IOException { - if (currentBlock == blocks.size()) { - return null; - } - BlockMetaData block = blocks.get(currentBlock); - if (block.getRowCount() == 0) { - throw new RuntimeException("Illegal row group of 0 rows"); - } - this.currentRowGroup = new RowGroupReader(block.getRowCount(), block.getRowIndexOffset()); - // prepare the list of consecutive parts to read them in one scan - List allParts = new ArrayList<>(); - ConsecutivePartList currentParts = null; - for (ColumnChunkMetaData mc : block.getColumns()) { - ColumnPath pathKey = mc.getPath(); - ColumnDescriptor columnDescriptor = paths.get(pathKey); - if (columnDescriptor != null) { - BenchmarkCounter.incrementTotalBytes(mc.getTotalSize()); - long startingPos = mc.getStartingPos(); - boolean mergeRanges = cometOptions.isIOMergeRangesEnabled(); - int mergeRangeDelta = cometOptions.getIOMergeRangesDelta(); - - // start a new list if - - // it is the first part or - // the part is consecutive or - // the part is not consecutive but within the merge range - if (currentParts == null - || (!mergeRanges && currentParts.endPos() != startingPos) - || (mergeRanges && startingPos - currentParts.endPos() > mergeRangeDelta)) { - currentParts = new ConsecutivePartList(startingPos); - allParts.add(currentParts); - } - // if we are in a consecutive part list and there is a gap in between the parts, - // we treat the gap as a skippable chunk - long delta = startingPos - currentParts.endPos(); - if (mergeRanges && delta > 0 && delta <= mergeRangeDelta) { - // add a chunk that will be skipped because it has no column descriptor - currentParts.addChunk(new ChunkDescriptor(null, null, startingPos, delta)); - } - currentParts.addChunk( - new ChunkDescriptor(columnDescriptor, mc, startingPos, mc.getTotalSize())); - } - } - // actually read all the chunks - return readChunks(block, allParts, new ChunkListBuilder()); - } - - /** - * Returns the next row group to read (after applying both row group filtering and page level - * column index filtering), or null if there's no more row group. - */ - public PageReadStore readNextFilteredRowGroup() throws IOException { - if (currentBlock == blocks.size()) { - return null; - } - if (!options.useColumnIndexFilter() - || !FilterCompat.isFilteringRequired(options.getRecordFilter())) { - return readNextRowGroup(); - } - BlockMetaData block = blocks.get(currentBlock); - if (block.getRowCount() == 0) { - throw new RuntimeException("Illegal row group of 0 rows"); - } - ColumnIndexStore ciStore = getColumnIndexReader(currentBlock); - RowRanges rowRanges = getRowRanges(currentBlock); - long rowCount = rowRanges.rowCount(); - if (rowCount == 0) { - // There are no matching rows -> skipping this row-group - advanceToNextBlock(); - return readNextFilteredRowGroup(); - } - if (rowCount == block.getRowCount()) { - // All rows are matching -> fall back to the non-filtering path - return readNextRowGroup(); - } - - this.currentRowGroup = new RowGroupReader(rowRanges); - // prepare the list of consecutive parts to read them in one scan - ChunkListBuilder builder = new ChunkListBuilder(); - List allParts = new ArrayList<>(); - ConsecutivePartList currentParts = null; - for (ColumnChunkMetaData mc : block.getColumns()) { - ColumnPath pathKey = mc.getPath(); - ColumnDescriptor columnDescriptor = paths.get(pathKey); - if (columnDescriptor != null) { - OffsetIndex offsetIndex = ciStore.getOffsetIndex(mc.getPath()); - IndexFilter indexFilter = new IndexFilter(rowRanges, offsetIndex, block.getRowCount()); - OffsetIndex filteredOffsetIndex = indexFilter.filterOffsetIndex(); - for (IndexFilter.OffsetRange range : - indexFilter.calculateOffsetRanges(filteredOffsetIndex, mc)) { - BenchmarkCounter.incrementTotalBytes(range.length); - long startingPos = range.offset; - // first part or not consecutive => new list - if (currentParts == null || currentParts.endPos() != startingPos) { - currentParts = new ConsecutivePartList(startingPos); - allParts.add(currentParts); - } - ChunkDescriptor chunkDescriptor = - new ChunkDescriptor(columnDescriptor, mc, startingPos, range.length); - currentParts.addChunk(chunkDescriptor); - builder.setOffsetIndex(chunkDescriptor, filteredOffsetIndex); - } - } - } - // actually read all the chunks - return readChunks(block, allParts, builder); - } - - // Visible for testing - ColumnIndexReader getColumnIndexReader(int blockIndex) { - ColumnIndexReader ciStore = blockIndexStores.get(blockIndex); - if (ciStore == null) { - ciStore = ColumnIndexReader.create(blocks.get(blockIndex), paths.keySet(), fileDecryptor, f); - blockIndexStores.set(blockIndex, ciStore); - } - return ciStore; - } - - private RowGroupReader readChunks( - BlockMetaData block, List allParts, ChunkListBuilder builder) - throws IOException { - if (shouldReadParallel()) { - readAllPartsParallel(allParts, builder); - } else { - for (ConsecutivePartList consecutiveChunks : allParts) { - consecutiveChunks.readAll(f, builder); - } - } - for (Chunk chunk : builder.build()) { - readChunkPages(chunk, block); - } - - advanceToNextBlock(); - - return currentRowGroup; - } - - private boolean shouldReadParallel() { - if (file instanceof CometInputFile) { - URI uri = ((CometInputFile) file).getPath().toUri(); - return shouldReadParallel(cometOptions, uri.getScheme()); - } - - return false; - } - - static boolean shouldReadParallel(ReadOptions options, String scheme) { - return options.isParallelIOEnabled() && shouldReadParallelForScheme(scheme); - } - - private static boolean shouldReadParallelForScheme(String scheme) { - if (scheme == null) { - return false; - } - - switch (scheme) { - case "s3a": - // Only enable parallel read for S3, so far. - return true; - default: - return false; - } - } - - static class ReadRange { - - long offset = 0; - long length = 0; - List buffers = new ArrayList<>(); - - @Override - public String toString() { - return "ReadRange{" - + "offset=" - + offset - + ", length=" - + length - + ", numBuffers=" - + buffers.size() - + '}'; - } - } - - List getReadRanges(List allParts, int nBuffers) { - int nThreads = cometOptions.parallelIOThreadPoolSize(); - long buffersPerThread = nBuffers / nThreads + 1; - boolean adjustSkew = cometOptions.adjustReadRangesSkew(); - List allRanges = new ArrayList<>(); - for (ConsecutivePartList consecutiveChunk : allParts) { - ReadRange readRange = null; - long offset = consecutiveChunk.offset; - for (int i = 0; i < consecutiveChunk.buffers.size(); i++) { - if ((adjustSkew && (i % buffersPerThread == 0)) || i == 0) { - readRange = new ReadRange(); - allRanges.add(readRange); - readRange.offset = offset; - } - ByteBuffer b = consecutiveChunk.buffers.get(i); - readRange.length += b.capacity(); - readRange.buffers.add(b); - offset += b.capacity(); - } - } - if (LOG.isDebugEnabled()) { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < allRanges.size(); i++) { - sb.append(allRanges.get(i).toString()); - if (i < allRanges.size() - 1) { - sb.append(","); - } - } - LOG.debug("Read Ranges: {}", sb); - } - return allRanges; - } - - private void readAllRangesParallel(List allRanges) { - int nThreads = cometOptions.parallelIOThreadPoolSize(); - ExecutorService threadPool = CometFileReaderThreadPool.getOrCreateThreadPool(nThreads); - List> futures = new ArrayList<>(); - - for (ReadRange readRange : allRanges) { - futures.add( - threadPool.submit( - () -> { - SeekableInputStream inputStream = null; - try { - if (file instanceof CometInputFile) { - // limit the max read ahead to length of the range - inputStream = - (((CometInputFile) file).newStream(readRange.offset, readRange.length)); - LOG.debug( - "Opened new input file: {}, at offset: {}", - ((CometInputFile) file).getPath().getName(), - readRange.offset); - } else { - inputStream = file.newStream(); - } - long curPos = readRange.offset; - for (ByteBuffer buffer : readRange.buffers) { - inputStream.seek(curPos); - LOG.debug( - "Thread: {} Offset: {} Size: {}", - Thread.currentThread().getId(), - curPos, - buffer.capacity()); - inputStream.readFully(buffer); - buffer.flip(); - curPos += buffer.capacity(); - } // for - } finally { - if (inputStream != null) { - inputStream.close(); - } - } - - return null; - })); - } - for (Future future : futures) { - try { - future.get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - } - } - - /** - * Read all the consecutive part list objects in parallel. - * - * @param allParts all consecutive parts - * @param builder chunk list builder - */ - public void readAllPartsParallel(List allParts, ChunkListBuilder builder) - throws IOException { - int nBuffers = 0; - for (ConsecutivePartList consecutiveChunks : allParts) { - consecutiveChunks.allocateReadBuffers(); - nBuffers += consecutiveChunks.buffers.size(); - } - List allRanges = getReadRanges(allParts, nBuffers); - - long startNs = System.nanoTime(); - readAllRangesParallel(allRanges); - - for (ConsecutivePartList consecutiveChunks : allParts) { - consecutiveChunks.setReadMetrics(startNs); - ByteBufferInputStream stream; - stream = ByteBufferInputStream.wrap(consecutiveChunks.buffers); - // report in a counter the data we just scanned - BenchmarkCounter.incrementBytesRead(consecutiveChunks.length); - for (int i = 0; i < consecutiveChunks.chunks.size(); i++) { - ChunkDescriptor descriptor = consecutiveChunks.chunks.get(i); - if (descriptor.col != null) { - builder.add(descriptor, stream.sliceBuffers(descriptor.size)); - } else { - stream.skipFully(descriptor.size); - } - } - } - } - - private void readChunkPages(Chunk chunk, BlockMetaData block) throws IOException { - if (fileDecryptor == null || fileDecryptor.plaintextFile()) { - currentRowGroup.addColumn(chunk.descriptor.col, chunk.readAllPages()); - return; - } - // Encrypted file - ColumnPath columnPath = ColumnPath.get(chunk.descriptor.col.getPath()); - InternalColumnDecryptionSetup columnDecryptionSetup = fileDecryptor.getColumnSetup(columnPath); - if (!columnDecryptionSetup.isEncrypted()) { // plaintext column - currentRowGroup.addColumn(chunk.descriptor.col, chunk.readAllPages()); - } else { // encrypted column - currentRowGroup.addColumn( - chunk.descriptor.col, - chunk.readAllPages( - columnDecryptionSetup.getMetaDataDecryptor(), - columnDecryptionSetup.getDataDecryptor(), - fileDecryptor.getFileAAD(), - block.getOrdinal(), - columnDecryptionSetup.getOrdinal())); - } - } - - private boolean advanceToNextBlock() { - if (currentBlock == blocks.size()) { - return false; - } - // update the current block and instantiate a dictionary reader for it - ++currentBlock; - return true; - } - - public long[] getRowIndices() { - return getRowIndices(blocks); - } - - public static long[] getRowIndices(List blocks) { - long[] rowIndices = new long[blocks.size() * 2]; - for (int i = 0, n = blocks.size(); i < n; i++) { - BlockMetaData block = blocks.get(i); - rowIndices[i * 2] = getRowIndexOffset(block); - rowIndices[i * 2 + 1] = block.getRowCount(); - } - return rowIndices; - } - - // Uses reflection to get row index offset from a Parquet block metadata. - // - // The reason reflection is used here is that some Spark versions still depend on a - // Parquet version where the method `getRowIndexOffset` is not public. - public static long getRowIndexOffset(BlockMetaData metaData) { - try { - Method method = BlockMetaData.class.getMethod("getRowIndexOffset"); - method.setAccessible(true); - return (long) method.invoke(metaData); - } catch (Exception e) { - throw new RuntimeException("Error when calling getRowIndexOffset", e); - } - } - - private RowRanges getRowRanges(int blockIndex) { - Preconditions.checkState( - FilterCompat.isFilteringRequired(options.getRecordFilter()), - "Should not be invoked if filter is null or NOOP"); - RowRanges rowRanges = blockRowRanges.get(blockIndex); - if (rowRanges == null) { - rowRanges = - ColumnIndexFilter.calculateRowRanges( - options.getRecordFilter(), - getColumnIndexReader(blockIndex), - paths.keySet(), - blocks.get(blockIndex).getRowCount()); - blockRowRanges.set(blockIndex, rowRanges); - } - return rowRanges; - } - - private static ParquetMetadata readFooter( - InputFile file, - ParquetReadOptions options, - SeekableInputStream f, - ParquetMetadataConverter converter) - throws IOException { - long fileLen = file.getLength(); - String filePath = file.toString(); - LOG.debug("File length {}", fileLen); - - int FOOTER_LENGTH_SIZE = 4; - - // MAGIC + data + footer + footerIndex + MAGIC - if (fileLen < MAGIC.length + FOOTER_LENGTH_SIZE + MAGIC.length) { - throw new RuntimeException( - filePath + " is not a Parquet file (length is too low: " + fileLen + ")"); - } - - // Read footer length and magic string - with a single seek - byte[] magic = new byte[MAGIC.length]; - long fileMetadataLengthIndex = fileLen - magic.length - FOOTER_LENGTH_SIZE; - LOG.debug("reading footer index at {}", fileMetadataLengthIndex); - f.seek(fileMetadataLengthIndex); - int fileMetadataLength = BytesUtils.readIntLittleEndian(f); - f.readFully(magic); - - boolean encryptedFooterMode; - if (Arrays.equals(MAGIC, magic)) { - encryptedFooterMode = false; - } else if (Arrays.equals(EFMAGIC, magic)) { - encryptedFooterMode = true; - } else { - throw new RuntimeException( - filePath - + " is not a Parquet file. Expected magic number " - + "at tail, but found " - + Arrays.toString(magic)); - } - - long fileMetadataIndex = fileMetadataLengthIndex - fileMetadataLength; - LOG.debug("read footer length: {}, footer index: {}", fileMetadataLength, fileMetadataIndex); - if (fileMetadataIndex < magic.length || fileMetadataIndex >= fileMetadataLengthIndex) { - throw new RuntimeException( - "corrupted file: the footer index is not within the file: " + fileMetadataIndex); - } - f.seek(fileMetadataIndex); - - FileDecryptionProperties fileDecryptionProperties = options.getDecryptionProperties(); - InternalFileDecryptor fileDecryptor = null; - if (null != fileDecryptionProperties) { - fileDecryptor = new InternalFileDecryptor(fileDecryptionProperties); - } - - // Read all the footer bytes in one time to avoid multiple read operations, - // since it can be pretty time consuming for a single read operation in HDFS. - byte[] footerBytes = new byte[fileMetadataLength]; - f.readFully(footerBytes); - ByteBuffer footerBytesBuffer = ByteBuffer.wrap(footerBytes); - LOG.debug("Finished to read all footer bytes."); - InputStream footerBytesStream = ByteBufferInputStream.wrap(footerBytesBuffer); - - // Regular file, or encrypted file with plaintext footer - if (!encryptedFooterMode) { - return converter.readParquetMetadata( - footerBytesStream, options.getMetadataFilter(), fileDecryptor, false, fileMetadataLength); - } - - // Encrypted file with encrypted footer - if (fileDecryptor == null) { - throw new ParquetCryptoRuntimeException( - "Trying to read file with encrypted footer. " + "No keys available"); - } - FileCryptoMetaData fileCryptoMetaData = Util.readFileCryptoMetaData(footerBytesStream); - fileDecryptor.setFileCryptoMetaData( - fileCryptoMetaData.getEncryption_algorithm(), true, fileCryptoMetaData.getKey_metadata()); - // footer length is required only for signed plaintext footers - return converter.readParquetMetadata( - footerBytesStream, options.getMetadataFilter(), fileDecryptor, true, 0); - } - - private List filterRowGroups(List blocks) { - return filterRowGroups(options, blocks, this); - } - - public static List filterRowGroups( - ParquetReadOptions options, List blocks, FileReader fileReader) { - FilterCompat.Filter recordFilter = options.getRecordFilter(); - if (FilterCompat.isFilteringRequired(recordFilter)) { - // set up data filters based on configured levels - List levels = new ArrayList<>(); - - if (options.useStatsFilter()) { - levels.add(STATISTICS); - } - - if (options.useDictionaryFilter()) { - levels.add(DICTIONARY); - } - - if (options.useBloomFilter()) { - levels.add(BLOOMFILTER); - } - return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, fileReader); - } - - return blocks; - } - - public static List filterRowGroups( - ParquetReadOptions options, List blocks, MessageType schema) { - FilterCompat.Filter recordFilter = options.getRecordFilter(); - if (FilterCompat.isFilteringRequired(recordFilter)) { - // set up data filters based on configured levels - List levels = new ArrayList<>(); - - if (options.useStatsFilter()) { - levels.add(STATISTICS); - } - - if (options.useDictionaryFilter()) { - levels.add(DICTIONARY); - } - - if (options.useBloomFilter()) { - levels.add(BLOOMFILTER); - } - return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, schema); - } - - return blocks; - } - - private static List listWithNulls(int size) { - return Stream.generate(() -> (T) null).limit(size).collect(Collectors.toList()); - } - - public void closeStream() throws IOException { - if (f != null) { - f.close(); - } - } - - @IcebergApi - @Override - public void close() throws IOException { - try { - if (f != null) { - f.close(); - } - } finally { - options.getCodecFactory().release(); - } - } - - /** - * Builder to concatenate the buffers of the discontinuous parts for the same column. These parts - * are generated as a result of the column-index based filtering when some pages might be skipped - * at reading. - */ - private class ChunkListBuilder { - private class ChunkData { - final List buffers = new ArrayList<>(); - OffsetIndex offsetIndex; - } - - private final Map map = new HashMap<>(); - - void add(ChunkDescriptor descriptor, List buffers) { - ChunkListBuilder.ChunkData data = map.get(descriptor); - if (data == null) { - data = new ChunkData(); - map.put(descriptor, data); - } - data.buffers.addAll(buffers); - } - - void setOffsetIndex(ChunkDescriptor descriptor, OffsetIndex offsetIndex) { - ChunkData data = map.get(descriptor); - if (data == null) { - data = new ChunkData(); - map.put(descriptor, data); - } - data.offsetIndex = offsetIndex; - } - - List build() { - List chunks = new ArrayList<>(); - for (Map.Entry entry : map.entrySet()) { - ChunkDescriptor descriptor = entry.getKey(); - ChunkData data = entry.getValue(); - chunks.add(new Chunk(descriptor, data.buffers, data.offsetIndex)); - } - return chunks; - } - } - - /** The data for a column chunk */ - private class Chunk { - private final ChunkDescriptor descriptor; - private final ByteBufferInputStream stream; - final OffsetIndex offsetIndex; - - /** - * @param descriptor descriptor for the chunk - * @param buffers ByteBuffers that contain the chunk - * @param offsetIndex the offset index for this column; might be null - */ - Chunk(ChunkDescriptor descriptor, List buffers, OffsetIndex offsetIndex) { - this.descriptor = descriptor; - this.stream = ByteBufferInputStream.wrap(buffers); - this.offsetIndex = offsetIndex; - } - - protected PageHeader readPageHeader(BlockCipher.Decryptor blockDecryptor, byte[] pageHeaderAAD) - throws IOException { - return Util.readPageHeader(stream, blockDecryptor, pageHeaderAAD); - } - - /** - * Calculate checksum of input bytes, throw decoding exception if it does not match the provided - * reference crc - */ - private void verifyCrc(int referenceCrc, byte[] bytes, String exceptionMsg) { - crc.reset(); - crc.update(bytes); - if (crc.getValue() != ((long) referenceCrc & 0xffffffffL)) { - throw new ParquetDecodingException(exceptionMsg); - } - } - - private ColumnPageReader readAllPages() throws IOException { - return readAllPages(null, null, null, -1, -1); - } - - private ColumnPageReader readAllPages( - BlockCipher.Decryptor headerBlockDecryptor, - BlockCipher.Decryptor pageBlockDecryptor, - byte[] aadPrefix, - int rowGroupOrdinal, - int columnOrdinal) - throws IOException { - List pagesInChunk = new ArrayList<>(); - DictionaryPage dictionaryPage = null; - PrimitiveType type = - fileMetaData.getSchema().getType(descriptor.col.getPath()).asPrimitiveType(); - - long valuesCountReadSoFar = 0; - int dataPageCountReadSoFar = 0; - byte[] dataPageHeaderAAD = null; - if (null != headerBlockDecryptor) { - dataPageHeaderAAD = - AesCipher.createModuleAAD( - aadPrefix, - ModuleCipherFactory.ModuleType.DataPageHeader, - rowGroupOrdinal, - columnOrdinal, - getPageOrdinal(dataPageCountReadSoFar)); - } - while (hasMorePages(valuesCountReadSoFar, dataPageCountReadSoFar)) { - byte[] pageHeaderAAD = dataPageHeaderAAD; - if (null != headerBlockDecryptor) { - // Important: this verifies file integrity (makes sure dictionary page had not been - // removed) - if (null == dictionaryPage && descriptor.metadata.hasDictionaryPage()) { - pageHeaderAAD = - AesCipher.createModuleAAD( - aadPrefix, - ModuleCipherFactory.ModuleType.DictionaryPageHeader, - rowGroupOrdinal, - columnOrdinal, - -1); - } else { - int pageOrdinal = getPageOrdinal(dataPageCountReadSoFar); - AesCipher.quickUpdatePageAAD(dataPageHeaderAAD, pageOrdinal); - } - } - - PageHeader pageHeader = readPageHeader(headerBlockDecryptor, pageHeaderAAD); - int uncompressedPageSize = pageHeader.getUncompressed_page_size(); - int compressedPageSize = pageHeader.getCompressed_page_size(); - final BytesInput pageBytes; - switch (pageHeader.type) { - case DICTIONARY_PAGE: - // there is only one dictionary page per column chunk - if (dictionaryPage != null) { - throw new ParquetDecodingException( - "more than one dictionary page in column " + descriptor.col); - } - pageBytes = this.readAsBytesInput(compressedPageSize); - if (options.usePageChecksumVerification() && pageHeader.isSetCrc()) { - verifyCrc( - pageHeader.getCrc(), - pageBytes.toByteArray(), - "could not verify dictionary page integrity, CRC checksum verification failed"); - } - DictionaryPageHeader dicHeader = pageHeader.getDictionary_page_header(); - dictionaryPage = - new DictionaryPage( - pageBytes, - uncompressedPageSize, - dicHeader.getNum_values(), - converter.getEncoding(dicHeader.getEncoding())); - // Copy crc to new page, used for testing - if (pageHeader.isSetCrc()) { - dictionaryPage.setCrc(pageHeader.getCrc()); - } - break; - - case DATA_PAGE: - DataPageHeader dataHeaderV1 = pageHeader.getData_page_header(); - pageBytes = this.readAsBytesInput(compressedPageSize); - if (options.usePageChecksumVerification() && pageHeader.isSetCrc()) { - verifyCrc( - pageHeader.getCrc(), - pageBytes.toByteArray(), - "could not verify page integrity, CRC checksum verification failed"); - } - DataPageV1 dataPageV1 = - new DataPageV1( - pageBytes, - dataHeaderV1.getNum_values(), - uncompressedPageSize, - converter.fromParquetStatistics( - getFileMetaData().getCreatedBy(), dataHeaderV1.getStatistics(), type), - converter.getEncoding(dataHeaderV1.getRepetition_level_encoding()), - converter.getEncoding(dataHeaderV1.getDefinition_level_encoding()), - converter.getEncoding(dataHeaderV1.getEncoding())); - // Copy crc to new page, used for testing - if (pageHeader.isSetCrc()) { - dataPageV1.setCrc(pageHeader.getCrc()); - } - pagesInChunk.add(dataPageV1); - valuesCountReadSoFar += dataHeaderV1.getNum_values(); - ++dataPageCountReadSoFar; - break; - - case DATA_PAGE_V2: - DataPageHeaderV2 dataHeaderV2 = pageHeader.getData_page_header_v2(); - int dataSize = - compressedPageSize - - dataHeaderV2.getRepetition_levels_byte_length() - - dataHeaderV2.getDefinition_levels_byte_length(); - pagesInChunk.add( - new DataPageV2( - dataHeaderV2.getNum_rows(), - dataHeaderV2.getNum_nulls(), - dataHeaderV2.getNum_values(), - this.readAsBytesInput(dataHeaderV2.getRepetition_levels_byte_length()), - this.readAsBytesInput(dataHeaderV2.getDefinition_levels_byte_length()), - converter.getEncoding(dataHeaderV2.getEncoding()), - this.readAsBytesInput(dataSize), - uncompressedPageSize, - converter.fromParquetStatistics( - getFileMetaData().getCreatedBy(), dataHeaderV2.getStatistics(), type), - dataHeaderV2.isIs_compressed())); - valuesCountReadSoFar += dataHeaderV2.getNum_values(); - ++dataPageCountReadSoFar; - break; - - default: - LOG.debug( - "skipping page of type {} of size {}", pageHeader.getType(), compressedPageSize); - stream.skipFully(compressedPageSize); - break; - } - } - if (offsetIndex == null && valuesCountReadSoFar != descriptor.metadata.getValueCount()) { - // Would be nice to have a CorruptParquetFileException or something as a subclass? - throw new IOException( - "Expected " - + descriptor.metadata.getValueCount() - + " values in column chunk at " - + file - + " offset " - + descriptor.metadata.getFirstDataPageOffset() - + " but got " - + valuesCountReadSoFar - + " values instead over " - + pagesInChunk.size() - + " pages ending at file offset " - + (descriptor.fileOffset + stream.position())); - } - CompressionCodecFactory.BytesInputDecompressor decompressor = - options.getCodecFactory().getDecompressor(descriptor.metadata.getCodec()); - return new ColumnPageReader( - decompressor, - pagesInChunk, - dictionaryPage, - offsetIndex, - blocks.get(currentBlock).getRowCount(), - pageBlockDecryptor, - aadPrefix, - rowGroupOrdinal, - columnOrdinal); - } - - private boolean hasMorePages(long valuesCountReadSoFar, int dataPageCountReadSoFar) { - return offsetIndex == null - ? valuesCountReadSoFar < descriptor.metadata.getValueCount() - : dataPageCountReadSoFar < offsetIndex.getPageCount(); - } - - private int getPageOrdinal(int dataPageCountReadSoFar) { - if (null == offsetIndex) { - return dataPageCountReadSoFar; - } - - return offsetIndex.getPageOrdinal(dataPageCountReadSoFar); - } - - /** - * @param size the size of the page - * @return the page - * @throws IOException if there is an error while reading from the file stream - */ - public BytesInput readAsBytesInput(int size) throws IOException { - return BytesInput.from(stream.sliceBuffers(size)); - } - } - - /** - * Describes a list of consecutive parts to be read at once. A consecutive part may contain whole - * column chunks or only parts of them (some pages). - */ - private class ConsecutivePartList { - private final long offset; - private final List chunks = new ArrayList<>(); - private long length; - private final SQLMetric fileReadTimeMetric; - private final SQLMetric fileReadSizeMetric; - private final SQLMetric readThroughput; - List buffers; - - /** - * Constructor - * - * @param offset where the first chunk starts - */ - ConsecutivePartList(long offset) { - if (metrics != null) { - this.fileReadTimeMetric = metrics.get("ParquetInputFileReadTime"); - this.fileReadSizeMetric = metrics.get("ParquetInputFileReadSize"); - this.readThroughput = metrics.get("ParquetInputFileReadThroughput"); - } else { - this.fileReadTimeMetric = null; - this.fileReadSizeMetric = null; - this.readThroughput = null; - } - this.offset = offset; - } - - /** - * Adds a chunk to the list. It must be consecutive to the previous chunk. - * - * @param descriptor a chunk descriptor - */ - public void addChunk(ChunkDescriptor descriptor) { - chunks.add(descriptor); - length += descriptor.size; - } - - private void allocateReadBuffers() { - int fullAllocations = Math.toIntExact(length / options.getMaxAllocationSize()); - int lastAllocationSize = Math.toIntExact(length % options.getMaxAllocationSize()); - - int numAllocations = fullAllocations + (lastAllocationSize > 0 ? 1 : 0); - this.buffers = new ArrayList<>(numAllocations); - - for (int i = 0; i < fullAllocations; i += 1) { - this.buffers.add(options.getAllocator().allocate(options.getMaxAllocationSize())); - } - - if (lastAllocationSize > 0) { - this.buffers.add(options.getAllocator().allocate(lastAllocationSize)); - } - } - - /** - * @param f file to read the chunks from - * @param builder used to build chunk list to read the pages for the different columns - * @throws IOException if there is an error while reading from the stream - */ - public void readAll(SeekableInputStream f, ChunkListBuilder builder) throws IOException { - f.seek(offset); - - allocateReadBuffers(); - long startNs = System.nanoTime(); - - for (ByteBuffer buffer : buffers) { - f.readFully(buffer); - buffer.flip(); - } - setReadMetrics(startNs); - - // report in a counter the data we just scanned - BenchmarkCounter.incrementBytesRead(length); - ByteBufferInputStream stream = ByteBufferInputStream.wrap(buffers); - for (int i = 0; i < chunks.size(); i++) { - ChunkDescriptor descriptor = chunks.get(i); - if (descriptor.col != null) { - builder.add(descriptor, stream.sliceBuffers(descriptor.size)); - } else { - stream.skipFully(descriptor.size); - } - } - } - - private void setReadMetrics(long startNs) { - long totalFileReadTimeNs = System.nanoTime() - startNs; - double sizeInMb = ((double) length) / (1024 * 1024); - double timeInSec = ((double) totalFileReadTimeNs) / 1000_0000_0000L; - double throughput = sizeInMb / timeInSec; - LOG.debug( - "Comet: File Read stats: Length: {} MB, Time: {} secs, throughput: {} MB/sec ", - sizeInMb, - timeInSec, - throughput); - if (fileReadTimeMetric != null) { - fileReadTimeMetric.add(totalFileReadTimeNs); - } - if (fileReadSizeMetric != null) { - fileReadSizeMetric.add(length); - } - if (readThroughput != null) { - readThroughput.set(throughput); - } - } - - /** - * End position of the last byte of these chunks - * - * @return the position following the last byte of these chunks - */ - public long endPos() { - return offset + length; - } - } - - /** Information needed to read a column chunk or a part of it. */ - private static class ChunkDescriptor { - - private final ColumnDescriptor col; - private final ColumnChunkMetaData metadata; - private final long fileOffset; - private final long size; - - /** - * @param col column this chunk is part of - * @param metadata metadata for the column - * @param fileOffset offset in the file where this chunk starts - * @param size size of the chunk - */ - ChunkDescriptor( - ColumnDescriptor col, ColumnChunkMetaData metadata, long fileOffset, long size) { - this.col = col; - this.metadata = metadata; - this.fileOffset = fileOffset; - this.size = size; - } - - @Override - public int hashCode() { - return col.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } else if (obj instanceof ChunkDescriptor) { - return col.equals(((ChunkDescriptor) obj).col); - } else { - return false; - } - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/FooterReader.java b/common/src/main/java/org/apache/comet/parquet/FooterReader.java deleted file mode 100644 index 092eb442ce..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/FooterReader.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.spark.sql.execution.datasources.PartitionedFile; - -/** - * Copied from Spark's `ParquetFooterReader` in order to avoid shading issue around Parquet. - * - *

`FooterReader` is a util class which encapsulates the helper methods of reading parquet file - * footer. - */ -public class FooterReader { - public static ParquetMetadata readFooter(Configuration configuration, PartitionedFile file) - throws IOException, URISyntaxException { - long start = file.start(); - long length = file.length(); - Path filePath = new Path(new URI(file.filePath().toString())); - CometInputFile inputFile = CometInputFile.fromPath(filePath, configuration); - ParquetReadOptions readOptions = - HadoopReadOptions.builder(inputFile.getConf(), inputFile.getPath()) - .withRange(start, start + length) - .build(); - ReadOptions cometReadOptions = ReadOptions.builder(configuration).build(); - // Use try-with-resources to ensure fd is closed. - try (FileReader fileReader = new FileReader(inputFile, readOptions, cometReadOptions)) { - return fileReader.getFooter(); - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/IcebergCometNativeBatchReader.java b/common/src/main/java/org/apache/comet/parquet/IcebergCometNativeBatchReader.java deleted file mode 100644 index 7748fbbe29..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/IcebergCometNativeBatchReader.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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.parquet; - -import java.util.Map; - -import org.apache.hadoop.conf.Configuration; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.execution.metric.SQLMetric; -import org.apache.spark.sql.types.StructType; - -/** - * A specialized NativeBatchReader for Iceberg that accepts ParquetMetadata as a Thrift encoded byte - * array . This allows Iceberg to pass metadata in serialized form with a two-step initialization - * pattern. - */ -public class IcebergCometNativeBatchReader extends NativeBatchReader { - - public IcebergCometNativeBatchReader(StructType requiredSchema) { - super(); - this.sparkSchema = requiredSchema; - } - - /** Initialize the reader using FileInfo instead of PartitionedFile. */ - public void init( - Configuration conf, - FileInfo fileInfo, - byte[] parquetMetadataBytes, - byte[] nativeFilter, - int capacity, - StructType dataSchema, - boolean isCaseSensitive, - boolean useFieldId, - boolean ignoreMissingIds, - boolean useLegacyDateTimestamp, - StructType partitionSchema, - InternalRow partitionValues, - AbstractColumnReader[] preInitializedReaders, - Map metrics) - throws Throwable { - - // Set parent fields - this.conf = conf; - this.fileInfo = fileInfo; - this.footer = new ParquetMetadataSerializer().deserialize(parquetMetadataBytes); - this.nativeFilter = nativeFilter; - this.capacity = capacity; - this.dataSchema = dataSchema; - this.isCaseSensitive = isCaseSensitive; - this.useFieldId = useFieldId; - this.ignoreMissingIds = ignoreMissingIds; - this.useLegacyDateTimestamp = useLegacyDateTimestamp; - this.partitionSchema = partitionSchema; - this.partitionValues = partitionValues; - this.preInitializedReaders = preInitializedReaders; - this.metrics.clear(); - if (metrics != null) { - this.metrics.putAll(metrics); - } - - // Call parent init method - super.init(); - } - - public StructType getSparkSchema() { - return this.sparkSchema; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/IndexFilter.java b/common/src/main/java/org/apache/comet/parquet/IndexFilter.java deleted file mode 100644 index afa5687cae..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/IndexFilter.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * 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.parquet; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.internal.filter2.columnindex.RowRanges; - -public class IndexFilter { - private final RowRanges rowRanges; - private final OffsetIndex offsetIndex; - private final long totalRowCount; - - public IndexFilter(RowRanges rowRanges, OffsetIndex offsetIndex, long totalRowCount) { - this.rowRanges = rowRanges; - this.offsetIndex = offsetIndex; - this.totalRowCount = totalRowCount; - } - - OffsetIndex filterOffsetIndex() { - List indexMap = new ArrayList<>(); - for (int i = 0, n = offsetIndex.getPageCount(); i < n; ++i) { - long from = offsetIndex.getFirstRowIndex(i); - if (rowRanges.isOverlapping(from, offsetIndex.getLastRowIndex(i, totalRowCount))) { - indexMap.add(i); - } - } - - int[] indexArray = new int[indexMap.size()]; - for (int i = 0; i < indexArray.length; i++) { - indexArray[i] = indexMap.get(i); - } - return new FilteredOffsetIndex(offsetIndex, indexArray); - } - - List calculateOffsetRanges(OffsetIndex filteredOffsetIndex, ColumnChunkMetaData cm) { - List ranges = new ArrayList<>(); - long firstPageOffset = offsetIndex.getOffset(0); - int n = filteredOffsetIndex.getPageCount(); - - if (n > 0) { - OffsetRange currentRange = null; - - // Add a range for the dictionary page if required - long rowGroupOffset = cm.getStartingPos(); - if (rowGroupOffset < firstPageOffset) { - currentRange = new OffsetRange(rowGroupOffset, (int) (firstPageOffset - rowGroupOffset)); - ranges.add(currentRange); - } - - for (int i = 0; i < n; ++i) { - long offset = filteredOffsetIndex.getOffset(i); - int length = filteredOffsetIndex.getCompressedPageSize(i); - if (currentRange == null || !currentRange.extend(offset, length)) { - currentRange = new OffsetRange(offset, length); - ranges.add(currentRange); - } - } - } - return ranges; - } - - private static class FilteredOffsetIndex implements OffsetIndex { - private final OffsetIndex offsetIndex; - private final int[] indexMap; - - private FilteredOffsetIndex(OffsetIndex offsetIndex, int[] indexMap) { - this.offsetIndex = offsetIndex; - this.indexMap = indexMap; - } - - @Override - public int getPageOrdinal(int pageIndex) { - return indexMap[pageIndex]; - } - - @Override - public int getPageCount() { - return indexMap.length; - } - - @Override - public long getOffset(int pageIndex) { - return offsetIndex.getOffset(indexMap[pageIndex]); - } - - @Override - public int getCompressedPageSize(int pageIndex) { - return offsetIndex.getCompressedPageSize(indexMap[pageIndex]); - } - - @Override - public long getFirstRowIndex(int pageIndex) { - return offsetIndex.getFirstRowIndex(indexMap[pageIndex]); - } - - @Override - public long getLastRowIndex(int pageIndex, long totalRowCount) { - int nextIndex = indexMap[pageIndex] + 1; - return (nextIndex >= offsetIndex.getPageCount() - ? totalRowCount - : offsetIndex.getFirstRowIndex(nextIndex)) - - 1; - } - } - - static class OffsetRange { - final long offset; - long length; - - private OffsetRange(long offset, int length) { - this.offset = offset; - this.length = length; - } - - private boolean extend(long offset, int length) { - if (this.offset + this.length == offset) { - this.length += length; - return true; - } else { - return false; - } - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java deleted file mode 100644 index f2772908b9..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; -import org.apache.spark.sql.types.DataType; - -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.vector.CometLazyVector; -import org.apache.comet.vector.CometVector; - -public class LazyColumnReader extends ColumnReader { - - // Remember the largest skipped index for sanity checking. - private int lastSkippedRowId = Integer.MAX_VALUE; - - // Track whether the underlying page is drained. - private boolean isPageDrained = true; - - // Leftover number of rows that did not skip in the previous batch. - private int numRowsToSkipFromPrevBatch; - - // The lazy vector being updated. - private final CometLazyVector vector; - - LazyColumnReader( - DataType sparkReadType, - ColumnDescriptor descriptor, - CometSchemaImporter importer, - int batchSize, - boolean useDecimal128, - boolean useLegacyDateTimestamp) { - super(sparkReadType, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); - this.batchSize = 0; // the batch size is set later in `readBatch` - this.vector = new CometLazyVector(sparkReadType, this, useDecimal128); - } - - @Override - public void setPageReader(PageReader pageReader) throws IOException { - super.setPageReader(pageReader); - lastSkippedRowId = Integer.MAX_VALUE; - isPageDrained = true; - numRowsToSkipFromPrevBatch = 0; - currentNumValues = batchSize; - } - - /** - * Lazily read a batch of 'total' rows for this column. The includes: 1) Skip any unused rows from - * the previous batch 2) Reset the native columnar batch 3) Reset tracking variables - * - * @param total the number of rows in the batch. MUST be <= the number of rows available in this - * column chunk. - */ - @Override - public void readBatch(int total) { - // Before starting a new batch, take care of the remaining rows to skip from the previous batch. - tryPageSkip(batchSize); - numRowsToSkipFromPrevBatch += batchSize - currentNumValues; - - // Now first reset the current columnar batch so that it can be used to fill in a new batch - // of values. Then, keep reading more data pages (via 'readBatch') until the current batch is - // full, or we have read 'total' number of values. - Native.resetBatch(nativeHandle); - - batchSize = total; - currentNumValues = 0; - lastSkippedRowId = -1; - } - - @Override - public CometVector currentBatch() { - return vector; - } - - /** Read all rows up to the `batchSize`. Expects no rows are skipped so far. */ - public void readAllBatch() { - // All rows should be read without any skips so far - assert (lastSkippedRowId == -1); - - readBatch(batchSize - 1, 0); - } - - /** - * Read at least up to `rowId`. It may read beyond `rowId` if enough rows available in the page. - * It may skip reading rows before `rowId`. In case `rowId` is already read, return immediately. - * - * @param rowId the row index in the batch to read. - * @return true if `rowId` is newly materialized, or false if `rowId` is already materialized. - */ - public boolean materializeUpToIfNecessary(int rowId) { - // Not allowed reading rowId if it may have skipped previously. - assert (rowId > lastSkippedRowId); - - // If `rowId` is already materialized, return immediately. - if (rowId < currentNumValues) return false; - - int numRowsWholePageSkipped = tryPageSkip(rowId); - readBatch(rowId, numRowsWholePageSkipped); - return true; - } - - /** - * Read up to `rowId` (inclusive). If the whole pages are skipped previously in `tryPageSkip()`, - * pad the number of whole page skipped rows with nulls to the underlying vector before reading. - * - * @param rowId the row index in the batch to read. - * @param numNullRowsToPad the number of nulls to pad before reading. - */ - private void readBatch(int rowId, int numNullRowsToPad) { - if (numRowsToSkipFromPrevBatch > 0) { - // Reaches here only when starting a new batch and the page is previously drained - readPage(); - isPageDrained = false; - Native.skipBatch(nativeHandle, numRowsToSkipFromPrevBatch, true); - numRowsToSkipFromPrevBatch = 0; - } - while (rowId >= currentNumValues) { - int numRowsToRead = batchSize - currentNumValues; - if (isPageDrained) { - readPage(); - } - int[] array = Native.readBatch(nativeHandle, numRowsToRead, numNullRowsToPad); - int read = array[0]; - isPageDrained = read < numRowsToRead; - currentNumValues += read; - currentNumNulls += array[1]; - // No need to update numNullRowsToPad. numNullRowsToPad > 0 means there were whole page skips. - // That guarantees that the Native.readBatch can read up to rowId in the current page. - } - } - - /** - * Try to skip until `rowId` (exclusive). If possible, it skips whole underlying pages without - * decompressing. In that case, it returns early at the page end, so that the next iteration can - * lazily decide to `readPage()` or `tryPageSkip()` again. - * - * @param rowId the row index in the batch that it tries to skip up until (exclusive). - * @return the number of rows that the whole page skips were applied. - */ - private int tryPageSkip(int rowId) { - int total = rowId - currentNumValues; - int wholePageSkipped = 0; - if (total > 0) { - // First try to skip from the non-drained underlying page. - int skipped = isPageDrained ? 0 : Native.skipBatch(nativeHandle, total); - total -= skipped; - isPageDrained = total > 0; - if (isPageDrained) { - ColumnPageReader columnPageReader = (ColumnPageReader) pageReader; - // It is always `columnPageReader.getPageValueCount() > numRowsToSkipFromPriorBatch` - int pageValueCount = columnPageReader.getPageValueCount() - numRowsToSkipFromPrevBatch; - while (pageValueCount <= total) { - // skip the entire page if the next page is small enough - columnPageReader.skipPage(); - numRowsToSkipFromPrevBatch = 0; - total -= pageValueCount; - wholePageSkipped += pageValueCount; - pageValueCount = columnPageReader.getPageValueCount(); - } - } - - currentNumValues += skipped + wholePageSkipped; - currentNumNulls += skipped; - lastSkippedRowId = currentNumValues - 1; - } - return wholePageSkipped; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/Native.java b/common/src/main/java/org/apache/comet/parquet/Native.java deleted file mode 100644 index fbe7f23875..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/Native.java +++ /dev/null @@ -1,266 +0,0 @@ -/* - * 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.parquet; - -import java.util.Map; - -import org.apache.comet.IcebergApi; -import org.apache.comet.NativeBase; - -public final class Native extends NativeBase { - public static int[] readBatch(long handle, int batchSize) { - return readBatch(handle, batchSize, 0); - } - - public static int skipBatch(long handle, int batchSize) { - return skipBatch(handle, batchSize, false); - } - - /** Native APIs * */ - - /** - * Creates a reader for a primitive Parquet column. - * - * @param physicalTypeId id for Parquet physical type - * @param logicalTypeId id for Parquet logical type - * @param expectedPhysicalTypeId id for Parquet physical type, converted from Spark read type. - * This is used for type promotion. - * @param path the path from the root schema to the column, derived from the method - * 'ColumnDescriptor#getPath()'. - * @param maxDl the maximum definition level of the primitive column - * @param maxRl the maximum repetition level of the primitive column - * @param bitWidth (only set when logical type is INT) the bit width for the integer type (INT8, - * INT16, INT32, etc) - * @param isSigned (only set when logical type is INT) whether it is signed or unsigned int. - * @param typeLength number of bytes required to store a value of the type, only set when the - * physical type is FIXED_LEN_BYTE_ARRAY, otherwise it's 0. - * @param precision (only set when logical type is DECIMAL) precision of the decimal type - * @param expectedPrecision (only set when logical type is DECIMAL) precision of the decimal type - * from Spark read schema. This is used for type promotion. - * @param scale (only set when logical type is DECIMAL) scale of the decimal type - * @param tu (only set when logical type is TIMESTAMP) unit for the timestamp - * @param isAdjustedUtc (only set when logical type is TIMESTAMP) whether the timestamp is - * adjusted to UTC or not - * @param batchSize the batch size for the columnar read - * @param useDecimal128 whether to always return 128 bit decimal regardless of precision - * @param useLegacyDateTimestampOrNTZ whether to read legacy dates/timestamps as it is - * @return a pointer to a native Parquet column reader created - */ - public static native long initColumnReader( - int physicalTypeId, - int logicalTypeId, - int expectedPhysicalTypeId, - String[] path, - int maxDl, - int maxRl, - int bitWidth, - int expectedBitWidth, - boolean isSigned, - int typeLength, - int precision, - int expectedPrecision, - int scale, - int expectedScale, - int tu, - boolean isAdjustedUtc, - int batchSize, - boolean useDecimal128, - boolean useLegacyDateTimestampOrNTZ); - - /** - * Pass a Parquet dictionary page to the native column reader. Note this should only be called - * once per Parquet column chunk. Otherwise it'll panic. - * - * @param handle the handle to the native Parquet column reader - * @param dictionaryValueCount the number of values in this dictionary - * @param dictionaryData the actual dictionary page data, including repetition/definition levels - * as well as values - * @param encoding the encoding used by the dictionary - */ - public static native void setDictionaryPage( - long handle, int dictionaryValueCount, byte[] dictionaryData, int encoding); - - /** - * Passes a Parquet data page V1 to the native column reader. - * - * @param handle the handle to the native Parquet column reader - * @param pageValueCount the number of values in this data page - * @param pageData the actual page data, which should only contain PLAIN-encoded values. - * @param valueEncoding the encoding used by the values - */ - public static native void setPageV1( - long handle, int pageValueCount, byte[] pageData, int valueEncoding); - - /** - * Passes a Parquet data page V2 to the native column reader. - * - * @param handle the handle to the native Parquet column reader - * @param pageValueCount the number of values in this data page - * @param defLevelData the data for definition levels - * @param repLevelData the data for repetition levels - * @param valueData the data for values - * @param valueEncoding the encoding used by the values - */ - public static native void setPageV2( - long handle, - int pageValueCount, - byte[] defLevelData, - byte[] repLevelData, - byte[] valueData, - int valueEncoding); - - /** - * Reset the current columnar batch. This will clear all the content of the batch as well as any - * internal state such as the current offset. - * - * @param handle the handle to the native Parquet column reader - */ - @IcebergApi - public static native void resetBatch(long handle); - - /** - * Reads at most 'batchSize' number of rows from the native Parquet column reader. Returns a tuple - * where the first element is the actual number of rows read (including both nulls and non-nulls), - * and the second element is the number of nulls read. - * - *

If the returned value is < 'batchSize' then it means the current page has been completely - * drained. In this case, the caller should call {@link Native#setPageV1} or {@link - * Native#setPageV2} before the next 'readBatch' call. - * - *

Note that the current page could also be drained if the returned value = 'batchSize', i.e., - * the remaining number of rows in the page is exactly equal to 'batchSize'. In this case, the - * next 'readBatch' call will return 0 and the caller should call {@link Native#setPageV1} or - * {@link Native#setPageV2} next. - * - *

If `nullPadSize` > 0, it pads nulls into the underlying vector before the values will be - * read into. - * - * @param handle the handle to the native Parquet column reader - * @param batchSize the number of rows to be read - * @param nullPadSize the number of nulls to pad before reading. - * @return a tuple: (the actual number of rows read, the number of nulls read) - */ - public static native int[] readBatch(long handle, int batchSize, int nullPadSize); - - /** - * Skips at most 'batchSize' number of rows from the native Parquet column reader, and returns the - * actual number of rows skipped. - * - *

If the returned value is < 'batchSize' then it means the current page has been completely - * drained. In this case, the caller should call {@link Native#setPageV1} or {@link - * Native#setPageV2} before the next 'skipBatch' call. - * - *

Note that the current page could also be drained if the returned value = 'batchSize', i.e., - * the remaining number of rows in the page is exactly equal to 'batchSize'. In this case, the - * next 'skipBatch' call will return 0 and the caller should call {@link Native#setPageV1} or - * {@link Native#setPageV2} next. - * - * @param handle the handle to the native Parquet column reader - * @param batchSize the number of rows to skip in the current page - * @param discard if true, discard read rows without padding nulls into the underlying vector - * @return the actual number of rows skipped - */ - public static native int skipBatch(long handle, int batchSize, boolean discard); - - /** - * Returns the current batch constructed via 'readBatch' - * - * @param handle the handle to the native Parquet column reader - * @param arrayAddr the memory address to the ArrowArray struct - * @param schemaAddr the memory address to the ArrowSchema struct - */ - public static native void currentBatch(long handle, long arrayAddr, long schemaAddr); - - /** - * Closes the native Parquet column reader and releases all resources associated with it. - * - * @param handle the handle to the native Parquet column reader - */ - public static native void closeColumnReader(long handle); - - ///////////// Arrow Native Parquet Reader APIs - // TODO: Add partitionValues(?), improve requiredColumns to use a projection mask that corresponds - // to arrow. - // Add batch size, datetimeRebaseModeSpec, metrics(how?)... - - /** - * Verify that object store options are valid. An exception will be thrown if the provided options - * are not valid. - */ - public static native void validateObjectStoreConfig( - String filePath, Map objectStoreOptions); - - /** - * Initialize a record batch reader for a PartitionedFile - * - * @param filePath - * @param starts - * @param lengths - * @return a handle to the record batch reader, used in subsequent calls. - */ - public static native long initRecordBatchReader( - String filePath, - long fileSize, - long[] starts, - long[] lengths, - byte[] filter, - byte[] requiredSchema, - byte[] dataSchema, - String sessionTimezone, - int batchSize, - boolean caseSensitive, - boolean returnNullStructIfAllFieldsMissing, - Map objectStoreOptions, - CometFileKeyUnwrapper keyUnwrapper, - Object metricsNode); - - // arrow native version of read batch - - /** - * Read the next batch of data into memory on native side - * - * @param handle - * @return the number of rows read - */ - public static native int readNextRecordBatch(long handle); - - // arrow native equivalent of currentBatch. 'columnNum' is number of the column in the record - // batch - - /** - * Load the column corresponding to columnNum in the currently loaded record batch into JVM - * - * @param handle - * @param columnNum - * @param arrayAddr - * @param schemaAddr - */ - public static native void currentColumnBatch( - long handle, int columnNum, long arrayAddr, long schemaAddr); - - // arrow native version to close record batch reader - - /** - * Close the record batch reader. Free the resources - * - * @param handle - */ - public static native void closeRecordBatchReader(long handle); -} diff --git a/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java b/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java deleted file mode 100644 index a2ee4963d9..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/NativeBatchReader.java +++ /dev/null @@ -1,1101 +0,0 @@ -/* - * 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.parquet; - -import java.io.ByteArrayOutputStream; -import java.io.Closeable; -import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.net.URI; -import java.net.URISyntaxException; -import java.nio.channels.Channels; -import java.util.*; -import java.util.stream.Collectors; - -import scala.Option; -import scala.collection.Seq; -import scala.collection.mutable.Buffer; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.arrow.vector.ipc.WriteChannel; -import org.apache.arrow.vector.ipc.message.MessageSerializer; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.Preconditions; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.Type; -import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.comet.parquet.CometParquetReadSupport; -import org.apache.spark.sql.comet.util.Utils$; -import org.apache.spark.sql.errors.QueryExecutionErrors; -import org.apache.spark.sql.execution.datasources.PartitionedFile; -import org.apache.spark.sql.execution.datasources.parquet.ParquetColumn; -import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter; -import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils; -import org.apache.spark.sql.execution.metric.SQLMetric; -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 org.apache.comet.CometConf; -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.objectstore.NativeConfig; -import org.apache.comet.shims.ShimBatchReader; -import org.apache.comet.shims.ShimFileFormat; -import org.apache.comet.vector.CometVector; -import org.apache.comet.vector.NativeUtil; - -import static scala.jdk.javaapi.CollectionConverters.asJava; - -/** - * A vectorized Parquet reader that reads a Parquet file in a batched fashion. - * - *

Example of how to use this: - * - *

- *   NativeBatchReader reader = new NativeBatchReader(parquetFile, batchSize);
- *   try {
- *     reader.init();
- *     while (reader.readBatch()) {
- *       ColumnarBatch batch = reader.currentBatch();
- *       // consume the batch
- *     }
- *   } finally { // resources associated with the reader should be released
- *     reader.close();
- *   }
- * 
- */ -public class NativeBatchReader extends RecordReader implements Closeable { - - /** - * A class that contains the necessary file information for reading a Parquet file. This class - * provides an abstraction over PartitionedFile properties. - */ - public static class FileInfo { - private final long start; - private final long length; - private final String filePath; - private final long fileSize; - - public FileInfo(long start, long length, String filePath, long fileSize) - throws URISyntaxException { - this.start = start; - this.length = length; - URI uri = new Path(filePath).toUri(); - if (uri.getScheme() == null) { - uri = new Path("file://" + filePath).toUri(); - } - this.filePath = uri.toString(); - this.fileSize = fileSize; - } - - public static FileInfo fromPartitionedFile(PartitionedFile file) throws URISyntaxException { - return new FileInfo(file.start(), file.length(), file.filePath().toString(), file.fileSize()); - } - - public long start() { - return start; - } - - public long length() { - return length; - } - - public String filePath() { - return filePath; - } - - public long fileSize() { - return fileSize; - } - - public URI pathUri() throws URISyntaxException { - return new URI(filePath); - } - } - - private static final Logger LOG = LoggerFactory.getLogger(NativeBatchReader.class); - protected static final BufferAllocator ALLOCATOR = new RootAllocator(); - private NativeUtil nativeUtil = new NativeUtil(); - - protected Configuration conf; - protected int capacity; - protected boolean isCaseSensitive; - protected boolean useFieldId; - protected boolean ignoreMissingIds; - // SPARK-53535 (Spark 4.1+): when reading a struct whose requested fields are all - // missing in the Parquet file, true returns the entire struct as null (legacy - // pre-4.1 behavior); false preserves the parent struct's nullness from the file - // so non-null parents materialize as a struct of all-null fields. - protected boolean returnNullStructIfAllFieldsMissing = true; - protected StructType partitionSchema; - protected InternalRow partitionValues; - protected PartitionedFile file; - protected FileInfo fileInfo; - protected final Map metrics; - // Unfortunately CometMetricNode is from the "spark" package and cannot be used directly here - // TODO: Move it to common package? - protected Object metricsNode = null; - - protected StructType sparkSchema; - protected StructType dataSchema; - MessageType fileSchema; - protected MessageType requestedSchema; - protected CometVector[] vectors; - protected AbstractColumnReader[] columnReaders; - protected CometSchemaImporter importer; - protected ColumnarBatch currentBatch; - // private FileReader fileReader; - protected boolean[] missingColumns; - protected boolean isInitialized; - protected ParquetMetadata footer; - protected byte[] nativeFilter; - protected AbstractColumnReader[] preInitializedReaders; - - private ParquetColumn parquetColumn; - - /** - * Map from field name to spark schema index for efficient lookups during batch loading. Built - * once during initialization and reused across all batch loads. - */ - private Map sparkFieldIndexMap; - - /** - * Whether the native scan should always return decimal represented by 128 bits, regardless of its - * precision. Normally, this should be true if native execution is enabled, since Arrow compute - * kernels doesn't support 32 and 64 bit decimals yet. - */ - // TODO: (ARROW NATIVE) - private boolean useDecimal128; - - /** - * Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian) - * calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them - * to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when - * seeing these dates/timestamps. - */ - // TODO: (ARROW NATIVE) - protected boolean useLegacyDateTimestamp; - - /** The TaskContext object for executing this task. */ - private final TaskContext taskContext; - - private long totalRowCount = 0; - private long handle; - - // Protected no-arg constructor for subclasses - protected NativeBatchReader() { - this.taskContext = TaskContext$.MODULE$.get(); - this.metrics = new HashMap<>(); - } - - // Only for testing - public NativeBatchReader(String file, int capacity) { - this(file, capacity, null, null); - } - - // Only for testing - public NativeBatchReader( - String file, int capacity, StructType partitionSchema, InternalRow partitionValues) { - this(new Configuration(), file, capacity, partitionSchema, partitionValues); - } - - // Only for testing - public NativeBatchReader( - Configuration conf, - String file, - int capacity, - StructType partitionSchema, - InternalRow partitionValues) { - - this.conf = conf; - this.capacity = capacity; - this.isCaseSensitive = false; - this.useFieldId = false; - this.ignoreMissingIds = false; - this.partitionSchema = partitionSchema; - this.partitionValues = partitionValues; - - this.file = ShimBatchReader.newPartitionedFile(partitionValues, file); - this.metrics = new HashMap<>(); - - this.taskContext = TaskContext$.MODULE$.get(); - } - - private NativeBatchReader(AbstractColumnReader[] columnReaders) { - // Todo: set useDecimal128 and useLazyMaterialization - int numColumns = columnReaders.length; - this.columnReaders = new AbstractColumnReader[numColumns]; - vectors = new CometVector[numColumns]; - currentBatch = new ColumnarBatch(vectors); - // This constructor is used by Iceberg only. The columnReaders are - // initialized in Iceberg, so no need to call the init() - isInitialized = true; - this.taskContext = TaskContext$.MODULE$.get(); - this.metrics = new HashMap<>(); - } - - NativeBatchReader( - Configuration conf, - PartitionedFile inputSplit, - ParquetMetadata footer, - byte[] nativeFilter, - int capacity, - StructType sparkSchema, - StructType dataSchema, - boolean isCaseSensitive, - boolean useFieldId, - boolean ignoreMissingIds, - boolean useLegacyDateTimestamp, - boolean returnNullStructIfAllFieldsMissing, - StructType partitionSchema, - InternalRow partitionValues, - Map metrics, - Object metricsNode) { - this.conf = conf; - this.capacity = capacity; - this.sparkSchema = sparkSchema; - this.dataSchema = dataSchema; - this.isCaseSensitive = isCaseSensitive; - this.useFieldId = useFieldId; - this.ignoreMissingIds = ignoreMissingIds; - this.useLegacyDateTimestamp = useLegacyDateTimestamp; - this.returnNullStructIfAllFieldsMissing = returnNullStructIfAllFieldsMissing; - this.partitionSchema = partitionSchema; - this.partitionValues = partitionValues; - this.file = inputSplit; - this.footer = footer; - this.nativeFilter = nativeFilter; - this.metrics = metrics; - this.metricsNode = metricsNode; - this.taskContext = TaskContext$.MODULE$.get(); - } - - /** Alternate constructor that accepts FileInfo instead of PartitionedFile. */ - NativeBatchReader( - Configuration conf, - FileInfo fileInfo, - ParquetMetadata footer, - byte[] nativeFilter, - int capacity, - StructType sparkSchema, - StructType dataSchema, - boolean isCaseSensitive, - boolean useFieldId, - boolean ignoreMissingIds, - boolean useLegacyDateTimestamp, - StructType partitionSchema, - InternalRow partitionValues, - Map metrics, - Object metricsNode) { - this.conf = conf; - this.capacity = capacity; - this.sparkSchema = sparkSchema; - this.dataSchema = dataSchema; - this.isCaseSensitive = isCaseSensitive; - this.useFieldId = useFieldId; - this.ignoreMissingIds = ignoreMissingIds; - this.useLegacyDateTimestamp = useLegacyDateTimestamp; - this.partitionSchema = partitionSchema; - this.partitionValues = partitionValues; - this.fileInfo = fileInfo; - this.footer = footer; - this.nativeFilter = nativeFilter; - this.metrics = metrics; - this.metricsNode = metricsNode; - this.taskContext = TaskContext$.MODULE$.get(); - } - - /** - * Initialize this reader. The reason we don't do it in the constructor is that we want to close - * any resource hold by this reader when error happens during the initialization. - */ - public void init() throws Throwable { - - useDecimal128 = - conf.getBoolean( - CometConf.COMET_USE_DECIMAL_128().key(), - (Boolean) CometConf.COMET_USE_DECIMAL_128().defaultValue().get()); - - // Use fileInfo if available, otherwise fall back to file - long start = fileInfo != null ? fileInfo.start() : file.start(); - long length = fileInfo != null ? fileInfo.length() : file.length(); - String filePath = fileInfo != null ? fileInfo.filePath() : file.filePath().toString(); - long fileSize = fileInfo != null ? fileInfo.fileSize() : file.fileSize(); - URI pathUri = fileInfo != null ? fileInfo.pathUri() : file.pathUri(); - - ParquetReadOptions.Builder builder = HadoopReadOptions.builder(conf, new Path(filePath)); - - if (start >= 0 && length >= 0) { - builder = builder.withRange(start, start + length); - } - ParquetReadOptions readOptions = builder.build(); - - Map objectStoreOptions = - asJava(NativeConfig.extractObjectStoreOptions(conf, pathUri)); - - // TODO: enable off-heap buffer when they are ready - ReadOptions cometReadOptions = ReadOptions.builder(conf).build(); - - Path path = new Path(new URI(filePath)); - try (FileReader fileReader = - new FileReader( - CometInputFile.fromPath(path, conf), footer, readOptions, cometReadOptions, metrics)) { - - requestedSchema = footer.getFileMetaData().getSchema(); - fileSchema = requestedSchema; - - if (sparkSchema == null) { - ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(conf); - sparkSchema = converter.convert(requestedSchema); - } else { - requestedSchema = - CometParquetReadSupport.clipParquetSchema( - requestedSchema, sparkSchema, isCaseSensitive, useFieldId, ignoreMissingIds); - if (requestedSchema.getFieldCount() != sparkSchema.size()) { - throw new IllegalArgumentException( - String.format( - "Spark schema has %d columns while " + "Parquet schema has %d columns", - sparkSchema.size(), requestedSchema.getFieldCount())); - } - } - - boolean caseSensitive = - conf.getBoolean( - SQLConf.CASE_SENSITIVE().key(), - (boolean) SQLConf.CASE_SENSITIVE().defaultValue().get()); - // rename spark fields based on field_id so name of spark schema field matches the parquet - // field name - if (useFieldId && ParquetUtils.hasFieldIds(sparkSchema)) { - sparkSchema = - getSparkSchemaByFieldId(sparkSchema, requestedSchema.asGroupType(), caseSensitive); - } - - this.parquetColumn = getParquetColumn(requestedSchema, this.sparkSchema); - - // Create Column readers - List fields = requestedSchema.getFields(); - List fileFields = fileSchema.getFields(); - ParquetColumn[] parquetFields = - asJava(parquetColumn.children()).toArray(new ParquetColumn[0]); - int numColumns = fields.size(); - if (partitionSchema != null) numColumns += partitionSchema.size(); - columnReaders = new AbstractColumnReader[numColumns]; - - // Initialize missing columns and use null vectors for them - missingColumns = new boolean[numColumns]; - // We do not need the column index of the row index; but this method has the - // side effect of throwing an exception if a column with the same name is - // found which we do want (spark unit tests explicitly test for that). - ShimFileFormat.findRowIndexColumnIndexInSchema(sparkSchema); - StructField[] nonPartitionFields = sparkSchema.fields(); - boolean hasRowIndexColumn = false; - // Ranges of rows to read (needed iff row indexes are being read) - List blocks = - FileReader.filterRowGroups(readOptions, footer.getBlocks(), fileReader); - totalRowCount = fileReader.getFilteredRecordCount(); - if (totalRowCount == 0) { - // all the data is filtered out. - isInitialized = true; - return; - } - long[] starts = new long[blocks.size()]; - long[] lengths = new long[blocks.size()]; - int blockIndex = 0; - for (BlockMetaData block : blocks) { - long blockStart = block.getStartingPos(); - long blockLength = block.getCompressedSize(); - starts[blockIndex] = blockStart; - lengths[blockIndex] = blockLength; - blockIndex++; - } - for (int i = 0; i < fields.size(); i++) { - Type field = fields.get(i); - Optional optFileField = - fileFields.stream().filter(f -> f.getName().equals(field.getName())).findFirst(); - if (nonPartitionFields[i].name().equals(ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME())) { - // Values of ROW_INDEX_TEMPORARY_COLUMN_NAME column are always populated with - // generated row indexes, rather than read from the file. - // TODO(SPARK-40059): Allow users to include columns named - // FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas. - long[] rowIndices = FileReader.getRowIndices(blocks); - columnReaders[i] = - new ArrowRowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices); - hasRowIndexColumn = true; - missingColumns[i] = true; - } else if (optFileField.isPresent()) { - // The column we are reading may be a complex type in which case we check if each field in - // the requested type is in the file type (and the same data type) - // This makes the same check as Spark's VectorizedParquetReader - checkColumn(parquetFields[i]); - missingColumns[i] = false; - } else { - if (preInitializedReaders != null - && i < preInitializedReaders.length - && preInitializedReaders[i] != null) { - columnReaders[i] = preInitializedReaders[i]; - missingColumns[i] = true; - } else { - if (field.getRepetition() == Type.Repetition.REQUIRED) { - throw new IOException( - "Required column '" - + field.getName() - + "' is missing" - + " in data file " - + filePath); - } - if (field.isPrimitive()) { - ArrowConstantColumnReader reader = - new ArrowConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128); - columnReaders[i] = reader; - missingColumns[i] = true; - } else { - // the column requested is not in the file, but the native reader can handle that - // and will return nulls for all rows requested - missingColumns[i] = false; - } - } - } - } - - // Initialize constant readers for partition columns - if (partitionSchema != null) { - StructField[] partitionFields = partitionSchema.fields(); - for (int i = fields.size(); i < columnReaders.length; i++) { - int fieldIndex = i - fields.size(); - StructField field = partitionFields[fieldIndex]; - ArrowConstantColumnReader reader = - new ArrowConstantColumnReader( - field, capacity, partitionValues, fieldIndex, useDecimal128); - columnReaders[i] = reader; - } - } - - vectors = new CometVector[numColumns]; - currentBatch = new ColumnarBatch(vectors); - - // For test purpose only - // If the last external accumulator is `NumRowGroupsAccumulator`, the row group number to read - // will be updated to the accumulator. So we can check if the row groups are filtered or not - // in test case. - // Note that this tries to get thread local TaskContext object, if this is called at other - // thread, it won't update the accumulator. - if (taskContext != null) { - Option> accu = getTaskAccumulator(taskContext.taskMetrics()); - if (accu.isDefined() && accu.get().getClass().getSimpleName().equals("NumRowGroupsAcc")) { - @SuppressWarnings("unchecked") - AccumulatorV2 intAccum = (AccumulatorV2) accu.get(); - intAccum.add(blocks.size()); - } - } - - boolean encryptionEnabled = CometParquetUtils.encryptionEnabled(conf); - - // Create keyUnwrapper if encryption is enabled - CometFileKeyUnwrapper keyUnwrapper = null; - if (encryptionEnabled) { - keyUnwrapper = new CometFileKeyUnwrapper(); - keyUnwrapper.storeDecryptionKeyRetriever(filePath, conf); - } - - // Filter out columns with preinitialized readers from sparkSchema before making the - // call to native - if (preInitializedReaders != null) { - StructType filteredSchema = new StructType(); - StructField[] sparkFields = sparkSchema.fields(); - // Build name map for efficient lookups - Map fileFieldNameMap = - caseSensitive - ? buildCaseSensitiveNameMap(fileFields) - : buildCaseInsensitiveNameMap(fileFields); - for (int i = 0; i < sparkFields.length; i++) { - // Keep the column if: - // 1. It doesn't have a preinitialized reader, OR - // 2. It has a preinitialized reader but exists in fileSchema - boolean hasPreInitializedReader = - i < preInitializedReaders.length && preInitializedReaders[i] != null; - String fieldName = - caseSensitive - ? sparkFields[i].name() - : sparkFields[i].name().toLowerCase(Locale.ROOT); - boolean existsInFileSchema = fileFieldNameMap.containsKey(fieldName); - if (!hasPreInitializedReader || existsInFileSchema) { - filteredSchema = filteredSchema.add(sparkFields[i]); - } - } - sparkSchema = filteredSchema; - } - - // Native code uses "UTC" always as the timeZoneId when converting from spark to arrow schema. - String timeZoneId = "UTC"; - Schema arrowSchema = Utils$.MODULE$.toArrowSchema(sparkSchema, timeZoneId); - byte[] serializedRequestedArrowSchema = serializeArrowSchema(arrowSchema); - Schema dataArrowSchema = Utils$.MODULE$.toArrowSchema(dataSchema, timeZoneId); - byte[] serializedDataArrowSchema = serializeArrowSchema(dataArrowSchema); - - int batchSize = - conf.getInt( - CometConf.COMET_BATCH_SIZE().key(), - (Integer) CometConf.COMET_BATCH_SIZE().defaultValue().get()); - this.handle = - Native.initRecordBatchReader( - filePath, - fileSize, - starts, - lengths, - hasRowIndexColumn ? null : nativeFilter, - serializedRequestedArrowSchema, - serializedDataArrowSchema, - timeZoneId, - batchSize, - caseSensitive, - returnNullStructIfAllFieldsMissing, - objectStoreOptions, - keyUnwrapper, - metricsNode); - - // Build spark field index map for efficient lookups during batch loading - StructField[] sparkFields = sparkSchema.fields(); - sparkFieldIndexMap = new HashMap<>(); - for (int j = 0; j < sparkFields.length; j++) { - String fieldName = - caseSensitive ? sparkFields[j].name() : sparkFields[j].name().toLowerCase(Locale.ROOT); - sparkFieldIndexMap.put(fieldName, j); - } - } - isInitialized = true; - } - - private ParquetColumn getParquetColumn(MessageType schema, StructType sparkSchema) { - // We use a different config from the config that is passed in. - // This follows the setting used in Spark's SpecificParquetRecordReaderBase - Configuration config = new Configuration(conf); - config.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(), false); - config.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), false); - config.setBoolean(SQLConf.CASE_SENSITIVE().key(), false); - config.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED().key(), false); - config.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG().key(), false); - ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(config); - return converter.convertParquetColumn(schema, Option.apply(sparkSchema)); - } - - private Map> getIdToParquetFieldMap(GroupType type) { - return type.getFields().stream() - .filter(f -> f.getId() != null) - .collect(Collectors.groupingBy(f -> f.getId().intValue())); - } - - private Map> getCaseSensitiveParquetFieldMap(GroupType schema) { - return schema.getFields().stream().collect(Collectors.toMap(Type::getName, Arrays::asList)); - } - - private Map> getCaseInsensitiveParquetFieldMap(GroupType schema) { - return schema.getFields().stream() - .collect(Collectors.groupingBy(f -> f.getName().toLowerCase(Locale.ROOT))); - } - - private Map buildCaseSensitiveNameMap(List types) { - return types.stream().collect(Collectors.toMap(Type::getName, t -> t)); - } - - private Map buildCaseInsensitiveNameMap(List types) { - return types.stream() - .collect(Collectors.toMap(t -> t.getName().toLowerCase(Locale.ROOT), t -> t)); - } - - private Type getMatchingParquetFieldById( - StructField f, - Map> idToParquetFieldMap, - Map> nameToParquetFieldMap, - boolean isCaseSensitive) { - List matched = null; - int fieldId = 0; - if (ParquetUtils.hasFieldId(f)) { - fieldId = ParquetUtils.getFieldId(f); - matched = idToParquetFieldMap.get(fieldId); - } else { - String fieldName = isCaseSensitive ? f.name() : f.name().toLowerCase(Locale.ROOT); - matched = nameToParquetFieldMap.get(fieldName); - } - - if (matched == null || matched.isEmpty()) { - return null; - } - if (matched.size() > 1) { - // Need to fail if there is ambiguity, i.e. more than one field is matched - String parquetTypesString = - matched.stream().map(Type::getName).collect(Collectors.joining("[", ", ", "]")); - throw QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError( - fieldId, parquetTypesString); - } else { - return matched.get(0); - } - } - - // Derived from CometParquetReadSupport.matchFieldId - private String getMatchingNameById( - StructField f, - Map> idToParquetFieldMap, - Map> nameToParquetFieldMap, - boolean isCaseSensitive) { - Type matched = - getMatchingParquetFieldById(f, idToParquetFieldMap, nameToParquetFieldMap, isCaseSensitive); - - // When there is no ID match, we use a fake name to avoid a name match by accident - // We need this name to be unique as well, otherwise there will be type conflicts - if (matched == null) { - return CometParquetReadSupport.generateFakeColumnName(); - } else { - return matched.getName(); - } - } - - // clip ParquetGroup Type - private StructType getSparkSchemaByFieldId( - StructType schema, GroupType parquetSchema, boolean caseSensitive) { - StructType newSchema = new StructType(); - Map> idToParquetFieldMap = getIdToParquetFieldMap(parquetSchema); - Map> nameToParquetFieldMap = - caseSensitive - ? getCaseSensitiveParquetFieldMap(parquetSchema) - : getCaseInsensitiveParquetFieldMap(parquetSchema); - for (StructField f : schema.fields()) { - DataType newDataType; - String fieldName = isCaseSensitive ? f.name() : f.name().toLowerCase(Locale.ROOT); - List parquetFieldList = nameToParquetFieldMap.get(fieldName); - if (parquetFieldList == null) { - newDataType = f.dataType(); - } else { - Type fieldType = parquetFieldList.get(0); - if (f.dataType() instanceof StructType) { - newDataType = - getSparkSchemaByFieldId( - (StructType) f.dataType(), fieldType.asGroupType(), caseSensitive); - } else { - newDataType = getSparkTypeByFieldId(f.dataType(), fieldType, caseSensitive); - } - } - String matchedName = - getMatchingNameById(f, idToParquetFieldMap, nameToParquetFieldMap, isCaseSensitive); - StructField newField = f.copy(matchedName, newDataType, f.nullable(), f.metadata()); - newSchema = newSchema.add(newField); - } - return newSchema; - } - - private static boolean isPrimitiveCatalystType(DataType dataType) { - return !(dataType instanceof ArrayType) - && !(dataType instanceof MapType) - && !(dataType instanceof StructType); - } - - private DataType getSparkTypeByFieldId( - DataType dataType, Type parquetType, boolean caseSensitive) { - DataType newDataType; - if (dataType instanceof StructType) { - newDataType = - getSparkSchemaByFieldId((StructType) dataType, parquetType.asGroupType(), caseSensitive); - } else if (dataType instanceof ArrayType - && !isPrimitiveCatalystType(((ArrayType) dataType).elementType())) { - - newDataType = - getSparkArrayTypeByFieldId( - (ArrayType) dataType, parquetType.asGroupType(), caseSensitive); - } else if (dataType instanceof MapType) { - MapType mapType = (MapType) dataType; - DataType keyType = mapType.keyType(); - DataType valueType = mapType.valueType(); - DataType newKeyType; - DataType newValueType; - Type parquetMapType = parquetType.asGroupType().getFields().get(0); - Type parquetKeyType = parquetMapType.asGroupType().getType("key"); - Type parquetValueType = parquetMapType.asGroupType().getType("value"); - if (keyType instanceof StructType) { - newKeyType = - getSparkSchemaByFieldId( - (StructType) keyType, parquetKeyType.asGroupType(), caseSensitive); - } else { - newKeyType = keyType; - } - if (valueType instanceof StructType) { - newValueType = - getSparkSchemaByFieldId( - (StructType) valueType, parquetValueType.asGroupType(), caseSensitive); - } else { - newValueType = valueType; - } - newDataType = new MapType(newKeyType, newValueType, mapType.valueContainsNull()); - } else { - newDataType = dataType; - } - return newDataType; - } - - private DataType getSparkArrayTypeByFieldId( - ArrayType arrayType, GroupType parquetList, boolean caseSensitive) { - DataType newDataType; - DataType elementType = arrayType.elementType(); - DataType newElementType; - Type parquetElementType; - if (parquetList.getLogicalTypeAnnotation() == null - && parquetList.isRepetition(Type.Repetition.REPEATED)) { - parquetElementType = parquetList; - } else { - // we expect only non-primitive types here (see clipParquetListTypes for related logic) - GroupType repeatedGroup = parquetList.asGroupType().getType(0).asGroupType(); - if (repeatedGroup.getFieldCount() > 1 - || Objects.equals(repeatedGroup.getName(), "array") - || Objects.equals(repeatedGroup.getName(), parquetList.getName() + "_tuple")) { - parquetElementType = repeatedGroup; - } else { - parquetElementType = repeatedGroup.getType(0); - } - } - if (elementType instanceof StructType) { - newElementType = - getSparkSchemaByFieldId( - (StructType) elementType, parquetElementType.asGroupType(), caseSensitive); - } else { - newElementType = getSparkTypeByFieldId(elementType, parquetElementType, caseSensitive); - } - newDataType = new ArrayType(newElementType, arrayType.containsNull()); - return newDataType; - } - - private void checkParquetType(ParquetColumn column) throws IOException { - String[] path = asJava(column.path()).toArray(new String[0]); - if (containsPath(fileSchema, path)) { - if (column.isPrimitive()) { - ColumnDescriptor desc = column.descriptor().get(); - ColumnDescriptor fd = fileSchema.getColumnDescription(desc.getPath()); - TypeUtil.checkParquetType(fd, column.sparkType()); - } else { - for (ParquetColumn childColumn : asJava(column.children())) { - checkColumn(childColumn); - } - } - } else { // A missing column which is either primitive or complex - if (column.required()) { - // check if we have a preinitialized column reader for this column. - int columnIndex = getColumnIndexFromParquetColumn(column); - if (columnIndex == -1 - || preInitializedReaders == null - || columnIndex >= preInitializedReaders.length - || preInitializedReaders[columnIndex] == null) { - // Column is missing in data but the required data is non-nullable. This file is invalid. - throw new IOException( - "Required column is missing in data file. Col: " + Arrays.toString(path)); - } - } - } - } - - /** - * Get the column index in the requested schema for a given ParquetColumn. Returns -1 if not - * found. - */ - private int getColumnIndexFromParquetColumn(ParquetColumn column) { - String[] targetPath = asJava(column.path()).toArray(new String[0]); - if (targetPath.length == 0) { - return -1; - } - - // For top-level columns, match by name - String columnName = targetPath[0]; - ParquetColumn[] parquetFields = asJava(parquetColumn.children()).toArray(new ParquetColumn[0]); - for (int i = 0; i < parquetFields.length; i++) { - String[] fieldPath = asJava(parquetFields[i].path()).toArray(new String[0]); - if (fieldPath.length > 0 && fieldPath[0].equals(columnName)) { - return i; - } - } - return -1; - } - - /** - * Checks whether the given 'path' exists in 'parquetType'. The difference between this and {@link - * MessageType#containsPath(String[])} is that the latter only support paths to leaf From Spark: - * VectorizedParquetRecordReader Check whether a column from requested schema is missing from the - * file schema, or whether it conforms to the type of the file schema. - */ - private void checkColumn(ParquetColumn column) throws IOException { - String[] path = asJava(column.path()).toArray(new String[0]); - if (containsPath(fileSchema, path)) { - if (column.isPrimitive()) { - ColumnDescriptor desc = column.descriptor().get(); - ColumnDescriptor fd = fileSchema.getColumnDescription(desc.getPath()); - if (!fd.equals(desc)) { - throw new UnsupportedOperationException("Schema evolution not supported."); - } - } else { - for (ParquetColumn childColumn : asJava(column.children())) { - checkColumn(childColumn); - } - } - } else { // A missing column which is either primitive or complex - if (column.required()) { - // Column is missing in data but the required data is non-nullable. This file is invalid. - throw new IOException( - "Required column is missing in data file. Col: " + Arrays.toString(path)); - } - } - } - - /** - * Checks whether the given 'path' exists in 'parquetType'. The difference between this and {@link - * MessageType#containsPath(String[])} is that the latter only support paths to leaf nodes, while - * this support paths both to leaf and non-leaf nodes. - */ - private boolean containsPath(Type parquetType, String[] path) { - return containsPath(parquetType, path, 0); - } - - private boolean containsPath(Type parquetType, String[] path, int depth) { - if (path.length == depth) return true; - if (parquetType instanceof GroupType) { - String fieldName = path[depth]; - GroupType parquetGroupType = (GroupType) parquetType; - if (parquetGroupType.containsField(fieldName)) { - return containsPath(parquetGroupType.getType(fieldName), path, depth + 1); - } - } - return false; - } - - public void setSparkSchema(StructType schema) { - this.sparkSchema = schema; - } - - public AbstractColumnReader[] getColumnReaders() { - return columnReaders; - } - - @Override - public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) - throws IOException, InterruptedException { - // Do nothing. The initialization work is done in 'init' already. - } - - @Override - public boolean nextKeyValue() throws IOException { - return nextBatch(); - } - - @Override - public Void getCurrentKey() { - return null; - } - - @Override - public ColumnarBatch getCurrentValue() { - return currentBatch(); - } - - @Override - public float getProgress() { - return 0; - } - - /** - * Returns the current columnar batch being read. - * - *

Note that this must be called AFTER {@link NativeBatchReader#nextBatch()}. - */ - public ColumnarBatch currentBatch() { - return currentBatch; - } - - /** - * Loads the next batch of rows. This is called by Spark _and_ Iceberg - * - * @return true if there are no more rows to read, false otherwise. - */ - public boolean nextBatch() throws IOException { - Preconditions.checkState(isInitialized, "init() should be called first!"); - - // if (rowsRead >= totalRowCount) return false; - - if (totalRowCount == 0) return false; - - int batchSize; - - try { - batchSize = loadNextBatch(); - } catch (RuntimeException e) { - // Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`. - throw e; - } catch (Throwable e) { - throw new IOException(e); - } - - if (batchSize == 0) return false; - - long totalDecodeTime = 0, totalLoadTime = 0; - for (int i = 0; i < columnReaders.length; i++) { - AbstractColumnReader reader = columnReaders[i]; - long startNs = System.nanoTime(); - // TODO: read from native reader - reader.readBatch(batchSize); - // totalDecodeTime += System.nanoTime() - startNs; - // startNs = System.nanoTime(); - vectors[i] = reader.currentBatch(); - totalLoadTime += System.nanoTime() - startNs; - } - - // TODO: (ARROW NATIVE) Add Metrics - // SQLMetric decodeMetric = metrics.get("ParquetNativeDecodeTime"); - // if (decodeMetric != null) { - // decodeMetric.add(totalDecodeTime); - // } - SQLMetric loadMetric = metrics.get("ParquetNativeLoadTime"); - if (loadMetric != null) { - loadMetric.add(totalLoadTime); - } - - currentBatch.setNumRows(batchSize); - return true; - } - - @Override - public void close() throws IOException { - if (columnReaders != null) { - for (AbstractColumnReader reader : columnReaders) { - if (reader != null) { - reader.close(); - } - } - } - if (importer != null) { - importer.close(); - importer = null; - } - nativeUtil.close(); - if (this.handle > 0) { - Native.closeRecordBatchReader(this.handle); - this.handle = 0; - } - } - - @SuppressWarnings("deprecation") - private int loadNextBatch() throws Throwable { - - for (ParquetColumn childColumn : asJava(parquetColumn.children())) { - checkParquetType(childColumn); - } - - int batchSize = Native.readNextRecordBatch(this.handle); - if (batchSize == 0) { - return batchSize; - } - if (importer != null) importer.close(); - importer = new CometSchemaImporter(ALLOCATOR); - - List fields = requestedSchema.getFields(); - StructField[] sparkFields = sparkSchema.fields(); - - boolean caseSensitive = - conf.getBoolean( - SQLConf.CASE_SENSITIVE().key(), - (boolean) SQLConf.CASE_SENSITIVE().defaultValue().get()); - - for (int i = 0; i < fields.size(); i++) { - if (!missingColumns[i]) { - if (columnReaders[i] != null) columnReaders[i].close(); - // TODO: (ARROW NATIVE) handle tz, datetime & int96 rebase - Type field = fields.get(i); - - // Find the corresponding spark field by matching field names using the prebuilt map - String fieldName = - caseSensitive ? field.getName() : field.getName().toLowerCase(Locale.ROOT); - Integer sparkSchemaIndex = sparkFieldIndexMap.get(fieldName); - - if (sparkSchemaIndex == null) { - throw new IOException( - "Could not find matching Spark field for Parquet field: " + field.getName()); - } - - DataType dataType = sparkFields[sparkSchemaIndex].dataType(); - NativeColumnReader reader = - new NativeColumnReader( - this.handle, - sparkSchemaIndex, - dataType, - field, - null, - importer, - nativeUtil, - capacity, - useDecimal128, - useLegacyDateTimestamp); - columnReaders[i] = reader; - } - } - return batchSize; - } - - // Signature of externalAccums changed from returning a Buffer to returning a Seq. If comet is - // expecting a Buffer but the Spark version returns a Seq or vice versa, we get a - // method not found exception. - @SuppressWarnings("unchecked") - private Option> getTaskAccumulator(TaskMetrics taskMetrics) { - Method externalAccumsMethod; - try { - externalAccumsMethod = TaskMetrics.class.getDeclaredMethod("externalAccums"); - externalAccumsMethod.setAccessible(true); - String returnType = externalAccumsMethod.getReturnType().getName(); - if (returnType.equals("scala.collection.mutable.Buffer")) { - return ((Buffer>) externalAccumsMethod.invoke(taskMetrics)) - .lastOption(); - } else if (returnType.equals("scala.collection.Seq")) { - return ((Seq>) externalAccumsMethod.invoke(taskMetrics)).lastOption(); - } else { - return Option.apply(null); // None - } - } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - return Option.apply(null); // None - } - } - - private byte[] serializeArrowSchema(Schema schema) throws IOException { - ByteArrayOutputStream out = new ByteArrayOutputStream(); - WriteChannel writeChannel = new WriteChannel(Channels.newChannel(out)); - MessageSerializer.serialize(writeChannel, schema); - return out.toByteArray(); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/NativeColumnReader.java b/common/src/main/java/org/apache/comet/parquet/NativeColumnReader.java deleted file mode 100644 index b170ae5830..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/NativeColumnReader.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.parquet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.arrow.c.ArrowArray; -import org.apache.arrow.c.ArrowSchema; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.Type; -import org.apache.spark.sql.types.DataType; - -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.vector.*; - -import static scala.jdk.javaapi.CollectionConverters.*; - -// TODO: extend ColumnReader instead of AbstractColumnReader to reduce code duplication -public class NativeColumnReader extends AbstractColumnReader { - protected static final Logger LOG = LoggerFactory.getLogger(NativeColumnReader.class); - protected final BufferAllocator ALLOCATOR = new RootAllocator(); - - /** - * The current Comet vector holding all the values read by this column reader. Owned by this - * reader and MUST be closed after use. - */ - private CometDecodedVector currentVector; - - /** Dictionary values for this column. Only set if the column is using dictionary encoding. */ - protected CometDictionary dictionary; - - /** - * The number of values in the current batch, used when we are skipping importing of Arrow - * vectors, in which case we'll simply update the null count of the existing vectors. - */ - int currentNumValues; - - /** - * Whether the last loaded vector contains any null value. This is used to determine if we can - * skip vector reloading. If the flag is false, Arrow C API will skip to import the validity - * buffer, and therefore we cannot skip vector reloading. - */ - boolean hadNull; - - private final CometSchemaImporter importer; - private final NativeUtil nativeUtil; - - private ArrowArray array = null; - private ArrowSchema schema = null; - - private long nativeBatchHandle = 0xDEADBEEFL; - private final int columnNum; - - NativeColumnReader( - long nativeBatchHandle, - int columnNum, - DataType type, - Type fieldType, - ColumnDescriptor descriptor, - CometSchemaImporter importer, - NativeUtil nativeUtil, - int batchSize, - boolean useDecimal128, - boolean useLegacyDateTimestamp) { - super(type, fieldType, descriptor, useDecimal128, useLegacyDateTimestamp); - assert batchSize > 0 : "Batch size must be positive, found " + batchSize; - this.batchSize = batchSize; - this.nativeUtil = nativeUtil; - this.importer = importer; - this.nativeBatchHandle = nativeBatchHandle; - this.columnNum = columnNum; - initNative(); - } - - @Override - // Override in order to avoid creation of JVM side column readers - protected void initNative() { - LOG.debug( - "Native column reader {} is initialized", String.join(".", this.type.catalogString())); - nativeHandle = 0; - } - - @Override - public void readBatch(int total) { - LOG.debug("Reading column batch of size = {}", total); - - this.currentNumValues = total; - } - - /** Returns the {@link CometVector} read by this reader. */ - @Override - public CometVector currentBatch() { - return loadVector(); - } - - @Override - public void close() { - if (currentVector != null) { - currentVector.close(); - currentVector = null; - } - super.close(); - } - - /** Returns a decoded {@link CometDecodedVector Comet vector}. */ - public CometDecodedVector loadVector() { - - LOG.debug("Loading vector for next batch"); - - // Close the previous vector first to release struct memory allocated to import Arrow array & - // schema from native side, through the C data interface - if (currentVector != null) { - currentVector.close(); - } - - // TODO: ARROW NATIVE : Handle Uuid? - - array = ArrowArray.allocateNew(ALLOCATOR); - schema = ArrowSchema.allocateNew(ALLOCATOR); - - long arrayAddr = array.memoryAddress(); - long schemaAddr = schema.memoryAddress(); - - Native.currentColumnBatch(nativeBatchHandle, columnNum, arrayAddr, schemaAddr); - - ArrowArray[] arrays = {array}; - ArrowSchema[] schemas = {schema}; - - CometDecodedVector cometVector = - (CometDecodedVector) asJava(nativeUtil.importVector(arrays, schemas)).get(0); - - // Update whether the current vector contains any null values. This is used in the following - // batch(s) to determine whether we can skip loading the native vector. - hadNull = cometVector.hasNull(); - - currentVector = cometVector; - return currentVector; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java deleted file mode 100644 index 95fed362d3..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.parquet; - -import java.util.Map; - -import org.apache.comet.IcebergApi; - -/** - * Parquet ColumnSpec encapsulates the information withing a Parquet ColumnDescriptor. Utility - * methods can convert from and to a ColumnDescriptor The only purpose of this class is to allow - * passing of Column descriptors between Comet and Iceberg. This is required because Iceberg shades - * Parquet, changing the package of Parquet classes and making then incompatible with Comet. - */ -@IcebergApi -public class ParquetColumnSpec { - - private final int fieldId; - private final String[] path; - private final String physicalType; - private final int typeLength; - private final boolean isRepeated; - private final int maxDefinitionLevel; - private final int maxRepetitionLevel; - - // Logical type info - private String logicalTypeName; - private Map logicalTypeParams; - - @IcebergApi - public ParquetColumnSpec( - int fieldId, - String[] path, - String physicalType, - int typeLength, - boolean isRepeated, - int maxDefinitionLevel, - int maxRepetitionLevel, - String logicalTypeName, - Map logicalTypeParams) { - this.fieldId = fieldId; - this.path = path; - this.physicalType = physicalType; - this.typeLength = typeLength; - this.isRepeated = isRepeated; - this.maxDefinitionLevel = maxDefinitionLevel; - this.maxRepetitionLevel = maxRepetitionLevel; - this.logicalTypeName = logicalTypeName; - this.logicalTypeParams = logicalTypeParams; - } - - @IcebergApi - public int getFieldId() { - return fieldId; - } - - @IcebergApi - public String[] getPath() { - return path; - } - - @IcebergApi - public String getPhysicalType() { - return physicalType; - } - - @IcebergApi - public int getTypeLength() { - return typeLength; - } - - public boolean isRepeated() { - return isRepeated; - } - - @IcebergApi - public int getMaxRepetitionLevel() { - return maxRepetitionLevel; - } - - @IcebergApi - public int getMaxDefinitionLevel() { - return maxDefinitionLevel; - } - - @IcebergApi - public String getLogicalTypeName() { - return logicalTypeName; - } - - @IcebergApi - public Map getLogicalTypeParams() { - return logicalTypeParams; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ParquetMetadataSerializer.java b/common/src/main/java/org/apache/comet/parquet/ParquetMetadataSerializer.java deleted file mode 100644 index 32b40940a6..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ParquetMetadataSerializer.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.parquet; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; - -import org.apache.parquet.format.FileMetaData; -import org.apache.parquet.format.Util; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; - -/** - * Utility class for serializing and deserializing ParquetMetadata instances to/from byte arrays. - * This uses the Parquet format's FileMetaData structure and the underlying Thrift compact protocol - * for serialization. - */ -public class ParquetMetadataSerializer { - - private final ParquetMetadataConverter converter; - - public ParquetMetadataSerializer() { - this.converter = new ParquetMetadataConverter(); - } - - public ParquetMetadataSerializer(ParquetMetadataConverter converter) { - this.converter = converter; - } - - /** - * Serializes a ParquetMetadata instance to a byte array. - * - * @param metadata the ParquetMetadata to serialize - * @return the serialized byte array - * @throws IOException if an error occurs during serialization - */ - public byte[] serialize(ParquetMetadata metadata) throws IOException { - FileMetaData fileMetaData = converter.toParquetMetadata(1, metadata); - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - Util.writeFileMetaData(fileMetaData, outputStream); - return outputStream.toByteArray(); - } - - /** - * Deserializes a byte array back into a ParquetMetadata instance. - * - * @param bytes the serialized byte array - * @return the deserialized ParquetMetadata - * @throws IOException if an error occurs during deserialization - */ - public ParquetMetadata deserialize(byte[] bytes) throws IOException { - ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); - FileMetaData fileMetaData = Util.readFileMetaData(inputStream); - return converter.fromParquetMetadata(fileMetaData); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java b/common/src/main/java/org/apache/comet/parquet/ReadOptions.java deleted file mode 100644 index ec5c16ce8e..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/ReadOptions.java +++ /dev/null @@ -1,216 +0,0 @@ -/* - * 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.parquet; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.spark.SparkEnv; -import org.apache.spark.launcher.SparkLauncher; - -import org.apache.comet.CometConf; -import org.apache.comet.IcebergApi; - -/** - * Comet specific Parquet related read options. - * - *

TODO: merge this with {@link org.apache.parquet.HadoopReadOptions} once PARQUET-2203 is done. - */ -@IcebergApi -public class ReadOptions { - private static final Logger LOG = LoggerFactory.getLogger(ReadOptions.class); - - // Max number of concurrent tasks we expect. Used to autoconfigure S3 client connections - public static final int S3A_MAX_EXPECTED_PARALLELISM = 32; - // defined in hadoop-aws - org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS - public static final String S3A_MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; - // default max connections in S3A - org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS - public static final int S3A_DEFAULT_MAX_HTTP_CONNECTIONS = 96; - - public static final String S3A_READAHEAD_RANGE = "fs.s3a.readahead.range"; - // Default read ahead range in Hadoop is 64K; we increase it to 1 MB - public static final long COMET_DEFAULT_READAHEAD_RANGE = 1 * 1024 * 1024; // 1 MB - - private final boolean parallelIOEnabled; - private final int parallelIOThreadPoolSize; - private final boolean ioMergeRanges; - private final int ioMergeRangesDelta; - private final boolean adjustReadRangeSkew; - - ReadOptions( - boolean parallelIOEnabled, - int parallelIOThreadPoolSize, - boolean ioMergeRanges, - int ioMergeRangesDelta, - boolean adjustReadRangeSkew) { - this.parallelIOEnabled = parallelIOEnabled; - this.parallelIOThreadPoolSize = parallelIOThreadPoolSize; - this.ioMergeRanges = ioMergeRanges; - this.ioMergeRangesDelta = ioMergeRangesDelta; - this.adjustReadRangeSkew = adjustReadRangeSkew; - } - - public boolean isParallelIOEnabled() { - return this.parallelIOEnabled; - } - - public int parallelIOThreadPoolSize() { - return this.parallelIOThreadPoolSize; - } - - public boolean isIOMergeRangesEnabled() { - return ioMergeRanges; - } - - public int getIOMergeRangesDelta() { - return ioMergeRangesDelta; - } - - public boolean adjustReadRangesSkew() { - return adjustReadRangeSkew; - } - - @IcebergApi - public static Builder builder(Configuration conf) { - return new Builder(conf); - } - - @IcebergApi - public static class Builder { - private final Configuration conf; - - private boolean parallelIOEnabled; - private int parallelIOThreadPoolSize; - private boolean ioMergeRanges; - private int ioMergeRangesDelta; - private boolean adjustReadRangeSkew; - - /** - * Whether to enable Parquet parallel IO when reading row groups. If true, Parquet reader will - * use multiple threads to read multiple chunks of data from the current row group in parallel. - */ - public Builder enableParallelIO(boolean b) { - this.parallelIOEnabled = b; - return this; - } - - /** - * Specify the number of threads to be used in parallel IO. - * - *

Note: this will only be effective if parallel IO is enabled (e.g., via {@link - * #enableParallelIO(boolean)}). - */ - public Builder withParallelIOThreadPoolSize(int numThreads) { - this.parallelIOThreadPoolSize = numThreads; - return this; - } - - public Builder enableIOMergeRanges(boolean enableIOMergeRanges) { - this.ioMergeRanges = enableIOMergeRanges; - return this; - } - - public Builder withIOMergeRangesDelta(int ioMergeRangesDelta) { - this.ioMergeRangesDelta = ioMergeRangesDelta; - return this; - } - - public Builder adjustReadRangeSkew(boolean adjustReadRangeSkew) { - this.adjustReadRangeSkew = adjustReadRangeSkew; - return this; - } - - @IcebergApi - public ReadOptions build() { - return new ReadOptions( - parallelIOEnabled, - parallelIOThreadPoolSize, - ioMergeRanges, - ioMergeRangesDelta, - adjustReadRangeSkew); - } - - @IcebergApi - public Builder(Configuration conf) { - this.conf = conf; - this.parallelIOEnabled = - conf.getBoolean( - CometConf.COMET_PARQUET_PARALLEL_IO_ENABLED().key(), - (Boolean) CometConf.COMET_PARQUET_PARALLEL_IO_ENABLED().defaultValue().get()); - this.parallelIOThreadPoolSize = - conf.getInt( - CometConf.COMET_PARQUET_PARALLEL_IO_THREADS().key(), - (Integer) CometConf.COMET_PARQUET_PARALLEL_IO_THREADS().defaultValue().get()); - this.ioMergeRanges = - conf.getBoolean( - CometConf.COMET_IO_MERGE_RANGES().key(), - (boolean) CometConf.COMET_IO_MERGE_RANGES().defaultValue().get()); - this.ioMergeRangesDelta = - conf.getInt( - CometConf.COMET_IO_MERGE_RANGES_DELTA().key(), - (Integer) CometConf.COMET_IO_MERGE_RANGES_DELTA().defaultValue().get()); - this.adjustReadRangeSkew = - conf.getBoolean( - CometConf.COMET_IO_ADJUST_READRANGE_SKEW().key(), - (Boolean) CometConf.COMET_IO_ADJUST_READRANGE_SKEW().defaultValue().get()); - // override some S3 defaults - setS3Config(); - } - - // For paths to S3, if the s3 connection pool max is less than twice the product of - // parallel reader threads * number of cores, then increase the connection pool max - private void setS3Config() { - int s3ConnectionsMax = S3A_DEFAULT_MAX_HTTP_CONNECTIONS; - SparkEnv env = SparkEnv.get(); - // Use a default number of cores in case we are using the FileReader outside the context - // of Spark. - int numExecutorCores = S3A_MAX_EXPECTED_PARALLELISM; - if (env != null) { - numExecutorCores = env.conf().getInt(SparkLauncher.EXECUTOR_CORES, numExecutorCores); - } - int parallelReaderThreads = this.parallelIOEnabled ? this.parallelIOThreadPoolSize : 1; - s3ConnectionsMax = Math.max(numExecutorCores * parallelReaderThreads * 2, s3ConnectionsMax); - - setS3ConfIfGreater(conf, S3A_MAXIMUM_CONNECTIONS, s3ConnectionsMax); - setS3ConfIfGreater(conf, S3A_READAHEAD_RANGE, COMET_DEFAULT_READAHEAD_RANGE); - } - - // Update the conf iff the new value is greater than the existing val - private void setS3ConfIfGreater(Configuration conf, String key, int newVal) { - int maxVal = newVal; - String curr = conf.get(key); - if (curr != null && !curr.isEmpty()) { - maxVal = Math.max(Integer.parseInt(curr), newVal); - } - LOG.info("File reader auto configured '{}={}'", key, maxVal); - conf.set(key, Integer.toString(maxVal)); - } - - // Update the conf iff the new value is greater than the existing val. This handles values that - // may have suffixes (K, M, G, T, P, E) indicating well known bytes size suffixes - private void setS3ConfIfGreater(Configuration conf, String key, long newVal) { - long maxVal = conf.getLongBytes(key, newVal); - maxVal = Math.max(maxVal, newVal); - LOG.info("File reader auto configured '{}={}'", key, maxVal); - conf.set(key, Long.toString(maxVal)); - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java b/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java deleted file mode 100644 index 60bbc4d19c..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/RowGroupFilter.java +++ /dev/null @@ -1,125 +0,0 @@ -/* - * 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.parquet; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.parquet.filter2.compat.FilterCompat; -import org.apache.parquet.filter2.compat.FilterCompat.Filter; -import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter; -import org.apache.parquet.filter2.compat.FilterCompat.Visitor; -import org.apache.parquet.filter2.dictionarylevel.DictionaryFilter; -import org.apache.parquet.filter2.predicate.FilterPredicate; -import org.apache.parquet.filter2.predicate.SchemaCompatibilityValidator; -import org.apache.parquet.filter2.statisticslevel.StatisticsFilter; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.schema.MessageType; - -public class RowGroupFilter implements Visitor> { - private final List blocks; - private final MessageType schema; - private final List levels; - private final FileReader reader; - - public enum FilterLevel { - STATISTICS, - DICTIONARY, - BLOOMFILTER - } - - public static List filterRowGroups( - List levels, Filter filter, List blocks, FileReader reader) { - return filter.accept(new RowGroupFilter(levels, blocks, reader)); - } - - public static List filterRowGroups( - List levels, Filter filter, List blocks, MessageType schema) { - return filter.accept(new RowGroupFilter(levels, blocks, schema)); - } - - private RowGroupFilter(List levels, List blocks, FileReader reader) { - this.levels = levels; - this.blocks = blocks; - this.reader = reader; - this.schema = reader.getFileMetaData().getSchema(); - } - - private RowGroupFilter(List levels, List blocks, MessageType schema) { - this.levels = levels; - this.blocks = blocks; - this.reader = null; - this.schema = schema; - } - - @Override - public List visit(FilterCompat.FilterPredicateCompat filterPredicateCompat) { - FilterPredicate filterPredicate = filterPredicateCompat.getFilterPredicate(); - - // check that the schema of the filter matches the schema of the file - SchemaCompatibilityValidator.validate(filterPredicate, schema); - - List filteredBlocks = new ArrayList<>(); - - for (BlockMetaData block : blocks) { - boolean drop = false; - - if (levels.contains(FilterLevel.STATISTICS)) { - drop = StatisticsFilter.canDrop(filterPredicate, block.getColumns()); - } - - if (!drop && levels.contains(FilterLevel.DICTIONARY)) { - drop = - DictionaryFilter.canDrop( - filterPredicate, - block.getColumns(), - new DictionaryPageReader( - block, - reader.getFileMetaData().getFileDecryptor(), - reader.getInputStream(), - reader.getOptions())); - } - - if (!drop && levels.contains(FilterLevel.BLOOMFILTER)) { - drop = - filterPredicate.accept( - new BloomFilterReader( - block, reader.getFileMetaData().getFileDecryptor(), reader.getInputStream())); - } - - if (!drop) { - filteredBlocks.add(block); - } - } - - return filteredBlocks; - } - - @Override - public List visit( - FilterCompat.UnboundRecordFilterCompat unboundRecordFilterCompat) { - return blocks; - } - - @Override - public List visit(NoOpFilter noOpFilter) { - return blocks; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java b/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java deleted file mode 100644 index 0ca7478b7b..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/RowGroupReader.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.parquet; - -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.PrimitiveIterator; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.column.page.PageReader; -import org.apache.parquet.internal.filter2.columnindex.RowRanges; - -import org.apache.comet.IcebergApi; - -@IcebergApi -public class RowGroupReader implements PageReadStore { - private final Map readers = new HashMap<>(); - private final long rowCount; - private final RowRanges rowRanges; - private final long rowIndexOffset; - - public RowGroupReader(long rowCount, long rowIndexOffset) { - this.rowCount = rowCount; - this.rowRanges = null; - this.rowIndexOffset = rowIndexOffset; - } - - RowGroupReader(RowRanges rowRanges) { - this.rowRanges = rowRanges; - this.rowCount = rowRanges.rowCount(); - this.rowIndexOffset = -1; - } - - @IcebergApi - @Override - public long getRowCount() { - return rowCount; - } - - @Override - public PageReader getPageReader(ColumnDescriptor path) { - return getPageReader(path.getPath()); - } - - public PageReader getPageReader(String[] path) { - final PageReader pageReader = readers.get(String.join(".", path)); - if (pageReader == null) { - throw new IllegalArgumentException( - path + " is not found: " + readers.keySet() + " " + rowCount); - } - return pageReader; - } - - @Override - public Optional getRowIndexes() { - return rowRanges == null ? Optional.empty() : Optional.of(rowRanges.iterator()); - } - - @Override - public Optional getRowIndexOffset() { - return this.rowIndexOffset < 0L ? Optional.empty() : Optional.of(this.rowIndexOffset); - } - - void addColumn(ColumnDescriptor path, ColumnPageReader reader) { - if (readers.put(String.join(".", path.getPath()), reader) != null) { - throw new IllegalStateException(path + " was already added"); - } - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java b/common/src/main/java/org/apache/comet/parquet/TypeUtil.java deleted file mode 100644 index 87cecdc65d..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/TypeUtil.java +++ /dev/null @@ -1,339 +0,0 @@ -/* - * 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.parquet; - -import java.util.Arrays; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.*; -import org.apache.parquet.schema.LogicalTypeAnnotation.*; -import org.apache.spark.package$; -import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException; -import org.apache.spark.sql.internal.SQLConf; -import org.apache.spark.sql.types.*; - -import org.apache.comet.CometConf; -import org.apache.comet.IcebergApi; - -import static org.apache.comet.parquet.Utils.descriptorToParquetColumnSpec; - -public class TypeUtil { - - /** - * Converts the input Spark 'field' into a Parquet column descriptor. - * - * @see Comet Issue #2079 - */ - @IcebergApi - public static ColumnDescriptor convertToParquet(StructField field) { - Type.Repetition repetition; - int maxDefinitionLevel; - if (field.nullable()) { - repetition = Type.Repetition.OPTIONAL; - maxDefinitionLevel = 1; - } else { - repetition = Type.Repetition.REQUIRED; - maxDefinitionLevel = 0; - } - String[] path = new String[] {field.name()}; - - DataType type = field.dataType(); - - Types.PrimitiveBuilder builder = null; - // Only partition column can be `NullType`. Here we piggy-back onto Parquet boolean type - // for constant vector of null values, we don't really care what Parquet type it is. - if (type == DataTypes.BooleanType || type == DataTypes.NullType) { - builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition); - } else if (type == DataTypes.IntegerType || type instanceof YearMonthIntervalType) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) - .as(LogicalTypeAnnotation.intType(32, true)); - } else if (type == DataTypes.DateType) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) - .as(LogicalTypeAnnotation.dateType()); - } else if (type == DataTypes.ByteType) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) - .as(LogicalTypeAnnotation.intType(8, true)); - } else if (type == DataTypes.ShortType) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.INT32, repetition) - .as(LogicalTypeAnnotation.intType(16, true)); - } else if (type == DataTypes.LongType) { - builder = Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition); - } else if (type == DataTypes.BinaryType) { - builder = Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition); - } else if (type == DataTypes.StringType - || (type.sameType(DataTypes.StringType) && isSpark40Plus())) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) - .as(LogicalTypeAnnotation.stringType()); - } else if (type == DataTypes.FloatType) { - builder = Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, repetition); - } else if (type == DataTypes.DoubleType) { - builder = Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, repetition); - } else if (type == DataTypes.TimestampType) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS)); - } else if (type == TimestampNTZType$.MODULE$) { - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.INT64, repetition) - .as(LogicalTypeAnnotation.timestampType(false, TimeUnit.MICROS)); - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType) type; - builder = - Types.primitive(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, repetition) - .length(16) // always store as Decimal128 - .as(LogicalTypeAnnotation.decimalType(decimalType.scale(), decimalType.precision())); - } - if (builder == null) { - throw new UnsupportedOperationException("Unsupported input Spark type: " + type); - } - - return new ColumnDescriptor(path, builder.named(field.name()), 0, maxDefinitionLevel); - } - - public static ParquetColumnSpec convertToParquetSpec(StructField field) { - return descriptorToParquetColumnSpec(convertToParquet(field)); - } - - /** - * Check whether the Parquet 'descriptor' and Spark read type 'sparkType' are compatible. If not, - * throw exception. - * - *

This mostly follows the logic in Spark's - * ParquetVectorUpdaterFactory#getUpdater(ColumnDescriptor, DataType) - * - * @param descriptor descriptor for a Parquet primitive column - * @param sparkType Spark read type - */ - public static void checkParquetType(ColumnDescriptor descriptor, DataType sparkType) { - PrimitiveType.PrimitiveTypeName typeName = descriptor.getPrimitiveType().getPrimitiveTypeName(); - LogicalTypeAnnotation logicalTypeAnnotation = - descriptor.getPrimitiveType().getLogicalTypeAnnotation(); - boolean allowTypePromotion = (boolean) CometConf.COMET_SCHEMA_EVOLUTION_ENABLED().get(); - - if (sparkType instanceof NullType) { - return; - } - - switch (typeName) { - case BOOLEAN: - if (sparkType == DataTypes.BooleanType) return; - break; - case INT32: - if (sparkType == DataTypes.IntegerType || canReadAsIntDecimal(descriptor, sparkType)) { - return; - } else if (sparkType == DataTypes.LongType - && isUnsignedIntTypeMatched(logicalTypeAnnotation, 32)) { - // In `ParquetToSparkSchemaConverter`, we map parquet UINT32 to our LongType. - // For unsigned int32, it stores as plain signed int32 in Parquet when dictionary - // fallbacks. We read them as long values. - return; - } else if (sparkType == DataTypes.LongType && allowTypePromotion) { - // In Comet we allow schema evolution from int to long, if - // `spark.comet.schemaEvolution.enabled` is enabled. - return; - } else if (sparkType == DataTypes.ByteType || sparkType == DataTypes.ShortType) { - return; - } else if (sparkType == DataTypes.DateType) { - // TODO: use dateTimeRebaseMode from Spark side - return; - } else if (sparkType instanceof YearMonthIntervalType) { - return; - } else if (sparkType == DataTypes.DoubleType && isSpark40Plus()) { - return; - } else if (sparkType == TimestampNTZType$.MODULE$ - && isSpark40Plus() - && logicalTypeAnnotation instanceof DateLogicalTypeAnnotation) { - return; - } - break; - case INT64: - if (sparkType == DataTypes.LongType || canReadAsLongDecimal(descriptor, sparkType)) { - return; - } else if (isLongDecimal(sparkType) - && isUnsignedIntTypeMatched(logicalTypeAnnotation, 64)) { - // In `ParquetToSparkSchemaConverter`, we map parquet UINT64 to our Decimal(20, 0). - // For unsigned int64, it stores as plain signed int64 in Parquet when dictionary - // fallbacks. We read them as decimal values. - return; - } else if (isTimestampTypeMatched(logicalTypeAnnotation, TimeUnit.MICROS) - && (sparkType == TimestampNTZType$.MODULE$ || sparkType == DataTypes.TimestampType)) { - validateTimestampType(logicalTypeAnnotation, sparkType); - // TODO: use dateTimeRebaseMode from Spark side - return; - } else if (isTimestampTypeMatched(logicalTypeAnnotation, TimeUnit.MILLIS) - && (sparkType == TimestampNTZType$.MODULE$ || sparkType == DataTypes.TimestampType)) { - validateTimestampType(logicalTypeAnnotation, sparkType); - return; - } - break; - case INT96: - if (sparkType == TimestampNTZType$.MODULE$) { - if (isSpark40Plus()) return; // Spark 4.0+ supports Timestamp NTZ with INT96 - convertErrorForTimestampNTZ(typeName.name()); - } else if (sparkType == DataTypes.TimestampType) { - return; - } - break; - case FLOAT: - if (sparkType == DataTypes.FloatType) return; - // In Comet we allow schema evolution from float to double, if - // `spark.comet.schemaEvolution.enabled` is enabled. - if (sparkType == DataTypes.DoubleType && allowTypePromotion) return; - break; - case DOUBLE: - if (sparkType == DataTypes.DoubleType) return; - break; - case BINARY: - if (sparkType == DataTypes.StringType - || sparkType == DataTypes.BinaryType - || canReadAsBinaryDecimal(descriptor, sparkType)) { - return; - } - - if (sparkType.sameType(DataTypes.StringType) && isSpark40Plus()) { - LogicalTypeAnnotation lta = descriptor.getPrimitiveType().getLogicalTypeAnnotation(); - if (lta instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { - return; - } - } - break; - case FIXED_LEN_BYTE_ARRAY: - if (canReadAsIntDecimal(descriptor, sparkType) - || canReadAsLongDecimal(descriptor, sparkType) - || canReadAsBinaryDecimal(descriptor, sparkType) - || sparkType == DataTypes.BinaryType - // for uuid, since iceberg maps uuid to StringType - || sparkType == DataTypes.StringType - && logicalTypeAnnotation - instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { - return; - } - break; - default: - break; - } - - throw new SchemaColumnConvertNotSupportedException( - Arrays.toString(descriptor.getPath()), - descriptor.getPrimitiveType().getPrimitiveTypeName().toString(), - sparkType.catalogString()); - } - - private static void validateTimestampType( - LogicalTypeAnnotation logicalTypeAnnotation, DataType sparkType) { - assert (logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation); - // Throw an exception if the Parquet type is TimestampLTZ and the Catalyst type is TimestampNTZ. - // This is to avoid mistakes in reading the timestamp values. - if (((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC() - && sparkType == TimestampNTZType$.MODULE$ - && !isSpark40Plus()) { - convertErrorForTimestampNTZ("int64 time(" + logicalTypeAnnotation + ")"); - } - } - - private static void convertErrorForTimestampNTZ(String parquetType) { - throw new RuntimeException( - "Unable to create Parquet converter for data type " - + TimestampNTZType$.MODULE$.json() - + " whose Parquet type is " - + parquetType); - } - - private static boolean canReadAsIntDecimal(ColumnDescriptor descriptor, DataType dt) { - if (!DecimalType.is32BitDecimalType(dt) && !(isSpark40Plus() && dt instanceof DecimalType)) - return false; - return isDecimalTypeMatched(descriptor, dt); - } - - private static boolean canReadAsLongDecimal(ColumnDescriptor descriptor, DataType dt) { - if (!DecimalType.is64BitDecimalType(dt) && !(isSpark40Plus() && dt instanceof DecimalType)) - return false; - return isDecimalTypeMatched(descriptor, dt); - } - - private static boolean canReadAsBinaryDecimal(ColumnDescriptor descriptor, DataType dt) { - if (!DecimalType.isByteArrayDecimalType(dt)) return false; - return isDecimalTypeMatched(descriptor, dt); - } - - private static boolean isLongDecimal(DataType dt) { - if (dt instanceof DecimalType) { - DecimalType d = (DecimalType) dt; - return d.precision() == 20 && d.scale() == 0; - } - return false; - } - - private static boolean isDecimalTypeMatched(ColumnDescriptor descriptor, DataType dt) { - DecimalType d = (DecimalType) dt; - LogicalTypeAnnotation typeAnnotation = descriptor.getPrimitiveType().getLogicalTypeAnnotation(); - if (typeAnnotation instanceof DecimalLogicalTypeAnnotation) { - DecimalLogicalTypeAnnotation decimalType = (DecimalLogicalTypeAnnotation) typeAnnotation; - // It's OK if the required decimal precision is larger than or equal to the physical decimal - // precision in the Parquet metadata, as long as the decimal scale is the same. - return (decimalType.getPrecision() <= d.precision() && decimalType.getScale() == d.scale()) - || (isSpark40Plus() - && (!SQLConf.get().parquetVectorizedReaderEnabled() - || (decimalType.getScale() <= d.scale() - && decimalType.getPrecision() - decimalType.getScale() - <= d.precision() - d.scale()))); - } else if (isSpark40Plus()) { - boolean isNullTypeAnnotation = typeAnnotation == null; - boolean isIntTypeAnnotation = typeAnnotation instanceof IntLogicalTypeAnnotation; - if (!SQLConf.get().parquetVectorizedReaderEnabled()) { - return isNullTypeAnnotation || isIntTypeAnnotation; - } else if (isNullTypeAnnotation - || (isIntTypeAnnotation && ((IntLogicalTypeAnnotation) typeAnnotation).isSigned())) { - PrimitiveType.PrimitiveTypeName typeName = - descriptor.getPrimitiveType().getPrimitiveTypeName(); - int integerPrecision = d.precision() - d.scale(); - switch (typeName) { - case INT32: - return integerPrecision >= DecimalType$.MODULE$.IntDecimal().precision(); - case INT64: - return integerPrecision >= DecimalType$.MODULE$.LongDecimal().precision(); - } - } - } - return false; - } - - private static boolean isTimestampTypeMatched( - LogicalTypeAnnotation logicalTypeAnnotation, LogicalTypeAnnotation.TimeUnit unit) { - return logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation - && ((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).getUnit() == unit; - } - - private static boolean isUnsignedIntTypeMatched( - LogicalTypeAnnotation logicalTypeAnnotation, int bitWidth) { - return logicalTypeAnnotation instanceof IntLogicalTypeAnnotation - && !((IntLogicalTypeAnnotation) logicalTypeAnnotation).isSigned() - && ((IntLogicalTypeAnnotation) logicalTypeAnnotation).getBitWidth() == bitWidth; - } - - static boolean isSpark40Plus() { - return package$.MODULE$.SPARK_VERSION().compareTo("4.0") >= 0; - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java deleted file mode 100644 index 87845ae760..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ /dev/null @@ -1,525 +0,0 @@ -/* - * 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.parquet; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; -import org.apache.parquet.schema.Types; -import org.apache.spark.sql.types.*; - -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.IcebergApi; - -public class Utils { - - /** This method is called from Apache Iceberg. */ - @IcebergApi - public static ColumnReader getColumnReader( - DataType type, - ParquetColumnSpec columnSpec, - CometSchemaImporter importer, - int batchSize, - boolean useDecimal128, - boolean useLazyMaterialization, - boolean useLegacyTimestamp) { - - ColumnDescriptor descriptor = buildColumnDescriptor(columnSpec); - return getColumnReader( - type, - descriptor, - importer, - batchSize, - useDecimal128, - useLazyMaterialization, - useLegacyTimestamp); - } - - /** - * This method is called from Apache Iceberg. - * - * @see Comet Issue #2079 - */ - @IcebergApi - public static ColumnReader getColumnReader( - DataType type, - ColumnDescriptor descriptor, - CometSchemaImporter importer, - int batchSize, - boolean useDecimal128, - boolean useLazyMaterialization) { - // TODO: support `useLegacyDateTimestamp` for Iceberg - return getColumnReader( - type, descriptor, importer, batchSize, useDecimal128, useLazyMaterialization, true); - } - - public static ColumnReader getColumnReader( - DataType type, - ColumnDescriptor descriptor, - CometSchemaImporter importer, - int batchSize, - boolean useDecimal128, - boolean useLazyMaterialization, - boolean useLegacyDateTimestamp) { - if (useLazyMaterialization && supportLazyMaterialization(type)) { - return new LazyColumnReader( - type, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); - } else { - return new ColumnReader( - type, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); - } - } - - private static boolean supportLazyMaterialization(DataType type) { - return (type instanceof StringType || type instanceof BinaryType); - } - - /** - * Initialize the Comet native Parquet reader. - * - * @param descriptor the Parquet column descriptor for the column to be read - * @param readType the Spark read type used for type promotion. Null if promotion is not enabled. - * @param batchSize the batch size, i.e., maximum number of elements per record batch - * @param useDecimal128 whether to always represent decimals using 128 bits. If false, the native - * reader may represent decimals using 32 or 64 bits, depending on the precision. - * @param useLegacyDateTimestampOrNTZ whether to read dates/timestamps that were written in the - * legacy hybrid Julian + Gregorian calendar as it is. If false, throw exceptions instead. If - * the spark type is TimestampNTZ, this should be true. - */ - public static long initColumnReader( - ColumnDescriptor descriptor, - DataType readType, - int batchSize, - boolean useDecimal128, - boolean useLegacyDateTimestampOrNTZ) { - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - int primitiveTypeId = getPhysicalTypeId(primitiveType.getPrimitiveTypeName()); - LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); - - // Process logical type information - - int bitWidth = -1; - boolean isSigned = false; - if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { - LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = - (LogicalTypeAnnotation.IntLogicalTypeAnnotation) annotation; - bitWidth = intAnnotation.getBitWidth(); - isSigned = intAnnotation.isSigned(); - } - - int precision, scale; - precision = scale = -1; - if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { - LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalAnnotation = - (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) annotation; - precision = decimalAnnotation.getPrecision(); - scale = decimalAnnotation.getScale(); - } - - int tu = -1; - boolean isAdjustedUtc = false; - if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampAnnotation = - (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) annotation; - tu = getTimeUnitId(timestampAnnotation.getUnit()); - isAdjustedUtc = timestampAnnotation.isAdjustedToUTC(); - } - - TypePromotionInfo promotionInfo; - if (readType != null) { - promotionInfo = new TypePromotionInfo(readType); - } else { - // If type promotion is not enable, we'll just use the Parquet primitive type and precision. - promotionInfo = new TypePromotionInfo(primitiveTypeId, precision, scale, bitWidth); - } - - return Native.initColumnReader( - primitiveTypeId, - getLogicalTypeId(annotation), - promotionInfo.physicalTypeId, - descriptor.getPath(), - descriptor.getMaxDefinitionLevel(), - descriptor.getMaxRepetitionLevel(), - bitWidth, - promotionInfo.bitWidth, - isSigned, - primitiveType.getTypeLength(), - precision, - promotionInfo.precision, - scale, - promotionInfo.scale, - tu, - isAdjustedUtc, - batchSize, - useDecimal128, - useLegacyDateTimestampOrNTZ); - } - - static class TypePromotionInfo { - // The Parquet physical type ID converted from the Spark read schema, or the original Parquet - // physical type ID if type promotion is not enabled. - int physicalTypeId; - // Decimal precision from the Spark read schema, or -1 if it's not decimal type. - int precision; - // Decimal scale from the Spark read schema, or -1 if it's not decimal type. - int scale; - // Integer bit width from the Spark read schema, or -1 if it's not integer type. - int bitWidth; - - TypePromotionInfo(int physicalTypeId, int precision, int scale, int bitWidth) { - this.physicalTypeId = physicalTypeId; - this.precision = precision; - this.scale = scale; - this.bitWidth = bitWidth; - } - - TypePromotionInfo(DataType sparkReadType) { - // Create a dummy `StructField` from the input Spark type. We don't care about - // field name, nullability and metadata. - StructField f = new StructField("f", sparkReadType, false, Metadata.empty()); - ColumnDescriptor descriptor = TypeUtil.convertToParquet(f); - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - int physicalTypeId = getPhysicalTypeId(primitiveType.getPrimitiveTypeName()); - LogicalTypeAnnotation annotation = primitiveType.getLogicalTypeAnnotation(); - int precision = -1; - int scale = -1; - int bitWidth = -1; - if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { - LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalAnnotation = - (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) annotation; - precision = decimalAnnotation.getPrecision(); - scale = decimalAnnotation.getScale(); - } - if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { - LogicalTypeAnnotation.IntLogicalTypeAnnotation intAnnotation = - (LogicalTypeAnnotation.IntLogicalTypeAnnotation) annotation; - bitWidth = intAnnotation.getBitWidth(); - } - this.physicalTypeId = physicalTypeId; - this.precision = precision; - this.scale = scale; - this.bitWidth = bitWidth; - } - } - - /** - * Maps the input Parquet physical type 'typeName' to an integer representing it. This is used for - * serialization between the Java and native side. - * - * @param typeName enum for the Parquet physical type - * @return an integer representing the input physical type - */ - static int getPhysicalTypeId(PrimitiveType.PrimitiveTypeName typeName) { - switch (typeName) { - case BOOLEAN: - return 0; - case INT32: - return 1; - case INT64: - return 2; - case INT96: - return 3; - case FLOAT: - return 4; - case DOUBLE: - return 5; - case BINARY: - return 6; - case FIXED_LEN_BYTE_ARRAY: - return 7; - } - throw new IllegalArgumentException("Invalid Parquet physical type: " + typeName); - } - - /** - * Maps the input Parquet logical type 'annotation' to an integer representing it. This is used - * for serialization between the Java and native side. - * - * @param annotation the Parquet logical type annotation - * @return an integer representing the input logical type - */ - static int getLogicalTypeId(LogicalTypeAnnotation annotation) { - if (annotation == null) { - return -1; // No logical type associated - } else if (annotation instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { - return 0; - } else if (annotation instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { - return 1; - } else if (annotation instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { - return 2; - } else if (annotation instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { - return 3; - } else if (annotation instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - return 4; - } else if (annotation instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { - return 5; - } else if (annotation instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { - return 6; - } - - throw new UnsupportedOperationException("Unsupported Parquet logical type " + annotation); - } - - static int getTimeUnitId(LogicalTypeAnnotation.TimeUnit tu) { - switch (tu) { - case MILLIS: - return 0; - case MICROS: - return 1; - case NANOS: - return 2; - default: - throw new UnsupportedOperationException("Unsupported TimeUnit " + tu); - } - } - - @IcebergApi - public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { - PrimitiveType.PrimitiveTypeName primType = - PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); - - Type.Repetition repetition; - if (columnSpec.getMaxRepetitionLevel() > 0) { - repetition = Type.Repetition.REPEATED; - } else if (columnSpec.getMaxDefinitionLevel() > 0) { - repetition = Type.Repetition.OPTIONAL; - } else { - repetition = Type.Repetition.REQUIRED; - } - - String name = columnSpec.getPath()[columnSpec.getPath().length - 1]; - // Reconstruct the logical type from parameters - LogicalTypeAnnotation logicalType = null; - if (columnSpec.getLogicalTypeName() != null) { - logicalType = - reconstructLogicalType( - columnSpec.getLogicalTypeName(), columnSpec.getLogicalTypeParams()); - } - - PrimitiveType primitiveType; - if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { - primitiveType = - Types.primitive(primType, repetition) - .length(columnSpec.getTypeLength()) - .as(logicalType) - .id(columnSpec.getFieldId()) - .named(name); - } else { - primitiveType = - Types.primitive(primType, repetition) - .as(logicalType) - .id(columnSpec.getFieldId()) - .named(name); - } - - return new ColumnDescriptor( - columnSpec.getPath(), - primitiveType, - columnSpec.getMaxRepetitionLevel(), - columnSpec.getMaxDefinitionLevel()); - } - - private static LogicalTypeAnnotation reconstructLogicalType( - String logicalTypeName, java.util.Map params) { - - switch (logicalTypeName) { - // MAP - case "MapLogicalTypeAnnotation": - return LogicalTypeAnnotation.mapType(); - - // LIST - case "ListLogicalTypeAnnotation": - return LogicalTypeAnnotation.listType(); - - // STRING - case "StringLogicalTypeAnnotation": - return LogicalTypeAnnotation.stringType(); - - // MAP_KEY_VALUE - case "MapKeyValueLogicalTypeAnnotation": - return LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(); - - // ENUM - case "EnumLogicalTypeAnnotation": - return LogicalTypeAnnotation.enumType(); - - // DECIMAL - case "DecimalLogicalTypeAnnotation": - if (!params.containsKey("scale") || !params.containsKey("precision")) { - throw new IllegalArgumentException( - "Missing required parameters for DecimalLogicalTypeAnnotation: " + params); - } - int scale = Integer.parseInt(params.get("scale")); - int precision = Integer.parseInt(params.get("precision")); - return LogicalTypeAnnotation.decimalType(scale, precision); - - // DATE - case "DateLogicalTypeAnnotation": - return LogicalTypeAnnotation.dateType(); - - // TIME - case "TimeLogicalTypeAnnotation": - if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { - throw new IllegalArgumentException( - "Missing required parameters for TimeLogicalTypeAnnotation: " + params); - } - - boolean isUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); - String timeUnitStr = params.get("unit"); - - LogicalTypeAnnotation.TimeUnit timeUnit; - switch (timeUnitStr) { - case "MILLIS": - timeUnit = LogicalTypeAnnotation.TimeUnit.MILLIS; - break; - case "MICROS": - timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; - break; - case "NANOS": - timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; - break; - default: - throw new IllegalArgumentException("Unknown time unit: " + timeUnitStr); - } - return LogicalTypeAnnotation.timeType(isUTC, timeUnit); - - // TIMESTAMP - case "TimestampLogicalTypeAnnotation": - if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { - throw new IllegalArgumentException( - "Missing required parameters for TimestampLogicalTypeAnnotation: " + params); - } - boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); - String unitStr = params.get("unit"); - - LogicalTypeAnnotation.TimeUnit unit; - switch (unitStr) { - case "MILLIS": - unit = LogicalTypeAnnotation.TimeUnit.MILLIS; - break; - case "MICROS": - unit = LogicalTypeAnnotation.TimeUnit.MICROS; - break; - case "NANOS": - unit = LogicalTypeAnnotation.TimeUnit.NANOS; - break; - default: - throw new IllegalArgumentException("Unknown timestamp unit: " + unitStr); - } - return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); - - // INTEGER - case "IntLogicalTypeAnnotation": - if (!params.containsKey("isSigned") || !params.containsKey("bitWidth")) { - throw new IllegalArgumentException( - "Missing required parameters for IntLogicalTypeAnnotation: " + params); - } - boolean isSigned = Boolean.parseBoolean(params.get("isSigned")); - int bitWidth = Integer.parseInt(params.get("bitWidth")); - return LogicalTypeAnnotation.intType(bitWidth, isSigned); - - // JSON - case "JsonLogicalTypeAnnotation": - return LogicalTypeAnnotation.jsonType(); - - // BSON - case "BsonLogicalTypeAnnotation": - return LogicalTypeAnnotation.bsonType(); - - // UUID - case "UUIDLogicalTypeAnnotation": - return LogicalTypeAnnotation.uuidType(); - - // INTERVAL - case "IntervalLogicalTypeAnnotation": - return LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(); - - default: - throw new IllegalArgumentException("Unknown logical type: " + logicalTypeName); - } - } - - @IcebergApi - public static ParquetColumnSpec descriptorToParquetColumnSpec(ColumnDescriptor descriptor) { - - String[] path = descriptor.getPath(); - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - String physicalType = primitiveType.getPrimitiveTypeName().name(); - - int typeLength = - primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY - ? primitiveType.getTypeLength() - : 0; - - boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; - - String logicalTypeName = null; - Map logicalTypeParams = new HashMap<>(); - LogicalTypeAnnotation logicalType = primitiveType.getLogicalTypeAnnotation(); - - if (logicalType != null) { - logicalTypeName = logicalType.getClass().getSimpleName(); - - // Handle specific logical types - if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { - LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal = - (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType; - logicalTypeParams.put("precision", String.valueOf(decimal.getPrecision())); - logicalTypeParams.put("scale", String.valueOf(decimal.getScale())); - } else if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp = - (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType; - logicalTypeParams.put("isAdjustedToUTC", String.valueOf(timestamp.isAdjustedToUTC())); - logicalTypeParams.put("unit", timestamp.getUnit().name()); - } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeLogicalTypeAnnotation time = - (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType; - logicalTypeParams.put("isAdjustedToUTC", String.valueOf(time.isAdjustedToUTC())); - logicalTypeParams.put("unit", time.getUnit().name()); - } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { - LogicalTypeAnnotation.IntLogicalTypeAnnotation intType = - (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType; - logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned())); - logicalTypeParams.put("bitWidth", String.valueOf(intType.getBitWidth())); - } - } - - int id = -1; - Type type = descriptor.getPrimitiveType(); - if (type != null && type.getId() != null) { - id = type.getId().intValue(); - } - - return new ParquetColumnSpec( - id, - path, - physicalType, - typeLength, - isRepeated, - descriptor.getMaxDefinitionLevel(), - descriptor.getMaxRepetitionLevel(), - logicalTypeName, - logicalTypeParams); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java b/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java deleted file mode 100644 index 9b5e50ddb4..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/WrappedInputFile.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.io.InputStream; -import java.lang.reflect.Method; - -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.SeekableInputStream; - -import org.apache.comet.IcebergApi; - -/** - * Wraps an Object that possibly implements the methods of a Parquet InputFile (but is not a Parquet - * InputFile). Such an object` exists, for instance, in Iceberg's InputFile - */ -@IcebergApi -public class WrappedInputFile implements InputFile { - Object wrapped; - - @IcebergApi - public WrappedInputFile(Object inputFile) { - this.wrapped = inputFile; - } - - @Override - public long getLength() throws IOException { - try { - Method targetMethod = wrapped.getClass().getDeclaredMethod("getLength"); // - targetMethod.setAccessible(true); - return (long) targetMethod.invoke(wrapped); - } catch (Exception e) { - throw new IOException(e); - } - } - - @Override - public SeekableInputStream newStream() throws IOException { - try { - Method targetMethod = wrapped.getClass().getDeclaredMethod("newStream"); // - targetMethod.setAccessible(true); - InputStream stream = (InputStream) targetMethod.invoke(wrapped); - return new WrappedSeekableInputStream(stream); - } catch (Exception e) { - throw new IOException(e); - } - } - - @Override - public String toString() { - return wrapped.toString(); - } -} diff --git a/common/src/main/java/org/apache/comet/parquet/WrappedSeekableInputStream.java b/common/src/main/java/org/apache/comet/parquet/WrappedSeekableInputStream.java deleted file mode 100644 index c463617bd6..0000000000 --- a/common/src/main/java/org/apache/comet/parquet/WrappedSeekableInputStream.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.parquet; - -import java.io.IOException; -import java.io.InputStream; -import java.lang.reflect.Method; -import java.util.Objects; - -import org.apache.parquet.io.DelegatingSeekableInputStream; - -/** - * Wraps an InputStream that possibly implements the methods of a Parquet SeekableInputStream (but - * is not a Parquet SeekableInputStream). Such an InputStream exists, for instance, in Iceberg's - * SeekableInputStream - */ -public class WrappedSeekableInputStream extends DelegatingSeekableInputStream { - - private final InputStream wrappedInputStream; // The InputStream we are wrapping - - public WrappedSeekableInputStream(InputStream inputStream) { - super(inputStream); - this.wrappedInputStream = Objects.requireNonNull(inputStream, "InputStream cannot be null"); - } - - @Override - public long getPos() throws IOException { - try { - Method targetMethod = wrappedInputStream.getClass().getDeclaredMethod("getPos"); // - targetMethod.setAccessible(true); - return (long) targetMethod.invoke(wrappedInputStream); - } catch (Exception e) { - throw new IOException(e); - } - } - - @Override - public void seek(long newPos) throws IOException { - try { - Method targetMethod = wrappedInputStream.getClass().getDeclaredMethod("seek", long.class); - targetMethod.setAccessible(true); - targetMethod.invoke(wrappedInputStream, newPos); - } catch (Exception e) { - throw new IOException(e); - } - } -} diff --git a/common/src/main/java/org/apache/comet/vector/CometLazyVector.java b/common/src/main/java/org/apache/comet/vector/CometLazyVector.java deleted file mode 100644 index 17b8d7e712..0000000000 --- a/common/src/main/java/org/apache/comet/vector/CometLazyVector.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.vector; - -import org.apache.arrow.vector.ValueVector; -import org.apache.spark.sql.types.DataType; - -import org.apache.comet.parquet.LazyColumnReader; - -public class CometLazyVector extends CometDelegateVector { - private final LazyColumnReader columnReader; - - public CometLazyVector(DataType type, LazyColumnReader columnReader, boolean useDecimal128) { - super(type, useDecimal128); - this.columnReader = columnReader; - } - - public CometDecodedVector getDecodedVector() { - return (CometDecodedVector) delegate; - } - - @Override - public ValueVector getValueVector() { - columnReader.readAllBatch(); - setDelegate(columnReader.loadVector()); - return super.getValueVector(); - } - - @Override - public void setNumNulls(int numNulls) { - throw new UnsupportedOperationException("CometLazyVector doesn't support 'setNumNulls'"); - } - - @Override - public void setNumValues(int numValues) { - throw new UnsupportedOperationException("CometLazyVector doesn't support 'setNumValues'"); - } - - @Override - public void close() { - // Do nothing. 'vector' is closed by 'columnReader' which owns it. - } - - @Override - public boolean hasNull() { - columnReader.readAllBatch(); - setDelegate(columnReader.loadVector()); - return super.hasNull(); - } - - @Override - public int numNulls() { - columnReader.readAllBatch(); - setDelegate(columnReader.loadVector()); - return super.numNulls(); - } - - @Override - public boolean isNullAt(int rowId) { - if (columnReader.materializeUpToIfNecessary(rowId)) { - setDelegate(columnReader.loadVector()); - } - return super.isNullAt(rowId); - } -} diff --git a/common/src/main/resources/log4j2.properties b/common/src/main/resources/log4j2.properties deleted file mode 100644 index 04cdf85330..0000000000 --- a/common/src/main/resources/log4j2.properties +++ /dev/null @@ -1,40 +0,0 @@ -# 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. - -# Set everything to be logged to the file target/unit-tests.log -rootLogger.level = info -rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} - -appender.file.type = File -appender.file.name = File -appender.file.fileName = target/unit-tests.log -appender.file.layout.type = PatternLayout -appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Tests that launch java subprocesses can set the "test.appender" system property to -# "console" to avoid having the child process's logs overwrite the unit test's -# log file. -appender.console.type = Console -appender.console.name = console -appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %t: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -logger.jetty.name = org.sparkproject.jetty -logger.jetty.level = warn - diff --git a/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala b/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala deleted file mode 100644 index 1759ea2765..0000000000 --- a/common/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.parquet - -import java.util.concurrent.{Executors, ExecutorService, ThreadFactory} -import java.util.concurrent.atomic.AtomicLong - -abstract class CometReaderThreadPool { - private var threadPool: Option[ExecutorService] = None - - protected def threadNamePrefix: String - - private def initThreadPool(maxThreads: Int): ExecutorService = synchronized { - if (threadPool.isEmpty) { - val threadFactory: ThreadFactory = new ThreadFactory() { - private val defaultThreadFactory = Executors.defaultThreadFactory - val count = new AtomicLong(0) - - override def newThread(r: Runnable): Thread = { - val thread = defaultThreadFactory.newThread(r) - thread.setName(s"${threadNamePrefix}_${count.getAndIncrement()}") - thread.setDaemon(true) - thread - } - } - - val threadPoolExecutor = Executors.newFixedThreadPool(maxThreads, threadFactory) - threadPool = Some(threadPoolExecutor) - } - - threadPool.get - } - - def getOrCreateThreadPool(numThreads: Int): ExecutorService = { - threadPool.getOrElse(initThreadPool(numThreads)) - } - -} - -// Thread pool used by the Parquet parallel reader -object CometFileReaderThreadPool extends CometReaderThreadPool { - override def threadNamePrefix: String = "file_reader_thread" -} diff --git a/common/src/main/scala/org/apache/comet/udf/CometLambdaRegistry.scala b/common/src/main/scala/org/apache/comet/udf/CometLambdaRegistry.scala deleted file mode 100644 index 5e020ae74a..0000000000 --- a/common/src/main/scala/org/apache/comet/udf/CometLambdaRegistry.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.udf - -import java.util.UUID -import java.util.concurrent.ConcurrentHashMap - -import org.apache.spark.sql.catalyst.expressions.Expression - -/** - * Thread-safe registry bridging plan-time Spark expressions to execution-time UDF lookup. At plan - * time the serde layer registers a lambda expression under a unique key; at execution time the - * UDF retrieves it by that key (passed as a scalar argument). - */ -object CometLambdaRegistry { - - private val registry = new ConcurrentHashMap[String, Expression]() - - def register(expression: Expression): String = { - val key = UUID.randomUUID().toString - registry.put(key, expression) - key - } - - def get(key: String): Expression = { - val expr = registry.get(key) - if (expr == null) { - throw new IllegalStateException( - s"Lambda expression not found in registry for key: $key. " + - "This indicates a lifecycle issue between plan creation and execution.") - } - expr - } - - def remove(key: String): Unit = { - registry.remove(key) - } - - // Visible for testing - def size(): Int = registry.size() -} diff --git a/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java b/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java deleted file mode 100644 index 63bb65d5c8..0000000000 --- a/common/src/test/java/org/apache/comet/parquet/TestCometInputFile.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.parquet; - -import org.junit.Assert; -import org.junit.Test; - -public class TestCometInputFile { - @Test - public void testIsAtLeastHadoop33() { - Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.3.0")); - Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.4.0-SNAPSHOT")); - Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.12.5")); - Assert.assertTrue(CometInputFile.isAtLeastHadoop33("3.20.6.4-xyz")); - - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7.2")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7.3-SNAPSHOT")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2.7")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("2")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.2")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.0.2.5-abc")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.1.2-test")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3-SNAPSHOT")); - Assert.assertFalse(CometInputFile.isAtLeastHadoop33("3.2-SNAPSHOT")); - } -} diff --git a/common/src/test/java/org/apache/comet/parquet/TestFileReader.java b/common/src/test/java/org/apache/comet/parquet/TestFileReader.java deleted file mode 100644 index d380fc16a9..0000000000 --- a/common/src/test/java/org/apache/comet/parquet/TestFileReader.java +++ /dev/null @@ -1,835 +0,0 @@ -/* - * 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.parquet; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.lang.reflect.Method; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.*; - -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.bytes.BytesInput; -import org.apache.parquet.bytes.BytesUtils; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.Encoding; -import org.apache.parquet.column.page.DataPage; -import org.apache.parquet.column.page.DataPageV1; -import org.apache.parquet.column.page.DataPageV2; -import org.apache.parquet.column.page.DictionaryPage; -import org.apache.parquet.column.page.PageReadStore; -import org.apache.parquet.column.page.PageReader; -import org.apache.parquet.column.statistics.BinaryStatistics; -import org.apache.parquet.column.statistics.Statistics; -import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; -import org.apache.parquet.column.values.bloomfilter.BloomFilter; -import org.apache.parquet.filter2.predicate.FilterApi; -import org.apache.parquet.filter2.predicate.FilterPredicate; -import org.apache.parquet.filter2.predicate.Operators; -import org.apache.parquet.hadoop.ParquetFileWriter; -import org.apache.parquet.hadoop.ParquetInputFormat; -import org.apache.parquet.hadoop.metadata.*; -import org.apache.parquet.hadoop.util.HadoopInputFile; -import org.apache.parquet.internal.column.columnindex.BoundaryOrder; -import org.apache.parquet.internal.column.columnindex.ColumnIndex; -import org.apache.parquet.internal.column.columnindex.OffsetIndex; -import org.apache.parquet.io.InputFile; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.MessageTypeParser; -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; -import org.apache.parquet.schema.Types; - -import org.apache.comet.CometConf; - -import static org.apache.parquet.column.Encoding.*; -import static org.apache.parquet.format.converter.ParquetMetadataConverter.MAX_STATS_SIZE; -import static org.junit.Assert.*; -import static org.junit.Assert.assertEquals; - -import static org.apache.comet.parquet.TypeUtil.isSpark40Plus; - -@SuppressWarnings("deprecation") -public class TestFileReader { - private static final MessageType SCHEMA = - MessageTypeParser.parseMessageType( - "" - + "message m {" - + " required group a {" - + " required binary b;" - + " }" - + " required group c {" - + " required int64 d;" - + " }" - + "}"); - - private static final MessageType SCHEMA2 = - MessageTypeParser.parseMessageType( - "" - + "message root { " - + "required int32 id;" - + "required binary name(UTF8); " - + "required int32 num; " - + "required binary comment(UTF8);" - + "}"); - - private static final MessageType PROJECTED_SCHEMA2 = - MessageTypeParser.parseMessageType( - "" - + "message root { " - + "required int32 id;" - + "required binary name(UTF8); " - + "required binary comment(UTF8);" - + "}"); - - private static final String[] PATH1 = {"a", "b"}; - private static final ColumnDescriptor C1 = SCHEMA.getColumnDescription(PATH1); - private static final String[] PATH2 = {"c", "d"}; - private static final ColumnDescriptor C2 = SCHEMA.getColumnDescription(PATH2); - - private static final byte[] BYTES1 = {0, 1, 2, 3}; - private static final byte[] BYTES2 = {1, 2, 3, 4}; - private static final byte[] BYTES3 = {2, 3, 4, 5}; - private static final byte[] BYTES4 = {3, 4, 5, 6}; - private static final CompressionCodecName CODEC = CompressionCodecName.UNCOMPRESSED; - - private static final org.apache.parquet.column.statistics.Statistics EMPTY_STATS = - org.apache.parquet.column.statistics.Statistics.getBuilderForReading( - Types.required(PrimitiveTypeName.BINARY).named("test_binary")) - .build(); - - @Rule public final TemporaryFolder temp = new TemporaryFolder(); - - @Test - public void testEnableReadParallel() { - Configuration configuration = new Configuration(); - ReadOptions options = ReadOptions.builder(configuration).build(); - - assertFalse(FileReader.shouldReadParallel(options, "hdfs")); - assertFalse(FileReader.shouldReadParallel(options, "file")); - assertFalse(FileReader.shouldReadParallel(options, null)); - assertTrue(FileReader.shouldReadParallel(options, "s3a")); - - options = ReadOptions.builder(configuration).enableParallelIO(false).build(); - assertFalse(FileReader.shouldReadParallel(options, "s3a")); - } - - @Test - public void testReadWrite() throws Exception { - File testFile = temp.newFile(); - testFile.delete(); - - Path path = new Path(testFile.toURI()); - Configuration configuration = new Configuration(); - - // Start a Parquet file with 2 row groups, each with 2 column chunks - ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); - w.start(); - w.startBlock(3); - w.startColumn(C1, 5, CODEC); - long c1Starts = w.getPos(); - long c1p1Starts = w.getPos(); - w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); - w.writeDataPage(3, 4, BytesInput.from(BYTES1), EMPTY_STATS, 3, RLE, RLE, PLAIN); - w.endColumn(); - long c1Ends = w.getPos(); - w.startColumn(C2, 6, CODEC); - long c2Starts = w.getPos(); - w.writeDictionaryPage(new DictionaryPage(BytesInput.from(BYTES2), 4, RLE_DICTIONARY)); - long c2p1Starts = w.getPos(); - w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); - w.writeDataPage(3, 4, BytesInput.from(BYTES2), EMPTY_STATS, 3, RLE, RLE, PLAIN); - w.writeDataPage(1, 4, BytesInput.from(BYTES2), EMPTY_STATS, 1, RLE, RLE, PLAIN); - w.endColumn(); - long c2Ends = w.getPos(); - w.endBlock(); - w.startBlock(4); - w.startColumn(C1, 7, CODEC); - w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, 7, RLE, RLE, PLAIN); - w.endColumn(); - w.startColumn(C2, 8, CODEC); - w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, 8, RLE, RLE, PLAIN); - w.endColumn(); - w.endBlock(); - w.end(new HashMap<>()); - - InputFile file = HadoopInputFile.fromPath(path, configuration); - ParquetReadOptions options = ParquetReadOptions.builder().build(); - ReadOptions cometOptions = ReadOptions.builder(configuration).build(); - - try (FileReader reader = new FileReader(file, options, cometOptions)) { - ParquetMetadata readFooter = reader.getFooter(); - assertEquals("footer: " + readFooter, 2, readFooter.getBlocks().size()); - BlockMetaData rowGroup = readFooter.getBlocks().get(0); - assertEquals(c1Ends - c1Starts, rowGroup.getColumns().get(0).getTotalSize()); - assertEquals(c2Ends - c2Starts, rowGroup.getColumns().get(1).getTotalSize()); - assertEquals(c2Ends - c1Starts, rowGroup.getTotalByteSize()); - - assertEquals(c1Starts, rowGroup.getColumns().get(0).getStartingPos()); - assertEquals(0, rowGroup.getColumns().get(0).getDictionaryPageOffset()); - assertEquals(c1p1Starts, rowGroup.getColumns().get(0).getFirstDataPageOffset()); - assertEquals(c2Starts, rowGroup.getColumns().get(1).getStartingPos()); - assertEquals(c2Starts, rowGroup.getColumns().get(1).getDictionaryPageOffset()); - assertEquals(c2p1Starts, rowGroup.getColumns().get(1).getFirstDataPageOffset()); - - HashSet expectedEncoding = new HashSet<>(); - expectedEncoding.add(PLAIN); - expectedEncoding.add(RLE); - assertEquals(expectedEncoding, rowGroup.getColumns().get(0).getEncodings()); - } - - // read first block of col #1 - try (FileReader r = new FileReader(file, options, cometOptions)) { - r.setRequestedSchema(Arrays.asList(SCHEMA.getColumnDescription(PATH1))); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(pages, PATH1, 3, BytesInput.from(BYTES1)); - assertTrue(r.skipNextRowGroup()); - assertNull(r.readNextRowGroup()); - } - - // read all blocks of col #1 and #2 - try (FileReader r = new FileReader(file, options, cometOptions)) { - r.setRequestedSchema( - Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(3, pages.getRowCount()); - validateContains(pages, PATH1, 2, BytesInput.from(BYTES1)); - validateContains(pages, PATH1, 3, BytesInput.from(BYTES1)); - validateContains(pages, PATH2, 2, BytesInput.from(BYTES2)); - validateContains(pages, PATH2, 3, BytesInput.from(BYTES2)); - validateContains(pages, PATH2, 1, BytesInput.from(BYTES2)); - - pages = r.readNextRowGroup(); - assertEquals(4, pages.getRowCount()); - - validateContains(pages, PATH1, 7, BytesInput.from(BYTES3)); - validateContains(pages, PATH2, 8, BytesInput.from(BYTES4)); - - assertNull(r.readNextRowGroup()); - } - } - - @Test - public void testBloomFilterReadWrite() throws Exception { - MessageType schema = - MessageTypeParser.parseMessageType("message test { required binary foo; }"); - File testFile = temp.newFile(); - testFile.delete(); - Path path = new Path(testFile.toURI()); - Configuration configuration = new Configuration(); - configuration.set("parquet.bloom.filter.column.names", "foo"); - String[] colPath = {"foo"}; - - ColumnDescriptor col = schema.getColumnDescription(colPath); - BinaryStatistics stats1 = new BinaryStatistics(); - ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); - w.start(); - w.startBlock(3); - w.startColumn(col, 5, CODEC); - w.writeDataPage(2, 4, BytesInput.from(BYTES1), stats1, 2, RLE, RLE, PLAIN); - w.writeDataPage(3, 4, BytesInput.from(BYTES1), stats1, 2, RLE, RLE, PLAIN); - w.endColumn(); - BloomFilter blockSplitBloomFilter = new BlockSplitBloomFilter(0); - blockSplitBloomFilter.insertHash(blockSplitBloomFilter.hash(Binary.fromString("hello"))); - blockSplitBloomFilter.insertHash(blockSplitBloomFilter.hash(Binary.fromString("world"))); - addBloomFilter(w, "foo", blockSplitBloomFilter); - w.endBlock(); - w.end(new HashMap<>()); - - InputFile file = HadoopInputFile.fromPath(path, configuration); - ParquetReadOptions options = ParquetReadOptions.builder().build(); - ReadOptions cometOptions = ReadOptions.builder(configuration).build(); - - try (FileReader r = new FileReader(file, options, cometOptions)) { - ParquetMetadata footer = r.getFooter(); - r.setRequestedSchema(Arrays.asList(schema.getColumnDescription(colPath))); - BloomFilterReader bloomFilterReader = - new BloomFilterReader( - footer.getBlocks().get(0), - r.getFileMetaData().getFileDecryptor(), - r.getInputStream()); - BloomFilter bloomFilter = - bloomFilterReader.readBloomFilter(footer.getBlocks().get(0).getColumns().get(0)); - assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("hello")))); - assertTrue(bloomFilter.findHash(blockSplitBloomFilter.hash(Binary.fromString("world")))); - } - } - - @Test - public void testReadWriteDataPageV2() throws Exception { - File testFile = temp.newFile(); - testFile.delete(); - - Path path = new Path(testFile.toURI()); - Configuration configuration = new Configuration(); - - ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); - w.start(); - w.startBlock(14); - - BytesInput repLevels = BytesInput.fromInt(2); - BytesInput defLevels = BytesInput.fromInt(1); - BytesInput data = BytesInput.fromInt(3); - BytesInput data2 = BytesInput.fromInt(10); - - org.apache.parquet.column.statistics.Statistics statsC1P1 = createStatistics("s", "z", C1); - org.apache.parquet.column.statistics.Statistics statsC1P2 = createStatistics("b", "d", C1); - - w.startColumn(C1, 6, CODEC); - long c1Starts = w.getPos(); - w.writeDataPageV2(4, 1, 3, repLevels, defLevels, PLAIN, data, 4, statsC1P1); - w.writeDataPageV2(3, 0, 3, repLevels, defLevels, PLAIN, data, 4, statsC1P2); - w.endColumn(); - long c1Ends = w.getPos(); - - w.startColumn(C2, 5, CODEC); - long c2Starts = w.getPos(); - w.writeDataPageV2(5, 2, 3, repLevels, defLevels, PLAIN, data2, 4, EMPTY_STATS); - w.writeDataPageV2(2, 0, 2, repLevels, defLevels, PLAIN, data2, 4, EMPTY_STATS); - w.endColumn(); - long c2Ends = w.getPos(); - - w.endBlock(); - w.end(new HashMap<>()); - - InputFile file = HadoopInputFile.fromPath(path, configuration); - ParquetReadOptions options = ParquetReadOptions.builder().build(); - ReadOptions cometOptions = ReadOptions.builder(configuration).build(); - - try (FileReader reader = new FileReader(file, options, cometOptions)) { - ParquetMetadata footer = reader.getFooter(); - assertEquals("footer: " + footer, 1, footer.getBlocks().size()); - assertEquals(c1Ends - c1Starts, footer.getBlocks().get(0).getColumns().get(0).getTotalSize()); - assertEquals(c2Ends - c2Starts, footer.getBlocks().get(0).getColumns().get(1).getTotalSize()); - assertEquals(c2Ends - c1Starts, footer.getBlocks().get(0).getTotalByteSize()); - - // check for stats - org.apache.parquet.column.statistics.Statistics expectedStats = - createStatistics("b", "z", C1); - assertStatsValuesEqual( - expectedStats, footer.getBlocks().get(0).getColumns().get(0).getStatistics()); - - HashSet expectedEncoding = new HashSet<>(); - expectedEncoding.add(PLAIN); - assertEquals(expectedEncoding, footer.getBlocks().get(0).getColumns().get(0).getEncodings()); - } - - try (FileReader r = new FileReader(file, options, cometOptions)) { - r.setRequestedSchema( - Arrays.asList(SCHEMA.getColumnDescription(PATH1), SCHEMA.getColumnDescription(PATH2))); - PageReadStore pages = r.readNextRowGroup(); - assertEquals(14, pages.getRowCount()); - validateV2Page( - pages, - PATH1, - 3, - 4, - 1, - repLevels.toByteArray(), - defLevels.toByteArray(), - data.toByteArray(), - 12); - validateV2Page( - pages, - PATH1, - 3, - 3, - 0, - repLevels.toByteArray(), - defLevels.toByteArray(), - data.toByteArray(), - 12); - validateV2Page( - pages, - PATH2, - 3, - 5, - 2, - repLevels.toByteArray(), - defLevels.toByteArray(), - data2.toByteArray(), - 12); - validateV2Page( - pages, - PATH2, - 2, - 2, - 0, - repLevels.toByteArray(), - defLevels.toByteArray(), - data2.toByteArray(), - 12); - assertNull(r.readNextRowGroup()); - } - } - - @Test - public void testColumnIndexFilter() throws Exception { - File testFile = temp.newFile(); - testFile.delete(); - - Path path = new Path(testFile.toURI()); - Configuration configuration = new Configuration(); - - ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); - - w.start(); - w.startBlock(4); - w.startColumn(C1, 7, CODEC); - w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); - w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); - w.endColumn(); - w.startColumn(C2, 8, CODEC); - // the first page contains one matching record - w.writeDataPage(1, 4, BytesInput.from(BYTES3), statsC2(2L), 1, RLE, RLE, PLAIN); - // all the records of the second page are larger than 2, so should be filtered out - w.writeDataPage(3, 4, BytesInput.from(BYTES4), statsC2(3L, 4L, 5L), 3, RLE, RLE, PLAIN); - w.endColumn(); - w.endBlock(); - - w.startBlock(4); - w.startColumn(C1, 7, CODEC); - w.writeDataPage(2, 4, BytesInput.from(BYTES1), EMPTY_STATS, 2, RLE, RLE, PLAIN); - w.writeDataPage(2, 4, BytesInput.from(BYTES2), EMPTY_STATS, 2, RLE, RLE, PLAIN); - w.endColumn(); - w.startColumn(C2, 8, CODEC); - // the first page should be filtered out - w.writeDataPage(1, 4, BytesInput.from(BYTES3), statsC2(4L), 1, RLE, RLE, PLAIN); - // the second page will be read since it contains matching record - w.writeDataPage(3, 4, BytesInput.from(BYTES4), statsC2(0L, 1L, 3L), 3, RLE, RLE, PLAIN); - w.endColumn(); - w.endBlock(); - - w.end(new HashMap<>()); - - // set a simple equality filter in the ParquetInputFormat - Operators.LongColumn c2 = FilterApi.longColumn("c.d"); - FilterPredicate p = FilterApi.eq(c2, 2L); - ParquetInputFormat.setFilterPredicate(configuration, p); - InputFile file = HadoopInputFile.fromPath(path, configuration); - ParquetReadOptions options = HadoopReadOptions.builder(configuration).build(); - ReadOptions cometOptions = ReadOptions.builder(configuration).build(); - - try (FileReader r = new FileReader(file, options, cometOptions)) { - assertEquals(4, r.getFilteredRecordCount()); - PageReadStore readStore = r.readNextFilteredRowGroup(); - - PageReader c1Reader = readStore.getPageReader(C1); - List c1Pages = new ArrayList<>(); - DataPage page; - while ((page = c1Reader.readPage()) != null) { - c1Pages.add(page); - } - // second page of c1 should be filtered out - assertEquals(1, c1Pages.size()); - validatePage(c1Pages.get(0), 2, BytesInput.from(BYTES1)); - - PageReader c2Reader = readStore.getPageReader(C2); - List c2Pages = new ArrayList<>(); - while ((page = c2Reader.readPage()) != null) { - c2Pages.add(page); - } - assertEquals(1, c2Pages.size()); - validatePage(c2Pages.get(0), 1, BytesInput.from(BYTES3)); - - // test the second row group - readStore = r.readNextFilteredRowGroup(); - assertNotNull(readStore); - - c1Reader = readStore.getPageReader(C1); - c1Pages.clear(); - while ((page = c1Reader.readPage()) != null) { - c1Pages.add(page); - } - // all pages of c1 should be retained - assertEquals(2, c1Pages.size()); - validatePage(c1Pages.get(0), 2, BytesInput.from(BYTES1)); - validatePage(c1Pages.get(1), 2, BytesInput.from(BYTES2)); - - c2Reader = readStore.getPageReader(C2); - c2Pages.clear(); - while ((page = c2Reader.readPage()) != null) { - c2Pages.add(page); - } - assertEquals(1, c2Pages.size()); - validatePage(c2Pages.get(0), 3, BytesInput.from(BYTES4)); - } - } - - @Test - public void testColumnIndexReadWrite() throws Exception { - File testFile = temp.newFile(); - testFile.delete(); - - Path path = new Path(testFile.toURI()); - Configuration configuration = new Configuration(); - - ParquetFileWriter w = new ParquetFileWriter(configuration, SCHEMA, path); - w.start(); - w.startBlock(4); - w.startColumn(C1, 7, CODEC); - w.writeDataPage(7, 4, BytesInput.from(BYTES3), EMPTY_STATS, RLE, RLE, PLAIN); - w.endColumn(); - w.startColumn(C2, 8, CODEC); - w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, RLE, RLE, PLAIN); - w.endColumn(); - w.endBlock(); - w.startBlock(4); - w.startColumn(C1, 5, CODEC); - long c1p1Starts = w.getPos(); - w.writeDataPage( - 2, 4, BytesInput.from(BYTES1), statsC1(null, Binary.fromString("aaa")), 1, RLE, RLE, PLAIN); - long c1p2Starts = w.getPos(); - w.writeDataPage( - 3, - 4, - BytesInput.from(BYTES1), - statsC1(Binary.fromString("bbb"), Binary.fromString("ccc")), - 3, - RLE, - RLE, - PLAIN); - w.endColumn(); - long c1Ends = w.getPos(); - w.startColumn(C2, 6, CODEC); - long c2p1Starts = w.getPos(); - w.writeDataPage(2, 4, BytesInput.from(BYTES2), statsC2(117L, 100L), 1, RLE, RLE, PLAIN); - long c2p2Starts = w.getPos(); - w.writeDataPage(3, 4, BytesInput.from(BYTES2), statsC2(null, null, null), 2, RLE, RLE, PLAIN); - long c2p3Starts = w.getPos(); - w.writeDataPage(1, 4, BytesInput.from(BYTES2), statsC2(0L), 1, RLE, RLE, PLAIN); - w.endColumn(); - long c2Ends = w.getPos(); - w.endBlock(); - w.startBlock(4); - w.startColumn(C1, 7, CODEC); - w.writeDataPage( - 7, - 4, - BytesInput.from(BYTES3), - // Creating huge stats so the column index will reach the limit and won't be written - statsC1( - Binary.fromConstantByteArray(new byte[(int) MAX_STATS_SIZE]), - Binary.fromConstantByteArray(new byte[1])), - 4, - RLE, - RLE, - PLAIN); - w.endColumn(); - w.startColumn(C2, 8, CODEC); - w.writeDataPage(8, 4, BytesInput.from(BYTES4), EMPTY_STATS, RLE, RLE, PLAIN); - w.endColumn(); - w.endBlock(); - w.end(new HashMap<>()); - - InputFile file = HadoopInputFile.fromPath(path, configuration); - ParquetReadOptions options = ParquetReadOptions.builder().build(); - ReadOptions cometOptions = ReadOptions.builder(configuration).build(); - - try (FileReader reader = new FileReader(file, options, cometOptions)) { - ParquetMetadata footer = reader.getFooter(); - assertEquals(3, footer.getBlocks().size()); - BlockMetaData blockMeta = footer.getBlocks().get(1); - assertEquals(2, blockMeta.getColumns().size()); - - ColumnIndexReader indexReader = reader.getColumnIndexReader(1); - ColumnIndex columnIndex = indexReader.readColumnIndex(blockMeta.getColumns().get(0)); - assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder()); - assertEquals(Arrays.asList(1L, 0L), columnIndex.getNullCounts()); - assertEquals(Arrays.asList(false, false), columnIndex.getNullPages()); - List minValues = columnIndex.getMinValues(); - assertEquals(2, minValues.size()); - List maxValues = columnIndex.getMaxValues(); - assertEquals(2, maxValues.size()); - assertEquals("aaa", new String(minValues.get(0).array(), StandardCharsets.UTF_8)); - assertEquals("aaa", new String(maxValues.get(0).array(), StandardCharsets.UTF_8)); - assertEquals("bbb", new String(minValues.get(1).array(), StandardCharsets.UTF_8)); - assertEquals("ccc", new String(maxValues.get(1).array(), StandardCharsets.UTF_8)); - - columnIndex = indexReader.readColumnIndex(blockMeta.getColumns().get(1)); - assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder()); - assertEquals(Arrays.asList(0L, 3L, 0L), columnIndex.getNullCounts()); - assertEquals(Arrays.asList(false, true, false), columnIndex.getNullPages()); - minValues = columnIndex.getMinValues(); - assertEquals(3, minValues.size()); - maxValues = columnIndex.getMaxValues(); - assertEquals(3, maxValues.size()); - assertEquals(100, BytesUtils.bytesToLong(minValues.get(0).array())); - assertEquals(117, BytesUtils.bytesToLong(maxValues.get(0).array())); - assertEquals(0, minValues.get(1).array().length); - assertEquals(0, maxValues.get(1).array().length); - assertEquals(0, BytesUtils.bytesToLong(minValues.get(2).array())); - assertEquals(0, BytesUtils.bytesToLong(maxValues.get(2).array())); - - OffsetIndex offsetIndex = indexReader.readOffsetIndex(blockMeta.getColumns().get(0)); - assertEquals(2, offsetIndex.getPageCount()); - assertEquals(c1p1Starts, offsetIndex.getOffset(0)); - assertEquals(c1p2Starts, offsetIndex.getOffset(1)); - assertEquals(c1p2Starts - c1p1Starts, offsetIndex.getCompressedPageSize(0)); - assertEquals(c1Ends - c1p2Starts, offsetIndex.getCompressedPageSize(1)); - assertEquals(0, offsetIndex.getFirstRowIndex(0)); - assertEquals(1, offsetIndex.getFirstRowIndex(1)); - - offsetIndex = indexReader.readOffsetIndex(blockMeta.getColumns().get(1)); - assertEquals(3, offsetIndex.getPageCount()); - assertEquals(c2p1Starts, offsetIndex.getOffset(0)); - assertEquals(c2p2Starts, offsetIndex.getOffset(1)); - assertEquals(c2p3Starts, offsetIndex.getOffset(2)); - assertEquals(c2p2Starts - c2p1Starts, offsetIndex.getCompressedPageSize(0)); - assertEquals(c2p3Starts - c2p2Starts, offsetIndex.getCompressedPageSize(1)); - assertEquals(c2Ends - c2p3Starts, offsetIndex.getCompressedPageSize(2)); - assertEquals(0, offsetIndex.getFirstRowIndex(0)); - assertEquals(1, offsetIndex.getFirstRowIndex(1)); - assertEquals(3, offsetIndex.getFirstRowIndex(2)); - - if (!isSpark40Plus()) { // TODO: https://github.com/apache/datafusion-comet/issues/1948 - assertNull(indexReader.readColumnIndex(footer.getBlocks().get(2).getColumns().get(0))); - } - } - } - - // Test reader with merging of scan ranges enabled - @Test - public void testWriteReadMergeScanRange() throws Throwable { - Configuration conf = new Configuration(); - conf.set(CometConf.COMET_IO_MERGE_RANGES().key(), Boolean.toString(true)); - // Set the merge range delta so small that ranges do not get merged - conf.set(CometConf.COMET_IO_MERGE_RANGES_DELTA().key(), Integer.toString(1024)); - testReadWrite(conf, 2, 1024); - // Set the merge range delta so large that all ranges get merged - conf.set(CometConf.COMET_IO_MERGE_RANGES_DELTA().key(), Integer.toString(1024 * 1024)); - testReadWrite(conf, 2, 1024); - } - - // `addBloomFilter` is package-private in Parquet, so this uses reflection to access it - private void addBloomFilter(ParquetFileWriter w, String s, BloomFilter filter) throws Exception { - Method method = - ParquetFileWriter.class.getDeclaredMethod( - "addBloomFilter", String.class, BloomFilter.class); - method.setAccessible(true); - method.invoke(w, s, filter); - } - - private void validateContains(PageReadStore pages, String[] path, int values, BytesInput bytes) - throws IOException { - PageReader pageReader = pages.getPageReader(SCHEMA.getColumnDescription(path)); - DataPage page = pageReader.readPage(); - validatePage(page, values, bytes); - } - - private void validatePage(DataPage page, int values, BytesInput bytes) throws IOException { - assertEquals(values, page.getValueCount()); - assertArrayEquals(bytes.toByteArray(), ((DataPageV1) page).getBytes().toByteArray()); - } - - private void validateV2Page( - PageReadStore pages, - String[] path, - int values, - int rows, - int nullCount, - byte[] repetition, - byte[] definition, - byte[] data, - int uncompressedSize) - throws IOException { - PageReader pageReader = pages.getPageReader(SCHEMA.getColumnDescription(path)); - DataPageV2 page = (DataPageV2) pageReader.readPage(); - assertEquals(values, page.getValueCount()); - assertEquals(rows, page.getRowCount()); - assertEquals(nullCount, page.getNullCount()); - assertEquals(uncompressedSize, page.getUncompressedSize()); - assertArrayEquals(repetition, page.getRepetitionLevels().toByteArray()); - assertArrayEquals(definition, page.getDefinitionLevels().toByteArray()); - assertArrayEquals(data, page.getData().toByteArray()); - } - - private Statistics createStatistics(String min, String max, ColumnDescriptor col) { - return Statistics.getBuilderForReading(col.getPrimitiveType()) - .withMin(Binary.fromString(min).getBytes()) - .withMax(Binary.fromString(max).getBytes()) - .withNumNulls(0) - .build(); - } - - public static void assertStatsValuesEqual(Statistics expected, Statistics actual) { - if (expected == actual) { - return; - } - if (expected == null || actual == null) { - assertEquals(expected, actual); - } - Assert.assertArrayEquals(expected.getMaxBytes(), actual.getMaxBytes()); - Assert.assertArrayEquals(expected.getMinBytes(), actual.getMinBytes()); - Assert.assertEquals(expected.getNumNulls(), actual.getNumNulls()); - } - - private Statistics statsC1(Binary... values) { - Statistics stats = Statistics.createStats(C1.getPrimitiveType()); - for (Binary value : values) { - if (value == null) { - stats.incrementNumNulls(); - } else { - stats.updateStats(value); - } - } - return stats; - } - - /** - * Generates arbitrary data for simple schemas, writes the data to a file and also returns the - * data. - * - * @return array of data pages for each column - */ - private HashMap generateAndWriteData( - Configuration configuration, - Path path, - MessageType schema, - int numPages, - int numRecordsPerPage) - throws IOException { - - HashMap dataPages = new HashMap<>(); - - Generator generator = new Generator(); - ParquetFileWriter writer = new ParquetFileWriter(configuration, schema, path); - writer.start(); - writer.startBlock((long) numPages * numRecordsPerPage); - for (ColumnDescriptor colDesc : schema.getColumns()) { - writer.startColumn(colDesc, (long) numPages * numRecordsPerPage, CODEC); - String type = colDesc.getPrimitiveType().getName(); - byte[][] allPages = new byte[numPages][]; - byte[] data; - for (int i = 0; i < numPages; i++) { - data = generator.generateValues(numRecordsPerPage, type); - writer.writeDataPage( - numRecordsPerPage, - data.length, - BytesInput.from(data), - EMPTY_STATS, - numRecordsPerPage, - RLE, - RLE, - PLAIN); - allPages[i] = data; - } - dataPages.put(String.join(".", colDesc.getPath()), allPages); - writer.endColumn(); - } - writer.endBlock(); - writer.end(new HashMap<>()); - return dataPages; - } - - private void readAndValidatePageData( - InputFile inputFile, - ParquetReadOptions options, - ReadOptions cometOptions, - MessageType schema, - HashMap expected, - int expectedValuesPerPage) - throws IOException { - try (FileReader fileReader = new FileReader(inputFile, options, cometOptions)) { - fileReader.setRequestedSchema(schema.getColumns()); - PageReadStore pages = fileReader.readNextRowGroup(); - for (ColumnDescriptor colDesc : schema.getColumns()) { - byte[][] allExpectedPages = expected.get(String.join(".", colDesc.getPath())); - PageReader pageReader = pages.getPageReader(colDesc); - for (byte[] expectedPage : allExpectedPages) { - DataPage page = pageReader.readPage(); - validatePage(page, expectedValuesPerPage, BytesInput.from(expectedPage)); - } - } - } - } - - public void testReadWrite(Configuration configuration, int numPages, int numRecordsPerPage) - throws Exception { - File testFile = temp.newFile(); - testFile.delete(); - - Path path = new Path(testFile.toURI()); - HashMap dataPages = - generateAndWriteData(configuration, path, SCHEMA2, numPages, numRecordsPerPage); - InputFile file = HadoopInputFile.fromPath(path, configuration); - ParquetReadOptions options = ParquetReadOptions.builder().build(); - ReadOptions cometOptions = ReadOptions.builder(configuration).build(); - - readAndValidatePageData( - file, options, cometOptions, PROJECTED_SCHEMA2, dataPages, numRecordsPerPage); - } - - static class Generator { - - static Random random = new Random(1729); - private static final String ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz -"; - private static final int STR_MIN_SIZE = 5; - private static final int STR_MAX_SIZE = 30; - - private byte[] getString(int minSize, int maxSize) { - int size = random.nextInt(maxSize - minSize) + minSize; - byte[] str = new byte[size]; - for (int i = 0; i < size; ++i) { - str[i] = (byte) ALPHABET.charAt(random.nextInt(ALPHABET.length())); - } - return str; - } - - private byte[] generateValues(int numValues, String type) throws IOException { - - if (type.equals("int32")) { - byte[] data = new byte[4 * numValues]; - random.nextBytes(data); - return data; - } else { - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - for (int i = 0; i < numValues; i++) { - outputStream.write(getString(STR_MIN_SIZE, STR_MAX_SIZE)); - } - return outputStream.toByteArray(); - } - } - } - - private Statistics statsC2(Long... values) { - Statistics stats = Statistics.createStats(C2.getPrimitiveType()); - for (Long value : values) { - if (value == null) { - stats.incrementNumNulls(); - } else { - stats.updateStats(value); - } - } - return stats; - } -} diff --git a/common/src/test/java/org/apache/comet/parquet/TestUtils.java b/common/src/test/java/org/apache/comet/parquet/TestUtils.java deleted file mode 100644 index 6daa9a254b..0000000000 --- a/common/src/test/java/org/apache/comet/parquet/TestUtils.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * 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.parquet; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import org.junit.Test; - -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.PrimitiveType; - -import static org.junit.Assert.*; - -public class TestUtils { - - @Test - public void testBuildColumnDescriptorWithTimestamp() { - Map params = new HashMap<>(); - params.put("isAdjustedToUTC", "true"); - params.put("unit", "MICROS"); - - ParquetColumnSpec spec = - new ParquetColumnSpec( - 10, - new String[] {"event_time"}, - "INT64", - 0, - false, - 0, - 0, - "TimestampLogicalTypeAnnotation", - params); - - ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); - assertNotNull(descriptor); - - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - assertEquals(PrimitiveType.PrimitiveTypeName.INT64, primitiveType.getPrimitiveTypeName()); - assertTrue( - primitiveType.getLogicalTypeAnnotation() - instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation); - - LogicalTypeAnnotation.TimestampLogicalTypeAnnotation ts = - (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) - primitiveType.getLogicalTypeAnnotation(); - assertTrue(ts.isAdjustedToUTC()); - assertEquals(LogicalTypeAnnotation.TimeUnit.MICROS, ts.getUnit()); - } - - @Test - public void testBuildColumnDescriptorWithDecimal() { - Map params = new HashMap<>(); - params.put("precision", "10"); - params.put("scale", "2"); - - ParquetColumnSpec spec = - new ParquetColumnSpec( - 11, - new String[] {"price"}, - "FIXED_LEN_BYTE_ARRAY", - 5, - false, - 0, - 0, - "DecimalLogicalTypeAnnotation", - params); - - ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - assertEquals( - PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, primitiveType.getPrimitiveTypeName()); - - LogicalTypeAnnotation.DecimalLogicalTypeAnnotation dec = - (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) - primitiveType.getLogicalTypeAnnotation(); - assertEquals(10, dec.getPrecision()); - assertEquals(2, dec.getScale()); - } - - @Test - public void testBuildColumnDescriptorWithIntLogicalType() { - Map params = new HashMap<>(); - params.put("bitWidth", "32"); - params.put("isSigned", "true"); - - ParquetColumnSpec spec = - new ParquetColumnSpec( - 12, - new String[] {"count"}, - "INT32", - 0, - false, - 0, - 0, - "IntLogicalTypeAnnotation", - params); - - ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - assertEquals(PrimitiveType.PrimitiveTypeName.INT32, primitiveType.getPrimitiveTypeName()); - - LogicalTypeAnnotation.IntLogicalTypeAnnotation ann = - (LogicalTypeAnnotation.IntLogicalTypeAnnotation) primitiveType.getLogicalTypeAnnotation(); - assertEquals(32, ann.getBitWidth()); - assertTrue(ann.isSigned()); - } - - @Test - public void testBuildColumnDescriptorWithStringLogicalType() { - ParquetColumnSpec spec = - new ParquetColumnSpec( - 13, - new String[] {"name"}, - "BINARY", - 0, - false, - 0, - 0, - "StringLogicalTypeAnnotation", - Collections.emptyMap()); - - ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - assertEquals(PrimitiveType.PrimitiveTypeName.BINARY, primitiveType.getPrimitiveTypeName()); - assertTrue( - primitiveType.getLogicalTypeAnnotation() - instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation); - } -} diff --git a/common/src/test/resources/log4j.properties b/common/src/test/resources/log4j.properties deleted file mode 100644 index 2f46ce1553..0000000000 --- a/common/src/test/resources/log4j.properties +++ /dev/null @@ -1,36 +0,0 @@ -# -# 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. -# - -# Set everything to be logged to the file target/unit-tests.log -test.appender=file -log4j.rootCategory=INFO, ${test.appender} -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Tests that launch java subprocesses can set the "test.appender" system property to -# "console" to avoid having the child process's logs overwrite the unit test's -# log file. -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%t: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.sparkproject.jetty=WARN diff --git a/common/src/test/resources/log4j2.properties b/common/src/test/resources/log4j2.properties deleted file mode 100644 index 04cdf85330..0000000000 --- a/common/src/test/resources/log4j2.properties +++ /dev/null @@ -1,40 +0,0 @@ -# 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. - -# Set everything to be logged to the file target/unit-tests.log -rootLogger.level = info -rootLogger.appenderRef.file.ref = ${sys:test.appender:-File} - -appender.file.type = File -appender.file.name = File -appender.file.fileName = target/unit-tests.log -appender.file.layout.type = PatternLayout -appender.file.layout.pattern = %d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Tests that launch java subprocesses can set the "test.appender" system property to -# "console" to avoid having the child process's logs overwrite the unit test's -# log file. -appender.console.type = Console -appender.console.name = console -appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %t: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -logger.jetty.name = org.sparkproject.jetty -logger.jetty.level = warn - diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index ebe53f49dd..91bfe70d53 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index d3544881af1..d075572c5b3 100644 +index d3544881af1..1126f287096 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -500,7 +500,17 @@ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/s index a6b295578d6..91acca4306f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -463,7 +463,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -260,7 +260,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet changes the WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -463,7 +464,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -510,7 +520,7 @@ index a6b295578d6..91acca4306f 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -541,7 +542,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -541,7 +543,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -608,10 +618,10 @@ index 2796b1cf154..53dcfde932e 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5691536c114 +index 00000000000..4b31bea33de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -640,9 +650,6 @@ index 00000000000..5691536c114 + * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). + */ +case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") + +/** + * Helper trait that disables Comet for all tests regardless of default config values. @@ -1040,7 +1047,7 @@ index 18123a4d6ec..0fe185baa33 100644 test("non-matching optional group") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 75eabcb96f2..7a681f147e4 100644 +index 75eabcb96f2..f8141c28f60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -21,10 +21,11 @@ import scala.collection.mutable.ArrayBuffer @@ -1061,7 +1068,7 @@ index 75eabcb96f2..7a681f147e4 100644 _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) + case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( -+ fs @ CometScanExec(_, _, _, _, partitionFilters, _, _, _, _, _, _)))) => ++ fs @ CometScanExec(_, _, _, partitionFilters, _, _, _, _, _, _)))) => + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( @@ -1969,14 +1976,14 @@ index 07e2849ce6f..3e73645b638 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 104b4e416cd..b8af360fa14 100644 +index 104b4e416cd..835aaa18e39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -38,6 +38,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ -+import org.apache.spark.sql.{IgnoreCometNativeDataFusion, IgnoreCometNativeScan} ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints @@ -1999,7 +2006,7 @@ index 104b4e416cd..b8af360fa14 100644 - test("Filters should be pushed down for vectorized Parquet reader at row group level") { + test("Filters should be pushed down for vectorized Parquet reader at row group level", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", @@ -2035,7 +2042,7 @@ index 104b4e416cd..b8af360fa14 100644 - test("filter pushdown - StringPredicate") { + test("filter pushdown - StringPredicate", -+ IgnoreCometNativeDataFusion("cannot be pushed down")) { ++ IgnoreComet("cannot be pushed down")) { import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( @@ -2045,7 +2052,7 @@ index 104b4e416cd..b8af360fa14 100644 - test("SPARK-17091: Convert IN predicate to Parquet filter push-down") { + test("SPARK-17091: Convert IN predicate to Parquet filter push-down", -+ IgnoreCometNativeScan("Comet has different push-down behavior")) { ++ IgnoreComet("Comet has different push-down behavior")) { val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) @@ -2090,7 +2097,7 @@ index 104b4e416cd..b8af360fa14 100644 - test("SPARK-34562: Bloom filter push down") { + test("SPARK-34562: Bloom filter push down", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") @@ -2121,28 +2128,10 @@ index 104b4e416cd..b8af360fa14 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8670d95c65e..9411af57a26 100644 +index 8670d95c65e..b624c3811dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -41,6 +41,7 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} - - import org.apache.spark.{SPARK_VERSION_SHORT, SparkException, TestUtils} - import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeDataFusion - import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} - import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} - import org.apache.spark.sql.catalyst.util.DateTimeUtils -@@ -1075,7 +1076,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("SPARK-35640: int as long should throw schema incompatible error") { -+ test("SPARK-35640: int as long should throw schema incompatible error", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - val data = (1 to 4).map(i => Tuple1(i)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) - -@@ -1335,7 +1337,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2153,14 +2142,14 @@ index 8670d95c65e..9411af57a26 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index 29cb224c878..ee5a87fa200 100644 +index 29cb224c878..62e3ab96004 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -27,6 +27,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.{DebugFilesystem, SparkConf, SparkException} import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeDataFusion ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.catalyst.util.ArrayData @@ -2170,7 +2159,7 @@ index 29cb224c878..ee5a87fa200 100644 - test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ") { + test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/4219")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4219")) { val data = (1 to 1000).map { i => val ts = new java.sql.Timestamp(i) Row(ts) @@ -2180,7 +2169,7 @@ index 29cb224c878..ee5a87fa200 100644 - test("SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + test("SPARK-26677: negated null-safe equality comparison should not filter matched row groups", -+ IgnoreCometNativeScan("Native scans had the filter pushed into DF operator, cannot strip")) { ++ IgnoreComet("Native scans had the filter pushed into DF operator, cannot strip")) { withAllParquetReaders { withTempPath { path => // Repeated values for dictionary encoding. @@ -2190,7 +2179,7 @@ index 29cb224c878..ee5a87fa200 100644 - test("SPARK-34212 Parquet should read decimals correctly") { + test("SPARK-34212 Parquet should read decimals correctly", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4354")) { def readParquet(schema: String, path: File): DataFrame = { spark.read.schema(schema).parquet(path.toString) } @@ -2220,7 +2209,7 @@ index 29cb224c878..ee5a87fa200 100644 - test("row group skipping doesn't overflow when reading into larger type") { + test("row group skipping doesn't overflow when reading into larger type", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4354")) { withTempPath { path => Seq(0).toDF("a").write.parquet(path.toString) // The vectorized and non-vectorized readers will produce different exceptions, we don't need @@ -2259,24 +2248,21 @@ index 240bb4e6dcb..8287ffa03ca 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala -index 351c6d698fc..cef6bb08b8c 100644 +index 351c6d698fc..d09a49f0482 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala @@ -26,6 +26,7 @@ import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -@@ -230,6 +231,17 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -230,6 +231,14 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { case f: FileSourceScanExec => numPartitions += f.inputRDD.partitions.length numOutputRows += f.metrics("numOutputRows").value -+ case b: CometScanExec => -+ numPartitions += b.inputRDD.partitions.length -+ numOutputRows += b.metrics("numOutputRows").value + case b: CometBatchScanExec => + numPartitions += b.inputRDD.partitions.length + numOutputRows += b.metrics("numOutputRows").value @@ -2309,14 +2295,14 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index bf5c51b89bb..dc3aac281c3 100644 +index bf5c51b89bb..f7402b7d883 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.Type._ import org.apache.spark.SparkException -+import org.apache.spark.sql.{IgnoreComet, IgnoreCometNativeDataFusion} ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -2336,7 +2322,7 @@ index bf5c51b89bb..dc3aac281c3 100644 - test("schema mismatch failure error message for parquet vectorized reader") { + test("schema mismatch failure error message for parquet vectorized reader", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SparkException]) @@ -2882,18 +2868,10 @@ index abe606ad9c1..2d930b64cca 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index dd55fcfe42c..99bc018008a 100644 +index dd55fcfe42c..d9a3f2df535 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -@@ -27,6 +27,7 @@ import scala.concurrent.duration._ - import scala.language.implicitConversions - import scala.util.control.NonFatal - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.scalactic.source.Position - import org.scalatest.{BeforeAndAfterAll, Suite, Tag} -@@ -41,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest +@@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ @@ -2901,7 +2879,7 @@ index dd55fcfe42c..99bc018008a 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -119,6 +121,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with +@@ -119,6 +120,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -2909,34 +2887,11 @@ index dd55fcfe42c..99bc018008a 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled) { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ return -+ } -+ if (isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return -+ } -+ if ((isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ return -+ } + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -242,6 +272,11 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +248,11 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2948,7 +2903,7 @@ index dd55fcfe42c..99bc018008a 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -434,6 +487,8 @@ private[sql] trait SQLTestUtilsBase +@@ -434,6 +445,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -2958,7 +2913,7 @@ index dd55fcfe42c..99bc018008a 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..a5ea58146ad 100644 +index ed2e309fa07..25b798d2c1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -74,6 +74,20 @@ trait SharedSparkSessionBase @@ -3071,7 +3026,7 @@ index a902cb3a69e..800a3acbe99 100644 test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala -index 07361cfdce9..97dab2a3506 100644 +index 07361cfdce9..4fdbcd18656 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -55,25 +55,41 @@ object TestHive diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 76ed210d31..51c5054f91 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index edd2ad57880..d5273840330 100644 +index edd2ad57880..15a0947abf4 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -478,10 +478,20 @@ index f33432ddb6f..b375e285dde 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index a206e97c353..fea1149b67d 100644 +index a206e97c353..8bd3ab5985a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -467,7 +467,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -264,7 +264,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet changes the WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -467,7 +468,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -491,7 +501,7 @@ index a206e97c353..fea1149b67d 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -545,7 +546,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -545,7 +547,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -503,18 +513,10 @@ index a206e97c353..fea1149b67d 100644 test("SPARK-35884: Explain Formatted") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala -index 93275487f29..78150c9163e 100644 +index 93275487f29..a5208b8d54b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala -@@ -23,6 +23,7 @@ import java.nio.file.{Files, StandardOpenOption} - - import scala.collection.mutable - -+import org.apache.comet.CometConf - import org.apache.hadoop.conf.Configuration - import org.apache.hadoop.fs.{LocalFileSystem, Path} - -@@ -33,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GreaterTha +@@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GreaterTha import org.apache.spark.sql.catalyst.expressions.IntegralLiteralTestUtils.{negativeInt, positiveInt} import org.apache.spark.sql.catalyst.plans.logical.Filter import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -522,20 +524,16 @@ index 93275487f29..78150c9163e 100644 import org.apache.spark.sql.execution.{FileSourceScanLike, SimpleMode} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.FilePartition -@@ -250,6 +252,12 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -250,6 +251,8 @@ class FileBasedDataSourceSuite extends QueryTest case "" => "_LEGACY_ERROR_TEMP_2062" case _ => "_LEGACY_ERROR_TEMP_2055" } -+ // native_datafusion Parquet scan cannot throw -+ // a SparkFileNotFoundException -+ assume(!Seq( -+ CometConf.SCAN_NATIVE_DATAFUSION, -+ CometConf.SCAN_AUTO -+ ).contains(CometConf.COMET_NATIVE_SCAN_IMPL.get())) ++ // native_datafusion Parquet scan cannot throw a SparkFileNotFoundException ++ assume(false) checkErrorMatchPVals( exception = intercept[SparkException] { testIgnoreMissingFiles(options) -@@ -656,18 +664,25 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -656,18 +659,25 @@ class FileBasedDataSourceSuite extends QueryTest checkAnswer(sql(s"select A from $tableName"), data.select("A")) // RuntimeException is triggered at executor side, which is then wrapped as @@ -568,7 +566,7 @@ index 93275487f29..78150c9163e 100644 errorClass = "_LEGACY_ERROR_TEMP_2093", parameters = Map("requiredFieldName" -> "b", "matchedOrcFields" -> "[b, B]") ) -@@ -955,6 +970,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -955,6 +965,7 @@ class FileBasedDataSourceSuite extends QueryTest assert(bJoinExec.isEmpty) val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { case smJoin: SortMergeJoinExec => smJoin @@ -576,7 +574,7 @@ index 93275487f29..78150c9163e 100644 } assert(smJoinExec.nonEmpty) } -@@ -1015,6 +1031,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1015,6 +1026,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -584,7 +582,7 @@ index 93275487f29..78150c9163e 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) -@@ -1056,6 +1073,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1056,6 +1068,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -592,7 +590,7 @@ index 93275487f29..78150c9163e 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) -@@ -1240,6 +1258,9 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1240,6 +1253,9 @@ class FileBasedDataSourceSuite extends QueryTest val filters = df.queryExecution.executedPlan.collect { case f: FileSourceScanLike => f.dataFilters case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters @@ -604,10 +602,10 @@ index 93275487f29..78150c9163e 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..1ee842b6f62 +index 00000000000..a42533c7c84 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -636,9 +634,6 @@ index 00000000000..1ee842b6f62 + * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). + */ +case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") + +/** + * Helper trait that disables Comet for all tests regardless of default config values. @@ -992,7 +987,7 @@ index 8b4ac474f87..3f79f20822f 100644 extensions.injectColumnar(session => MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 04702201f82..a4b5c6c93ce 100644 +index 04702201f82..c5ab5443ff7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -22,10 +22,11 @@ import scala.collection.mutable.ArrayBuffer @@ -1013,7 +1008,7 @@ index 04702201f82..a4b5c6c93ce 100644 _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) + case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( -+ fs @ CometScanExec(_, _, _, _, partitionFilters, _, _, _, _, _, _)))) => ++ fs @ CometScanExec(_, _, _, partitionFilters, _, _, _, _, _, _)))) => + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( @@ -1958,7 +1953,7 @@ index 07e2849ce6f..3e73645b638 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 8e88049f51e..20d7ef7b1bc 100644 +index 8e88049f51e..f8e2194a8ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1095,7 +1095,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @@ -1980,7 +1975,7 @@ index 8e88049f51e..20d7ef7b1bc 100644 - test("Filters should be pushed down for vectorized Parquet reader at row group level") { + test("Filters should be pushed down for vectorized Parquet reader at row group level", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", @@ -2015,7 +2010,7 @@ index 8e88049f51e..20d7ef7b1bc 100644 } - test("filter pushdown - StringPredicate") { -+ test("filter pushdown - StringPredicate", IgnoreCometNativeScan("cannot be pushed down")) { ++ test("filter pushdown - StringPredicate", IgnoreComet("cannot be pushed down")) { import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( @@ -2025,7 +2020,7 @@ index 8e88049f51e..20d7ef7b1bc 100644 - test("SPARK-17091: Convert IN predicate to Parquet filter push-down") { + test("SPARK-17091: Convert IN predicate to Parquet filter push-down", -+ IgnoreCometNativeScan("Comet has different push-down behavior")) { ++ IgnoreComet("Comet has different push-down behavior")) { val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) @@ -2073,7 +2068,7 @@ index 8e88049f51e..20d7ef7b1bc 100644 - test("SPARK-34562: Bloom filter push down") { + test("SPARK-34562: Bloom filter push down", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") @@ -2104,20 +2099,10 @@ index 8e88049f51e..20d7ef7b1bc 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8ed9ef1630e..71e22972a47 100644 +index 8ed9ef1630e..eed2a6f5ad5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -1075,7 +1075,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("SPARK-35640: int as long should throw schema incompatible error") { -+ test("SPARK-35640: int as long should throw schema incompatible error", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - val data = (1 to 4).map(i => Tuple1(i)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) - -@@ -1345,7 +1346,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1345,7 +1345,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2128,7 +2113,7 @@ index 8ed9ef1630e..71e22972a47 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index f6472ba3d9d..5ea2d938664 100644 +index f6472ba3d9d..e8ccb8dec54 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -185,7 +185,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS @@ -2137,7 +2122,7 @@ index f6472ba3d9d..5ea2d938664 100644 - test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ") { + test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/4219")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4219")) { val data = (1 to 1000).map { i => val ts = new java.sql.Timestamp(i) Row(ts) @@ -2147,7 +2132,7 @@ index f6472ba3d9d..5ea2d938664 100644 - test("SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + test("SPARK-26677: negated null-safe equality comparison should not filter matched row groups", -+ IgnoreCometNativeScan("Native scans had the filter pushed into DF operator, cannot strip")) { ++ IgnoreComet("Native scans had the filter pushed into DF operator, cannot strip")) { withAllParquetReaders { withTempPath { path => // Repeated values for dictionary encoding. @@ -2157,7 +2142,7 @@ index f6472ba3d9d..5ea2d938664 100644 - test("SPARK-34212 Parquet should read decimals correctly") { + test("SPARK-34212 Parquet should read decimals correctly", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4354")) { def readParquet(schema: String, path: File): DataFrame = { spark.read.schema(schema).parquet(path.toString) } @@ -2187,7 +2172,7 @@ index f6472ba3d9d..5ea2d938664 100644 - test("row group skipping doesn't overflow when reading into larger type") { + test("row group skipping doesn't overflow when reading into larger type", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4354")) { withTempPath { path => Seq(0).toDF("a").write.parquet(path.toString) // The vectorized and non-vectorized readers will produce different exceptions, we don't need @@ -2226,24 +2211,21 @@ index 4f906411345..6cc69f7e915 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala -index 27c2a2148fd..808baf9e778 100644 +index 27c2a2148fd..b3d753b9824 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala @@ -26,6 +26,7 @@ import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -@@ -243,6 +244,17 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -243,6 +244,14 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { case f: FileSourceScanExec => numPartitions += f.inputRDD.partitions.length numOutputRows += f.metrics("numOutputRows").value -+ case b: CometScanExec => -+ numPartitions += b.inputRDD.partitions.length -+ numOutputRows += b.metrics("numOutputRows").value + case b: CometBatchScanExec => + numPartitions += b.inputRDD.partitions.length + numOutputRows += b.metrics("numOutputRows").value @@ -2276,14 +2258,14 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index 3f47c5e506f..2ac0868407e 100644 +index 3f47c5e506f..8e8d2a1634d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.Type._ import org.apache.spark.SparkException -+import org.apache.spark.sql.{IgnoreComet, IgnoreCometNativeDataFusion} ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -2303,7 +2285,7 @@ index 3f47c5e506f..2ac0868407e 100644 - test("schema mismatch failure error message for parquet vectorized reader") { + test("schema mismatch failure error message for parquet vectorized reader", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SparkException]) @@ -2834,18 +2816,10 @@ index abe606ad9c1..2d930b64cca 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index e937173a590..7d20538bc68 100644 +index e937173a590..263934fbe7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -@@ -27,6 +27,7 @@ import scala.concurrent.duration._ - import scala.language.implicitConversions - import scala.util.control.NonFatal - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.scalactic.source.Position - import org.scalatest.{BeforeAndAfterAll, Suite, Tag} -@@ -41,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest +@@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ @@ -2853,7 +2827,7 @@ index e937173a590..7d20538bc68 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -119,6 +121,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with +@@ -119,6 +120,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -2861,34 +2835,11 @@ index e937173a590..7d20538bc68 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled) { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ return -+ } -+ if (isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return -+ } -+ if ((isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ return -+ } + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -242,6 +272,11 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +248,11 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2900,7 +2851,7 @@ index e937173a590..7d20538bc68 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +488,8 @@ private[sql] trait SQLTestUtilsBase +@@ -435,6 +446,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -2910,7 +2861,7 @@ index e937173a590..7d20538bc68 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..a5ea58146ad 100644 +index ed2e309fa07..25b798d2c1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -74,6 +74,20 @@ trait SharedSparkSessionBase @@ -3023,7 +2974,7 @@ index 6160c3e5f6c..0956d7d9edc 100644 test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala -index 1d646f40b3e..5babe505301 100644 +index 1d646f40b3e..df108c17c42 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -53,25 +53,41 @@ object TestHive diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index 34deaa5825..5bfc423cd5 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -39,7 +39,7 @@ index 6c51bd4ff2e..e72ec1d26e2 100644 withSpark(sc) { sc => TestUtils.waitUntilExecutorsUp(sc, 2, 60000) diff --git a/pom.xml b/pom.xml -index 252cfdf9073..cc878eb3cd9 100644 +index 252cfdf9073..64e899efe6b 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -615,10 +615,20 @@ index 2c24cc7d570..12d897866da 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index 9c90e0105a4..fadf2f0f698 100644 +index 9c90e0105a4..ed6d4887b13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -470,7 +470,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -267,7 +267,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet changes the WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -470,7 +471,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -628,7 +638,7 @@ index 9c90e0105a4..fadf2f0f698 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -548,7 +549,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -548,7 +550,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -739,10 +749,10 @@ index 9c529d14221..ab2850b5d68 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5691536c114 +index 00000000000..4b31bea33de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -771,9 +781,6 @@ index 00000000000..5691536c114 + * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). + */ +case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") + +/** + * Helper trait that disables Comet for all tests regardless of default config values. @@ -1199,7 +1206,7 @@ index 0df7f806272..92390bd819f 100644 test("non-matching optional group") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 2e33f6505ab..54f5081e10a 100644 +index 2e33f6505ab..fc1a2c8f964 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -23,10 +23,11 @@ import org.apache.spark.SparkRuntimeException @@ -1220,7 +1227,7 @@ index 2e33f6505ab..54f5081e10a 100644 _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) + case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( -+ fs @ CometScanExec(_, _, _, _, partitionFilters, _, _, _, _, _, _)))) => ++ fs @ CometScanExec(_, _, _, partitionFilters, _, _, _, _, _, _)))) => + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( @@ -2544,14 +2551,14 @@ index cd6f41b4ef4..4b6a17344bc 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 6080a5e8e4b..ea058d57b4b 100644 +index 6080a5e8e4b..23a451d5bcf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -38,6 +38,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.{SparkConf, SparkException, SparkRuntimeException} import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeScan ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints @@ -2574,7 +2581,7 @@ index 6080a5e8e4b..ea058d57b4b 100644 - test("Filters should be pushed down for vectorized Parquet reader at row group level") { + test("Filters should be pushed down for vectorized Parquet reader at row group level", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", @@ -2609,7 +2616,7 @@ index 6080a5e8e4b..ea058d57b4b 100644 } - test("filter pushdown - StringPredicate") { -+ test("filter pushdown - StringPredicate", IgnoreCometNativeScan("cannot be pushed down")) { ++ test("filter pushdown - StringPredicate", IgnoreComet("cannot be pushed down")) { import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( @@ -2619,7 +2626,7 @@ index 6080a5e8e4b..ea058d57b4b 100644 - test("SPARK-17091: Convert IN predicate to Parquet filter push-down") { + test("SPARK-17091: Convert IN predicate to Parquet filter push-down", -+ IgnoreCometNativeScan("Comet has different push-down behavior")) { ++ IgnoreComet("Comet has different push-down behavior")) { val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) @@ -2666,7 +2673,7 @@ index 6080a5e8e4b..ea058d57b4b 100644 - test("SPARK-34562: Bloom filter push down") { + test("SPARK-34562: Bloom filter push down", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") @@ -2697,28 +2704,10 @@ index 6080a5e8e4b..ea058d57b4b 100644 case _ => assert(false, "Can not match ParquetTable in the query.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 4474ec1fd42..05fa0257c82 100644 +index 4474ec1fd42..97910c4fc3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -39,6 +39,7 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} - - import org.apache.spark.{SPARK_VERSION_SHORT, SparkException, TestUtils} - import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeDataFusion - import org.apache.spark.sql.catalyst.InternalRow - import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} - import org.apache.spark.sql.catalyst.util.DateTimeUtils -@@ -1059,7 +1060,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("SPARK-35640: read binary as timestamp should throw schema incompatible error") { -+ test("SPARK-35640: read binary as timestamp should throw schema incompatible error", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - val data = (1 to 4).map(i => Tuple1(i.toString)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.TimestampType))) - -@@ -1344,7 +1346,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1344,7 +1344,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2729,18 +2718,10 @@ index 4474ec1fd42..05fa0257c82 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index bba71f1c48d..5a111a937a9 100644 +index bba71f1c48d..38c60ee2584 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -27,6 +27,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat - - import org.apache.spark.{DebugFilesystem, SparkConf, SparkException} - import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeDataFusion - import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} - import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow - import org.apache.spark.sql.catalyst.util.ArrayData -@@ -996,7 +997,11 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -996,7 +996,11 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS Seq(Some("A"), Some("A"), None).toDF().repartition(1) .write.parquet(path.getAbsolutePath) val df = spark.read.parquet(path.getAbsolutePath) @@ -2753,17 +2734,7 @@ index bba71f1c48d..5a111a937a9 100644 } } } -@@ -1042,7 +1047,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") - } - -- test("SPARK-34212 Parquet should read decimals correctly") { -+ test("SPARK-34212 Parquet should read decimals correctly", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - def readParquet(schema: String, path: File): DataFrame = { - spark.read.schema(schema).parquet(path.toString) - } -@@ -1060,7 +1066,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1060,7 +1064,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema2, path), df) } @@ -2773,7 +2744,7 @@ index bba71f1c48d..5a111a937a9 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1084,7 +1091,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1084,7 +1089,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -2783,16 +2754,6 @@ index bba71f1c48d..5a111a937a9 100644 checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("a DECIMAL(11, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) -@@ -1131,7 +1139,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - } - } - -- test("row group skipping doesn't overflow when reading into larger type") { -+ test("row group skipping doesn't overflow when reading into larger type", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - withTempPath { path => - Seq(0).toDF("a").write.parquet(path.toString) - withAllParquetReaders { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala index 30503af0fab..1491f4bc2d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala @@ -2819,32 +2780,21 @@ index 30503af0fab..1491f4bc2d5 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala -index 08fd8a9ecb5..27aee839b8c 100644 +index 08fd8a9ecb5..306958da489 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala -@@ -20,6 +20,7 @@ import java.io.File - - import scala.jdk.CollectionConverters._ - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.apache.parquet.column.ParquetProperties._ - import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} -@@ -27,6 +28,7 @@ import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE +@@ -27,6 +27,7 @@ import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE import org.apache.spark.SparkException import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -@@ -245,6 +247,17 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -245,6 +246,14 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { case f: FileSourceScanExec => numPartitions += f.inputRDD.partitions.length numOutputRows += f.metrics("numOutputRows").value -+ case b: CometScanExec => -+ numPartitions += b.inputRDD.partitions.length -+ numOutputRows += b.metrics("numOutputRows").value + case b: CometBatchScanExec => + numPartitions += b.inputRDD.partitions.length + numOutputRows += b.metrics("numOutputRows").value @@ -2856,16 +2806,13 @@ index 08fd8a9ecb5..27aee839b8c 100644 case _ => } assert(numPartitions > 0) -@@ -303,6 +316,12 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -303,6 +312,9 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { val conf = RowIndexTestConf(useDataSourceV2 = useDataSourceV2) test(s"invalid row index column type - ${conf.desc}") { + // https://github.com/apache/datafusion-comet/issues/3886 + // Comet throws RuntimeException instead of SparkException -+ assume(!Seq( -+ CometConf.SCAN_NATIVE_DATAFUSION, -+ CometConf.SCAN_AUTO -+ ).contains(CometConf.COMET_NATIVE_SCAN_IMPL.get())) ++ assume(false) withSQLConf(conf.sqlConfs: _*) { withTempPath{ path => val df = spark.range(0, 10, 1, 1).toDF("id") @@ -2890,7 +2837,7 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index 0acb21f3e6f..8d60dfb686d 100644 +index 0acb21f3e6f..15bd866d8aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,7 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName @@ -2898,7 +2845,7 @@ index 0acb21f3e6f..8d60dfb686d 100644 import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row} import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -2918,23 +2865,14 @@ index 0acb21f3e6f..8d60dfb686d 100644 - test("schema mismatch failure error message for parquet vectorized reader") { + test("schema mismatch failure error message for parquet vectorized reader", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala -index 09ed6955a51..5cd856ff7b6 100644 +index 09ed6955a51..236a4e99824 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala -@@ -24,7 +24,7 @@ import org.apache.parquet.format.converter.ParquetMetadataConverter - import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} - - import org.apache.spark.SparkException --import org.apache.spark.sql.{DataFrame, QueryTest, Row} -+import org.apache.spark.sql.{DataFrame, IgnoreCometNativeDataFusion, QueryTest, Row} - import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper - import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException - import org.apache.spark.sql.functions.col @@ -65,7 +65,9 @@ class ParquetTypeWideningSuite withClue( s"with dictionary encoding '$dictionaryEnabled' with timestamp rebase mode " + @@ -2965,56 +2903,6 @@ index 09ed6955a51..5cd856ff7b6 100644 ) } test(s"parquet widening conversion $fromType -> $toType") { -@@ -231,7 +234,8 @@ class ParquetTypeWideningSuite - (Seq("2020-01-01", "2020-01-02", "1312-02-27"), DateType, TimestampType) - ) - } -- test(s"unsupported parquet conversion $fromType -> $toType") { -+ test(s"unsupported parquet conversion $fromType -> $toType", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - checkAllParquetReaders(values, fromType, toType, expectError = true) - } - -@@ -257,7 +261,8 @@ class ParquetTypeWideningSuite - (Seq("1", "2"), LongType, DecimalType(LongDecimal.precision, 1)) - ) - } -- test(s"unsupported parquet conversion $fromType -> $toType") { -+ test(s"unsupported parquet conversion $fromType -> $toType", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - checkAllParquetReaders(values, fromType, toType, - expectError = - // parquet-mr allows reading decimals into a smaller precision decimal type without -@@ -271,7 +276,8 @@ class ParquetTypeWideningSuite - (Seq("2020-01-01", "2020-01-02", "1312-02-27"), TimestampNTZType, DateType)) - outputTimestampType <- ParquetOutputTimestampType.values - } -- test(s"unsupported parquet timestamp conversion $fromType ($outputTimestampType) -> $toType") { -+ test(s"unsupported parquet timestamp conversion $fromType ($outputTimestampType) -> $toType", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - withSQLConf( - SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> outputTimestampType.toString, - SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> LegacyBehaviorPolicy.CORRECTED.toString -@@ -291,7 +297,8 @@ class ParquetTypeWideningSuite - Seq(7 -> 5, 10 -> 5, 20 -> 5, 12 -> 10, 20 -> 10, 22 -> 20) - } - test( -- s"parquet decimal precision change Decimal($fromPrecision, 2) -> Decimal($toPrecision, 2)") { -+ s"parquet decimal precision change Decimal($fromPrecision, 2) -> Decimal($toPrecision, 2)", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - checkAllParquetReaders( - values = Seq("1.23", "10.34"), - fromType = DecimalType(fromPrecision, 2), -@@ -322,7 +329,8 @@ class ParquetTypeWideningSuite - Seq((5, 2) -> (6, 4), (10, 4) -> (12, 7), (20, 5) -> (22, 8)) - } - test(s"parquet decimal precision and scale change Decimal($fromPrecision, $fromScale) -> " + -- s"Decimal($toPrecision, $toScale)" -+ s"Decimal($toPrecision, $toScale)", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720") - ) { - checkAllParquetReaders( - values = Seq("1.23", "10.34"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index b8f3ea3c6f3..bbd44221288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -3565,26 +3453,19 @@ index 86c4e49f6f6..2e639e5f38d 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index f0f3f94b811..f77b54dcef9 100644 +index f0f3f94b811..b7d18771314 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -@@ -27,13 +27,14 @@ import scala.jdk.CollectionConverters._ - import scala.language.implicitConversions - import scala.util.control.NonFatal - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.scalactic.source.Position - import org.scalatest.{BeforeAndAfterAll, Suite, Tag} +@@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, Suite, Tag} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, IgnoreCometNativeIcebergCompat, IgnoreCometNativeScan, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE -@@ -42,6 +43,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase +@@ -42,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.classic.{ClassicConversions, ColumnConversions, ColumnNodeToExpressionConverter, DataFrame, Dataset, SparkSession, SQLImplicits} @@ -3592,7 +3473,7 @@ index f0f3f94b811..f77b54dcef9 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -121,6 +123,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with +@@ -121,6 +122,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -3600,34 +3481,11 @@ index f0f3f94b811..f77b54dcef9 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled) { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ return -+ } -+ if (isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return -+ } -+ if ((isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ return -+ } + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -248,8 +278,15 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +254,15 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3643,7 +3501,7 @@ index f0f3f94b811..f77b54dcef9 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +497,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +464,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3653,7 +3511,7 @@ index f0f3f94b811..f77b54dcef9 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index 245219c1756..a611836f086 100644 +index 245219c1756..b566f970ccd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -75,6 +75,21 @@ trait SharedSparkSessionBase @@ -3796,7 +3654,7 @@ index b67370f6eb9..746b3974b29 100644 override def beforeEach(): Unit = { super.beforeEach() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala -index a394d0b7393..a4bc3d3fd8e 100644 +index a394d0b7393..3e1f0404a37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -53,24 +53,34 @@ object TestHive diff --git a/dev/diffs/4.1.1.diff b/dev/diffs/4.1.1.diff index f685e6146a..5cf6326dbf 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.diff @@ -695,10 +695,20 @@ index e1a2fd33c7c..632f4b695df 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index b27122a8de2..a4c5aac8212 100644 +index b27122a8de2..3c690dbe788 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -470,7 +470,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -267,7 +267,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("SPARK-33853: explain codegen - check presence of subquery") { ++ test("SPARK-33853: explain codegen - check presence of subquery", ++ IgnoreComet("Comet plan has a different WholeStageCodegen subtree count")) { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { + withTempView("df") { + val df1 = spark.range(1, 100) +@@ -470,7 +471,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -708,7 +718,7 @@ index b27122a8de2..a4c5aac8212 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -548,7 +549,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -548,7 +550,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -800,10 +810,10 @@ index 95e86fe4311..fb2b6363af6 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5691536c114 +index 00000000000..4b31bea33de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -832,9 +842,6 @@ index 00000000000..5691536c114 + * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). + */ +case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") + +/** + * Helper trait that disables Comet for all tests regardless of default config values. @@ -1292,7 +1299,7 @@ index ff0ee19ae97..01958e0c45b 100644 test("non-matching optional group") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala -index 3ba48da0e32..401a2851b66 100644 +index 3ba48da0e32..a33e65d4420 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -23,10 +23,11 @@ import org.apache.spark.SparkRuntimeException @@ -1313,7 +1320,7 @@ index 3ba48da0e32..401a2851b66 100644 _.asInstanceOf[FileScanRDD].filePartitions.forall( _.files.forall(_.urlEncodedPath.contains("p=0")))) + case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( -+ fs @ CometScanExec(_, _, _, _, partitionFilters, _, _, _, _, _, _)))) => ++ fs @ CometScanExec(_, _, _, partitionFilters, _, _, _, _, _, _)))) => + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( @@ -2685,14 +2692,14 @@ index cd6f41b4ef4..4b6a17344bc 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 6b73cc8618d..81a58f43784 100644 +index 6b73cc8618d..e67aaeff9df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -38,6 +38,7 @@ import org.apache.parquet.schema.MessageType import org.apache.spark.{SparkConf, SparkException, SparkRuntimeException} import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeScan ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints @@ -2715,7 +2722,7 @@ index 6b73cc8618d..81a58f43784 100644 - test("Filters should be pushed down for vectorized Parquet reader at row group level") { + test("Filters should be pushed down for vectorized Parquet reader at row group level", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", @@ -2750,7 +2757,7 @@ index 6b73cc8618d..81a58f43784 100644 } - test("filter pushdown - StringPredicate") { -+ test("filter pushdown - StringPredicate", IgnoreCometNativeScan("cannot be pushed down")) { ++ test("filter pushdown - StringPredicate", IgnoreComet("cannot be pushed down")) { import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( @@ -2760,7 +2767,7 @@ index 6b73cc8618d..81a58f43784 100644 - test("SPARK-17091: Convert IN predicate to Parquet filter push-down") { + test("SPARK-17091: Convert IN predicate to Parquet filter push-down", -+ IgnoreCometNativeScan("Comet has different push-down behavior")) { ++ IgnoreComet("Comet has different push-down behavior")) { val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) @@ -2807,7 +2814,7 @@ index 6b73cc8618d..81a58f43784 100644 - test("SPARK-34562: Bloom filter push down") { + test("SPARK-34562: Bloom filter push down", -+ IgnoreCometNativeScan("Native scans do not support the tested accumulator")) { ++ IgnoreComet("Native scans do not support the tested accumulator")) { withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") @@ -2838,14 +2845,14 @@ index 6b73cc8618d..81a58f43784 100644 case _ => assert(false, "Can not match ParquetTable in the query.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 3072657a095..6b5b9103363 100644 +index 3072657a095..599d169cf8a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -40,6 +40,7 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.{SPARK_VERSION_SHORT, SparkException, TestUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -+import org.apache.spark.sql.{IgnoreComet, IgnoreCometNativeDataFusion} ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.{DateTimeConstants, DateTimeUtils} @@ -2859,17 +2866,7 @@ index 3072657a095..6b5b9103363 100644 val data = Seq( Tuple1((null, null)), Tuple1((null, null)), -@@ -1282,7 +1284,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("SPARK-35640: read binary as timestamp should throw schema incompatible error") { -+ test("SPARK-35640: read binary as timestamp should throw schema incompatible error", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - val data = (1 to 4).map(i => Tuple1(i.toString)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.TimestampType))) - -@@ -1567,7 +1570,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1567,7 +1569,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2880,18 +2877,10 @@ index 3072657a095..6b5b9103363 100644 checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index c530dc0d3df..7e1dd663873 100644 +index c530dc0d3df..d23069689da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -27,6 +27,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat - - import org.apache.spark.{DebugFilesystem, SparkConf, SparkException} - import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeDataFusion - import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} - import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow - import org.apache.spark.sql.catalyst.util.ArrayData -@@ -996,7 +997,11 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -996,7 +996,11 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS Seq(Some("A"), Some("A"), None).toDF().repartition(1) .write.parquet(path.getAbsolutePath) val df = spark.read.parquet(path.getAbsolutePath) @@ -2904,17 +2893,7 @@ index c530dc0d3df..7e1dd663873 100644 } } } -@@ -1042,7 +1047,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") - } - -- test("SPARK-34212 Parquet should read decimals correctly") { -+ test("SPARK-34212 Parquet should read decimals correctly", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - def readParquet(schema: String, path: File): DataFrame = { - spark.read.schema(schema).parquet(path.toString) - } -@@ -1060,7 +1066,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1060,7 +1064,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema2, path), df) } @@ -2924,7 +2903,7 @@ index c530dc0d3df..7e1dd663873 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1084,7 +1091,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1084,7 +1089,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -2934,16 +2913,6 @@ index c530dc0d3df..7e1dd663873 100644 checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("a DECIMAL(11, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) -@@ -1131,7 +1139,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - } - } - -- test("row group skipping doesn't overflow when reading into larger type") { -+ test("row group skipping doesn't overflow when reading into larger type", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - withTempPath { path => - Seq(0).toDF("a").write.parquet(path.toString) - withAllParquetReaders { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala index 30503af0fab..1491f4bc2d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala @@ -2970,32 +2939,21 @@ index 30503af0fab..1491f4bc2d5 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala -index 08fd8a9ecb5..27aee839b8c 100644 +index 08fd8a9ecb5..306958da489 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala -@@ -20,6 +20,7 @@ import java.io.File - - import scala.jdk.CollectionConverters._ - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.apache.parquet.column.ParquetProperties._ - import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} -@@ -27,6 +28,7 @@ import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE +@@ -27,6 +27,7 @@ import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE import org.apache.spark.SparkException import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -@@ -245,6 +247,17 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -245,6 +246,14 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { case f: FileSourceScanExec => numPartitions += f.inputRDD.partitions.length numOutputRows += f.metrics("numOutputRows").value -+ case b: CometScanExec => -+ numPartitions += b.inputRDD.partitions.length -+ numOutputRows += b.metrics("numOutputRows").value + case b: CometBatchScanExec => + numPartitions += b.inputRDD.partitions.length + numOutputRows += b.metrics("numOutputRows").value @@ -3007,16 +2965,13 @@ index 08fd8a9ecb5..27aee839b8c 100644 case _ => } assert(numPartitions > 0) -@@ -303,6 +316,12 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -303,6 +312,9 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { val conf = RowIndexTestConf(useDataSourceV2 = useDataSourceV2) test(s"invalid row index column type - ${conf.desc}") { + // https://github.com/apache/datafusion-comet/issues/3886 + // Comet throws RuntimeException instead of SparkException -+ assume(!Seq( -+ CometConf.SCAN_NATIVE_DATAFUSION, -+ CometConf.SCAN_AUTO -+ ).contains(CometConf.COMET_NATIVE_SCAN_IMPL.get())) ++ assume(false) withSQLConf(conf.sqlConfs: _*) { withTempPath{ path => val df = spark.range(0, 10, 1, 1).toDF("id") @@ -3041,7 +2996,7 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index 56076175d60..78c4a9755c0 100644 +index 56076175d60..a3d47b24634 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,7 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName @@ -3049,7 +3004,7 @@ index 56076175d60..78c4a9755c0 100644 import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row} import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -3069,33 +3024,14 @@ index 56076175d60..78c4a9755c0 100644 - test("schema mismatch failure error message for parquet vectorized reader") { + test("schema mismatch failure error message for parquet vectorized reader", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) -@@ -1079,7 +1081,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { - } - } - -- test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array") { -+ test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - import testImplicits._ - - withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala -index 09ed6955a51..5cd856ff7b6 100644 +index 09ed6955a51..236a4e99824 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTypeWideningSuite.scala -@@ -24,7 +24,7 @@ import org.apache.parquet.format.converter.ParquetMetadataConverter - import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} - - import org.apache.spark.SparkException --import org.apache.spark.sql.{DataFrame, QueryTest, Row} -+import org.apache.spark.sql.{DataFrame, IgnoreCometNativeDataFusion, QueryTest, Row} - import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper - import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException - import org.apache.spark.sql.functions.col @@ -65,7 +65,9 @@ class ParquetTypeWideningSuite withClue( s"with dictionary encoding '$dictionaryEnabled' with timestamp rebase mode " + @@ -3126,56 +3062,6 @@ index 09ed6955a51..5cd856ff7b6 100644 ) } test(s"parquet widening conversion $fromType -> $toType") { -@@ -231,7 +234,8 @@ class ParquetTypeWideningSuite - (Seq("2020-01-01", "2020-01-02", "1312-02-27"), DateType, TimestampType) - ) - } -- test(s"unsupported parquet conversion $fromType -> $toType") { -+ test(s"unsupported parquet conversion $fromType -> $toType", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - checkAllParquetReaders(values, fromType, toType, expectError = true) - } - -@@ -257,7 +261,8 @@ class ParquetTypeWideningSuite - (Seq("1", "2"), LongType, DecimalType(LongDecimal.precision, 1)) - ) - } -- test(s"unsupported parquet conversion $fromType -> $toType") { -+ test(s"unsupported parquet conversion $fromType -> $toType", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - checkAllParquetReaders(values, fromType, toType, - expectError = - // parquet-mr allows reading decimals into a smaller precision decimal type without -@@ -271,7 +276,8 @@ class ParquetTypeWideningSuite - (Seq("2020-01-01", "2020-01-02", "1312-02-27"), TimestampNTZType, DateType)) - outputTimestampType <- ParquetOutputTimestampType.values - } -- test(s"unsupported parquet timestamp conversion $fromType ($outputTimestampType) -> $toType") { -+ test(s"unsupported parquet timestamp conversion $fromType ($outputTimestampType) -> $toType", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - withSQLConf( - SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> outputTimestampType.toString, - SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> LegacyBehaviorPolicy.CORRECTED.toString -@@ -291,7 +297,8 @@ class ParquetTypeWideningSuite - Seq(7 -> 5, 10 -> 5, 20 -> 5, 12 -> 10, 20 -> 10, 22 -> 20) - } - test( -- s"parquet decimal precision change Decimal($fromPrecision, 2) -> Decimal($toPrecision, 2)") { -+ s"parquet decimal precision change Decimal($fromPrecision, 2) -> Decimal($toPrecision, 2)", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720")) { - checkAllParquetReaders( - values = Seq("1.23", "10.34"), - fromType = DecimalType(fromPrecision, 2), -@@ -322,7 +329,8 @@ class ParquetTypeWideningSuite - Seq((5, 2) -> (6, 4), (10, 4) -> (12, 7), (20, 5) -> (22, 8)) - } - test(s"parquet decimal precision and scale change Decimal($fromPrecision, $fromScale) -> " + -- s"Decimal($toPrecision, $toScale)" -+ s"Decimal($toPrecision, $toScale)", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3720") - ) { - checkAllParquetReaders( - values = Seq("1.23", "10.34"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala index 1cc6d3afbee..8275727fbb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala @@ -3831,26 +3717,19 @@ index 89f65562295..166a11a9079 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index f0f3f94b811..be5e113c3ed 100644 +index f0f3f94b811..b7d18771314 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -@@ -27,13 +27,14 @@ import scala.jdk.CollectionConverters._ - import scala.language.implicitConversions - import scala.util.control.NonFatal - -+import org.apache.comet.CometConf - import org.apache.hadoop.fs.Path - import org.scalactic.source.Position - import org.scalatest.{BeforeAndAfterAll, Suite, Tag} +@@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, Suite, Tag} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, IgnoreCometNativeIcebergCompat, IgnoreCometNativeScan, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.SessionCatalog.DEFAULT_DATABASE -@@ -42,6 +43,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase +@@ -42,6 +42,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.classic.{ClassicConversions, ColumnConversions, ColumnNodeToExpressionConverter, DataFrame, Dataset, SparkSession, SQLImplicits} @@ -3858,7 +3737,7 @@ index f0f3f94b811..be5e113c3ed 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -121,6 +123,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with +@@ -121,6 +122,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -3866,34 +3745,11 @@ index f0f3f94b811..be5e113c3ed 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled) { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ return -+ } -+ if (isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return -+ } -+ if ((isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ return -+ } + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -248,8 +278,15 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +254,15 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3909,7 +3765,7 @@ index f0f3f94b811..be5e113c3ed 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +488,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +464,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child diff --git a/dev/release/build-release-comet.sh b/dev/release/build-release-comet.sh index 23363c8761..159394e0f5 100755 --- a/dev/release/build-release-comet.sh +++ b/dev/release/build-release-comet.sh @@ -125,7 +125,7 @@ docker build --no-cache \ # Clean previous Java build pushd $COMET_HOME_DIR && ./mvnw clean && popd -# Clean previous native build. This is required because common/pom.xml has +# Clean previous native build. This is required because spark/pom.xml has # unconditional resource entries that bundle libcomet.dylib from # native/target/{x86_64,aarch64}-apple-darwin/release. If a release manager # previously cross-compiled those targets locally, stale dylibs would leak @@ -169,7 +169,7 @@ fi echo "Building binaries completed" echo "Copying to java build directories" -JVM_TARGET_DIR=$COMET_HOME_DIR/common/target/classes/org/apache/comet +JVM_TARGET_DIR=$COMET_HOME_DIR/spark/target/classes/org/apache/comet mkdir -p $JVM_TARGET_DIR mkdir -p $JVM_TARGET_DIR/linux/amd64 diff --git a/dev/release/create-tarball.sh b/dev/release/create-tarball.sh index 6387cf485a..fbb66815f5 100755 --- a/dev/release/create-tarball.sh +++ b/dev/release/create-tarball.sh @@ -104,7 +104,7 @@ Here is my vote: [1]: https://github.com/apache/datafusion-comet/tree/${release_hash} [2]: ${url} [3]: https://repository.apache.org/#nexus-search;quick~org.apache.datafusion -[4]: https://github.com/apache/datafusion-comet/blob/${release_hash}/CHANGELOG.md +[4]: https://github.com/apache/datafusion-comet/blob/${release_hash}/docs/source/changelog/${version}.md MAIL echo "---------------------------------------------------------" diff --git a/docs/source/about/versioning_policy.md b/docs/source/about/versioning_policy.md new file mode 100644 index 0000000000..917b164f4b --- /dev/null +++ b/docs/source/about/versioning_policy.md @@ -0,0 +1,118 @@ + + +# Versioning Policy + +Apache DataFusion Comet follows [semantic versioning](https://semver.org/) with the format +`MAJOR.MINOR.PATCH`. This document describes what each component of a release means, what +compatibility guarantees Comet provides, and how Comet relates to Apache Spark versions. + +This policy is inspired by, but is not identical to, the +[Apache Spark versioning policy](https://spark.apache.org/versioning-policy.html). The differences +reflect the fact that Comet is currently pre-1.0 and ships much more frequently than Spark. + +## Pre-1.0 Releases + +Comet is currently in the `0.x` series. Per semantic versioning, the `0.x` series is considered +unstable: + +- A minor release (`0.X.0`) may include breaking changes. +- A patch release (`0.X.Y`, where `Y > 0`) contains bug fixes only. + +In particular, the following are explicitly **not** stable in the `0.x` series and may change in any +minor release without prior notice: + +- Configuration keys under `spark.comet.*` (names, defaults, and semantics). +- The protobuf format used to serialize query plans between the JVM and the native library. +- Internal Scala, Java, and Rust APIs that are not part of the documented public API. + +Where a breaking change is significant, it will be called out in the release notes. + +## Compatibility Commitments + +The following commitments apply to every Comet release, including the `0.x` series. + +### Public Scala and Java API + +Public classes and methods (for example, `org.apache.spark.CometPlugin`) are considered part of +Comet's public API. Removing or making source- or binary-incompatible changes to a public API +requires a deprecation cycle: the API must remain available, with a deprecation warning, for at +least one minor release before removal. + +Public APIs annotated with `@Unstable` are exempt from this guarantee and may change in any minor +release without a deprecation cycle. The `@Unstable` annotation does not yet exist and will be +introduced as the need arises. + +### Query Result Semantics + +Expressions and operators whose support level is `Compatible` are expected to produce results that +match Apache Spark. Result differences in `Compatible` items are tracked as bugs and fixed in +subsequent releases. + +Items whose support level is `Incompatible` or `Unsupported` have no result-compatibility +guarantees. `Incompatible` items require an explicit per-expression or per-operator opt-in +(for example, `spark.comet.expr..allowIncompatible=true`). + +For details on per-expression and per-operator support levels, see the +[compatibility guide](../user-guide/latest/compatibility/index.md). + +## Apache Spark Version Support + +Each Comet release supports the **latest patch version** of every Apache Spark minor release that +Comet targets at the time of release. The currently supported Spark versions are listed on the +[Spark Version Compatibility](../user-guide/latest/compatibility/spark-versions.md) page. + +When the upstream Spark project publishes a new patch within a supported minor (for example, +`3.5.8` → `3.5.9`), the next Comet release will pick it up. Older Spark patches within the same +minor are not separately supported. + +Dropping support for an entire Spark minor (for example, removing Spark 3.4) is announced in the +release notes of the **preceding** Comet minor release, giving users at least one release of +notice before the change takes effect. + +Comet binaries are published per `(Spark minor × Scala binary version)` combination. Users must +select the binary that matches their Spark and Scala installation. + +## Release Cadence + +Comet targets a `0.X.0` minor release every four to six weeks. Patch releases (`0.X.Y`) are made +on demand, only when a critical bug or security fix needs to ship before the next minor release. + +Only the most recent minor release receives patch releases. Comet does not currently backport +fixes to older minor releases; users are expected to upgrade forward. + +## Native Library Coupling + +Each Comet release ships a JVM jar and a native library that are built and tested together. The +two artifacts must come from the **same Comet release**. Mixing a JVM jar from one Comet release +with a native library from another is unsupported and may fail at runtime due to protobuf or FFI +incompatibilities. + +## Road to 1.0 + +When `1.0.0` ships: + +- Strict semantic versioning will apply to the public Scala and Java API: breaking changes will + only be made in a future major release. APIs annotated with `@Unstable` remain exempt and may + change in any minor release. +- The stability commitments for configuration keys and the protobuf plan format will be + re-evaluated and documented as part of the `1.0.0` release. + +Tracking and planning for the `1.0.0` release happens in +[issue #4082](https://github.com/apache/datafusion-comet/issues/4082). diff --git a/dev/changelog/0.1.0.md b/docs/source/changelog/0.1.0.md similarity index 100% rename from dev/changelog/0.1.0.md rename to docs/source/changelog/0.1.0.md diff --git a/dev/changelog/0.10.0.md b/docs/source/changelog/0.10.0.md similarity index 100% rename from dev/changelog/0.10.0.md rename to docs/source/changelog/0.10.0.md diff --git a/dev/changelog/0.11.0.md b/docs/source/changelog/0.11.0.md similarity index 100% rename from dev/changelog/0.11.0.md rename to docs/source/changelog/0.11.0.md diff --git a/dev/changelog/0.12.0.md b/docs/source/changelog/0.12.0.md similarity index 100% rename from dev/changelog/0.12.0.md rename to docs/source/changelog/0.12.0.md diff --git a/dev/changelog/0.13.0.md b/docs/source/changelog/0.13.0.md similarity index 100% rename from dev/changelog/0.13.0.md rename to docs/source/changelog/0.13.0.md diff --git a/dev/changelog/0.14.0.md b/docs/source/changelog/0.14.0.md similarity index 100% rename from dev/changelog/0.14.0.md rename to docs/source/changelog/0.14.0.md diff --git a/dev/changelog/0.14.1.md b/docs/source/changelog/0.14.1.md similarity index 100% rename from dev/changelog/0.14.1.md rename to docs/source/changelog/0.14.1.md diff --git a/dev/changelog/0.15.0.md b/docs/source/changelog/0.15.0.md similarity index 100% rename from dev/changelog/0.15.0.md rename to docs/source/changelog/0.15.0.md diff --git a/docs/source/changelog/0.16.0.md b/docs/source/changelog/0.16.0.md new file mode 100644 index 0000000000..1e0aea5bd5 --- /dev/null +++ b/docs/source/changelog/0.16.0.md @@ -0,0 +1,190 @@ + + +# DataFusion Comet 0.16.0 Changelog + +This release consists of 127 commits from 17 contributors. See credits at the end of this changelog for more information. + +**Fixed bugs:** + +- fix: report task output metrics in Spark UI [#3999](https://github.com/apache/datafusion-comet/pull/3999) (0lai0) +- fix: cast to and from timestamp_ntz [#4008](https://github.com/apache/datafusion-comet/pull/4008) (parthchandra) +- fix: support to_json on Spark 4.0 [#4036](https://github.com/apache/datafusion-comet/pull/4036) (andygrove) +- fix: enable arrays_overlap [#3901](https://github.com/apache/datafusion-comet/pull/3901) (kazuyukitanimura) +- fix: Iceberg reflection for current() on TableOperations hierarchy [#3895](https://github.com/apache/datafusion-comet/pull/3895) (karuppayya) +- fix: fall back to Spark for shuffle/sort/aggregate on non-default collated strings [Spark 4] [#4035](https://github.com/apache/datafusion-comet/pull/4035) (andygrove) +- fix: scalar subquery pushdown and reuse for CometNativeScanExec (SPARK-43402) [#4053](https://github.com/apache/datafusion-comet/pull/4053) (mbutrovich) +- fix: fall back for shredded Variant scans on Spark 4.0 [#4084](https://github.com/apache/datafusion-comet/pull/4084) (andygrove) +- fix: enable Spark 4 SQL tests previously ignored for issues #3313 and #3314 [#4092](https://github.com/apache/datafusion-comet/pull/4092) (andygrove) +- fix: fall back to Spark for hash join and sort-merge join on non-default collated string keys [Spark 4] [#4095](https://github.com/apache/datafusion-comet/pull/4095) (0lai0) +- fix: reject string/binary read as numeric in native_datafusion scan [#4091](https://github.com/apache/datafusion-comet/pull/4091) (andygrove) +- fix: reject incompatible decimal precision/scale in native_datafusion scan [#4090](https://github.com/apache/datafusion-comet/pull/4090) (andygrove) +- fix: throw SchemaColumnConvertNotSupportedException from native_datafusion schema mismatch [#4117](https://github.com/apache/datafusion-comet/pull/4117) (andygrove) +- fix: substring with negative start index [#4017](https://github.com/apache/datafusion-comet/pull/4017) (kazuyukitanimura) +- fix: honor strictFloatingPoint in RangePartitioning [#4167](https://github.com/apache/datafusion-comet/pull/4167) (0lai0) +- fix: [Spark 4.1.1] preserve stored allowDecimalPrecisionLoss in DecimalPrecision rule [#4179](https://github.com/apache/datafusion-comet/pull/4179) (andygrove) +- fix: [Spark 4.1.1] preserve parent struct nullness when all requested fields missing in Parquet [#4190](https://github.com/apache/datafusion-comet/pull/4190) (andygrove) +- fix: support Spark 4.1 BloomFilter V2 format and bit-scattering [#4196](https://github.com/apache/datafusion-comet/pull/4196) (andygrove) +- fix: JNI local reference cleanup in JVMClasses::with_env [#4225](https://github.com/apache/datafusion-comet/pull/4225) (0lai0) +- fix: broadcast exchange bypasses AQE partition coalescing [#4163](https://github.com/apache/datafusion-comet/pull/4163) (andygrove) +- fix: resolve Scala compiler warnings for auto-tupling and bare try [#4227](https://github.com/apache/datafusion-comet/pull/4227) (andygrove) +- fix: [Spark 4.1] preserve union output partitioning in CometUnionExec [#4207](https://github.com/apache/datafusion-comet/pull/4207) (andygrove) +- fix: re-enable tests skipped for Spark 4.1 (issue #4098) [#4253](https://github.com/apache/datafusion-comet/pull/4253) (andygrove) +- fix: cargo clean before release build to avoid stale native libs [#4257](https://github.com/apache/datafusion-comet/pull/4257) (andygrove) + +**Performance related:** + +- perf: avoid redundant columnar shuffle when both parent and child are non-Comet [#4010](https://github.com/apache/datafusion-comet/pull/4010) (andygrove) +- perf: reduce per-node allocations in to_native_metric_node [#4075](https://github.com/apache/datafusion-comet/pull/4075) (andygrove) + +**Implemented enhancements:** + +- feat: enable native Iceberg reader by default [#3819](https://github.com/apache/datafusion-comet/pull/3819) (andygrove) +- feat: support `collect_set` [#3954](https://github.com/apache/datafusion-comet/pull/3954) (comphead) +- feat: non-AQE DPP for native Parquet scans, broadcast exchange reuse for DPP subqueries [#4011](https://github.com/apache/datafusion-comet/pull/4011) (mbutrovich) +- feat: add support for array_position expression [#3172](https://github.com/apache/datafusion-comet/pull/3172) (andygrove) +- feat: Cast string to timestamp_ntz [#4034](https://github.com/apache/datafusion-comet/pull/4034) (parthchandra) +- feat: Add TimestampNTZType support for unix_timestamp [#4039](https://github.com/apache/datafusion-comet/pull/4039) (parthchandra) +- feat: fix array_compact for Spark 4.0 and correct return type metadata [#3796](https://github.com/apache/datafusion-comet/pull/3796) (andygrove) +- feat: task-level input metrics (bytesRead) for Iceberg native scan [#4128](https://github.com/apache/datafusion-comet/pull/4128) (mbutrovich) +- feat: add MapSort expression support for Spark 4.0 [#4076](https://github.com/apache/datafusion-comet/pull/4076) (andygrove) +- feat: Support Spark expression `str_to_map` [#3654](https://github.com/apache/datafusion-comet/pull/3654) (unknowntpo) +- feat: add support for timestamp_seconds expression [#3146](https://github.com/apache/datafusion-comet/pull/3146) (andygrove) +- feat: add config to gate converting Spark shuffle to Comet shuffle when child is non-Comet plan [#4166](https://github.com/apache/datafusion-comet/pull/4166) (andygrove) +- feat: AQE DPP for native Parquet scans with broadcast reuse [#4112](https://github.com/apache/datafusion-comet/pull/4112) (mbutrovich) +- feat: support regular BuildRight+LeftAnti hash join [#4073](https://github.com/apache/datafusion-comet/pull/4073) (viirya) +- feat: add bug-triage Claude skill [#4109](https://github.com/apache/datafusion-comet/pull/4109) (andygrove) +- feat: support `PartialMerge` aggregation mode [#4003](https://github.com/apache/datafusion-comet/pull/4003) (comphead) +- feat: add encode time tracking for shuffle operations [#4068](https://github.com/apache/datafusion-comet/pull/4068) (0lai0) +- feat: Add support for Spark ToDegrees and ToRadians math expressions [#3786](https://github.com/apache/datafusion-comet/pull/3786) (rafafrdz) +- feat: Add support for Spark Acosh, Asinh, Atanh math expressions [#3787](https://github.com/apache/datafusion-comet/pull/3787) (rafafrdz) +- feat: Add support for Spark Cbrt math expression [#3788](https://github.com/apache/datafusion-comet/pull/3788) (rafafrdz) +- feat: Add support for Spark Pi math expression [#3789](https://github.com/apache/datafusion-comet/pull/3789) (rafafrdz) +- feat: support Parquet field ID matching in native_datafusion scan [#4216](https://github.com/apache/datafusion-comet/pull/4216) (mbutrovich) +- feat: support AQE DPP broadcast reuse for Iceberg native scans [#4215](https://github.com/apache/datafusion-comet/pull/4215) (mbutrovich) +- feat: add support for url_encode, url_decode, and try_url_decode [#4231](https://github.com/apache/datafusion-comet/pull/4231) (parthchandra) +- feat: support TimestampType join keys in SortMergeJoin [#3986](https://github.com/apache/datafusion-comet/pull/3986) (andygrove) + +**Documentation updates:** + +- docs: Add changelog for 0.15.0 [#4000](https://github.com/apache/datafusion-comet/pull/4000) (andygrove) +- docs: Update README and benchmark results for 0.15.0 release [#3995](https://github.com/apache/datafusion-comet/pull/3995) (andygrove) +- docs: fix errors in benchmark pages [#4001](https://github.com/apache/datafusion-comet/pull/4001) (andygrove) +- docs: split compatibility guide into multiple pages [#4055](https://github.com/apache/datafusion-comet/pull/4055) (andygrove) +- docs: Generate expression compatibility docs from code [#4057](https://github.com/apache/datafusion-comet/pull/4057) (andygrove) +- doc: update documentation for cast and datetime functions [#4058](https://github.com/apache/datafusion-comet/pull/4058) (parthchandra) +- docs: add compatibility documentation to all expressions [#4067](https://github.com/apache/datafusion-comet/pull/4067) (andygrove) +- docs: rename SQL File Tests to Comet SQL Tests [#4108](https://github.com/apache/datafusion-comet/pull/4108) (andygrove) +- docs: add Understanding Comet Plans user guide page [#4086](https://github.com/apache/datafusion-comet/pull/4086) (andygrove) +- docs: support conditional content for snapshot vs release builds [#4030](https://github.com/apache/datafusion-comet/pull/4030) (andygrove) +- docs: update Spark version support and add version compatibility page [#4138](https://github.com/apache/datafusion-comet/pull/4138) (andygrove) +- docs: improve review skill and contributor guide for serde patterns [#4132](https://github.com/apache/datafusion-comet/pull/4132) (andygrove) +- docs: Fix errors in list of supported Spark versions [#4141](https://github.com/apache/datafusion-comet/pull/4141) (andygrove) +- docs: Update roadmap in contributor guide [#4144](https://github.com/apache/datafusion-comet/pull/4144) (andygrove) +- docs: add implement-comet-expression Claude skill [#4158](https://github.com/apache/datafusion-comet/pull/4158) (andygrove) +- docs: add roadmap items for spillable hash join, UDF support, memory management, and 1.0.0 [#4171](https://github.com/apache/datafusion-comet/pull/4171) (andygrove) +- docs: start Spark 4.1 known-limitations section, seeded with #4199 [#4202](https://github.com/apache/datafusion-comet/pull/4202) (andygrove) +- docs: document Spark 4 IntelliJ setup [#4198](https://github.com/apache/datafusion-comet/pull/4198) (yuboxx) +- docs: refresh Gluten comparison with ANSI, Spark 4, and Iceberg coverage [#4169](https://github.com/apache/datafusion-comet/pull/4169) (andygrove) +- docs: check off 53 implemented expressions in support doc [#4147](https://github.com/apache/datafusion-comet/pull/4147) (andygrove) +- docs: replace project logos with updated branding [#4230](https://github.com/apache/datafusion-comet/pull/4230) (pranamya123) +- docs: Documentation updates in preparation for 0.16 release [#4244](https://github.com/apache/datafusion-comet/pull/4244) (andygrove) + +**Other:** + +- chor: enable array_distinct [#3987](https://github.com/apache/datafusion-comet/pull/3987) (kazuyukitanimura) +- chore: Improve shuffle fallback logic [#3989](https://github.com/apache/datafusion-comet/pull/3989) (andygrove) +- chore: Start 0.16 development [#4016](https://github.com/apache/datafusion-comet/pull/4016) (andygrove) +- chore: update compatibility guide for primitive to string casts [#4012](https://github.com/apache/datafusion-comet/pull/4012) (parthchandra) +- test: add sql-file test confirming fallback on parquet variant reads [#4021](https://github.com/apache/datafusion-comet/pull/4021) (andygrove) +- chore: skip Iceberg and Spark SQL test workflows on test-only changes [#4023](https://github.com/apache/datafusion-comet/pull/4023) (andygrove) +- ci: force GNU ld on Linux to fix -ljvm linker error [#4024](https://github.com/apache/datafusion-comet/pull/4024) (andygrove) +- chore: update pending PR filter [#4025](https://github.com/apache/datafusion-comet/pull/4025) (comphead) +- test: run more Spark 4 tests [#4013](https://github.com/apache/datafusion-comet/pull/4013) (kazuyukitanimura) +- chore: update documentation links for 0.15.0 release [#4029](https://github.com/apache/datafusion-comet/pull/4029) (andygrove) +- test: run more JVM tests [#4026](https://github.com/apache/datafusion-comet/pull/4026) (kazuyukitanimura) +- feature: Support Spark expression: arrays_zip [#3643](https://github.com/apache/datafusion-comet/pull/3643) (hsiang-c) +- test: fix SparkToColumnar plan-shape assertions on Spark 4 [#4032](https://github.com/apache/datafusion-comet/pull/4032) (andygrove) +- test: unignore DynamicPartitionPruning static scan metrics test [#4038](https://github.com/apache/datafusion-comet/pull/4038) (andygrove) +- test: add non-AQE DPP edge case coverage for native Parquet scans [#4037](https://github.com/apache/datafusion-comet/pull/4037) (mbutrovich) +- test: unignore passing Spark 4.0 #3321 tests, retag remaining failures [#4041](https://github.com/apache/datafusion-comet/pull/4041) (andygrove) +- test: unignore passing DPP test, retag remaining failures [#4046](https://github.com/apache/datafusion-comet/pull/4046) (andygrove) +- chore: enable array_union [#4043](https://github.com/apache/datafusion-comet/pull/4043) (kazuyukitanimura) +- test: add date_trunc DST regression test in non-UTC session timezone [#4040](https://github.com/apache/datafusion-comet/pull/4040) (andygrove) +- ci: skip heavy test workflows for GenerateDocs and changelog changes [#4056](https://github.com/apache/datafusion-comet/pull/4056) (andygrove) +- chore(deps): bump the all-other-cargo-deps group in /native with 3 updates [#4061](https://github.com/apache/datafusion-comet/pull/4061) (dependabot[bot]) +- ci: add concurrency blocks to more workflows to cancel on new commit [#4064](https://github.com/apache/datafusion-comet/pull/4064) (mbutrovich) +- chore: gitignore generated docs directories [#4066](https://github.com/apache/datafusion-comet/pull/4066) (andygrove) +- chore: promote array_insert to compatible and consolidate expression docs [#4065](https://github.com/apache/datafusion-comet/pull/4065) (andygrove) +- test: re-enable sql_hive-1 for Spark 4.0 and fix two small failures [#4047](https://github.com/apache/datafusion-comet/pull/4047) (andygrove) +- ci: stopping running PR builds with 3.5.5/3.5.6 [#4103](https://github.com/apache/datafusion-comet/pull/4103) (andygrove) +- chore: Bump Spark 4.0.1 to 4.0.2 [#4114](https://github.com/apache/datafusion-comet/pull/4114) (andygrove) +- build: add spark-4.1 Maven profile and shim sources [#4097](https://github.com/apache/datafusion-comet/pull/4097) (andygrove) +- refactor: consolidate identical spark-4.0 and spark-4.1 shims into spark-4.x [#4118](https://github.com/apache/datafusion-comet/pull/4118) (andygrove) +- ci: enable Spark 4.1 PR test matrix [#4106](https://github.com/apache/datafusion-comet/pull/4106) (andygrove) +- build: add spark-4.2 Maven profile targeting 4.2.0-preview4 [#4119](https://github.com/apache/datafusion-comet/pull/4119) (andygrove) +- ci: reduce macOS PR matrix to single Spark 4.0 profile [#4104](https://github.com/apache/datafusion-comet/pull/4104) (andygrove) +- chore: audit array_intersect and expand SQL test coverage [#4071](https://github.com/apache/datafusion-comet/pull/4071) (andygrove) +- chore: support Spark 4.1/4.2 for golden files generation [#4134](https://github.com/apache/datafusion-comet/pull/4134) (comphead) +- ci: add Spark 4.0 / JDK 21 profile [#4060](https://github.com/apache/datafusion-comet/pull/4060) (james-willis) +- ci: Enable Comet PR test matrix and TPCDS plan-stability for Spark 4.2 [#4126](https://github.com/apache/datafusion-comet/pull/4126) (andygrove) +- test: support fallback chain in CometPlanStabilitySuite, dedupe existing goldens [#4129](https://github.com/apache/datafusion-comet/pull/4129) (andygrove) +- chore: Update `spark_expressions_support.md` doc [#4165](https://github.com/apache/datafusion-comet/pull/4165) (comphead) +- build: Enable Spark SQL tests for Spark 4.1.1 [#4093](https://github.com/apache/datafusion-comet/pull/4093) (andygrove) +- test: unignore InjectRuntimeFilterSuite tests gated on issue #242 [#4178](https://github.com/apache/datafusion-comet/pull/4178) (andygrove) +- chore(deps): bump cc from 1.2.60 to 1.2.61 in /native in the all-other-cargo-deps group [#4168](https://github.com/apache/datafusion-comet/pull/4168) (dependabot[bot]) +- test: [Spark 4.1.1] unignore SPARK-52921 union partitioning tests [#4195](https://github.com/apache/datafusion-comet/pull/4195) (andygrove) +- test: [Spark 4.1.1] unignore CachedBatchSerializerNoUnwrapSuite [#4204](https://github.com/apache/datafusion-comet/pull/4204) (andygrove) +- test: relax bytesRead ratio assertion for Spark 4.1+ [#4197](https://github.com/apache/datafusion-comet/pull/4197) (andygrove) +- deps: Bump OpenDAL to 0.56.0 [#4217](https://github.com/apache/datafusion-comet/pull/4217) (mbutrovich) +- ci: pin JDK per Spark version in Iceberg workflow matrix [#4120](https://github.com/apache/datafusion-comet/pull/4120) (dmefs) +- test: skip flaky StateStoreSuite under Comet and disambiguate JDK matrix names [#4226](https://github.com/apache/datafusion-comet/pull/4226) (andygrove) +- Map ProfileCredentialsProvider to profile credential chain [#4213](https://github.com/apache/datafusion-comet/pull/4213) (karuppayya) +- test: add explicit negative-count cases for array_repeat [#4182](https://github.com/apache/datafusion-comet/pull/4182) (slavlotski) +- test: add unhex dictionary coverage [#4222](https://github.com/apache/datafusion-comet/pull/4222) (yuboxx) +- chore: Enable GitHub project view [#4247](https://github.com/apache/datafusion-comet/pull/4247) (andygrove) +- test: dedupe redundant Spark 4.2 TPC-DS plan-stability golden files [#4243](https://github.com/apache/datafusion-comet/pull/4243) (andygrove) +- build: change default Maven profile to Spark 4.1 / Scala 2.13 [#4140](https://github.com/apache/datafusion-comet/pull/4140) (andygrove) +- chore: remove legacy ENABLE_COMET_SCAN_ONLY and ENABLE_COMET_ANSI_MODE env vars from Spark diffs [#4252](https://github.com/apache/datafusion-comet/pull/4252) (parthchandra) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 78 Andy Grove + 9 Matt Butrovich + 7 Parth Chandra + 6 KAZUYUKI TANIMURA + 5 ChenChen Lai + 5 Oleks V + 4 Rafael Fernández + 2 Karuppayya + 2 Yubo Xu + 2 dependabot[bot] + 1 Eric Chang + 1 James Willis + 1 Kuo-Hao Huang + 1 Liang-Chi Hsieh + 1 Pranamya Vadlamani + 1 Vladislav Zabolotsky + 1 hsiang-c +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/dev/changelog/0.2.0.md b/docs/source/changelog/0.2.0.md similarity index 100% rename from dev/changelog/0.2.0.md rename to docs/source/changelog/0.2.0.md diff --git a/dev/changelog/0.3.0.md b/docs/source/changelog/0.3.0.md similarity index 100% rename from dev/changelog/0.3.0.md rename to docs/source/changelog/0.3.0.md diff --git a/dev/changelog/0.4.0.md b/docs/source/changelog/0.4.0.md similarity index 100% rename from dev/changelog/0.4.0.md rename to docs/source/changelog/0.4.0.md diff --git a/dev/changelog/0.5.0.md b/docs/source/changelog/0.5.0.md similarity index 100% rename from dev/changelog/0.5.0.md rename to docs/source/changelog/0.5.0.md diff --git a/dev/changelog/0.6.0.md b/docs/source/changelog/0.6.0.md similarity index 100% rename from dev/changelog/0.6.0.md rename to docs/source/changelog/0.6.0.md diff --git a/dev/changelog/0.7.0.md b/docs/source/changelog/0.7.0.md similarity index 100% rename from dev/changelog/0.7.0.md rename to docs/source/changelog/0.7.0.md diff --git a/dev/changelog/0.8.0.md b/docs/source/changelog/0.8.0.md similarity index 100% rename from dev/changelog/0.8.0.md rename to docs/source/changelog/0.8.0.md diff --git a/dev/changelog/0.9.0.md b/docs/source/changelog/0.9.0.md similarity index 100% rename from dev/changelog/0.9.0.md rename to docs/source/changelog/0.9.0.md diff --git a/dev/changelog/0.9.1.md b/docs/source/changelog/0.9.1.md similarity index 100% rename from dev/changelog/0.9.1.md rename to docs/source/changelog/0.9.1.md diff --git a/docs/source/changelog/index.md b/docs/source/changelog/index.md new file mode 100644 index 0000000000..d8a02c060d --- /dev/null +++ b/docs/source/changelog/index.md @@ -0,0 +1,45 @@ + + +# Changelog + +Per-release change logs for Apache DataFusion Comet. + +```{toctree} +:maxdepth: 1 + +0.16.0 +0.15.0 +0.14.1 +0.14.0 +0.13.0 +0.12.0 +0.11.0 +0.10.0 +0.9.1 +0.9.0 +0.8.0 +0.7.0 +0.6.0 +0.5.0 +0.4.0 +0.3.0 +0.2.0 +0.1.0 +``` diff --git a/docs/source/contributor-guide/adding_a_new_spark_version.md b/docs/source/contributor-guide/adding_a_new_spark_version.md index 1f16e10153..65109e4d0c 100644 --- a/docs/source/contributor-guide/adding_a_new_spark_version.md +++ b/docs/source/contributor-guide/adding_a_new_spark_version.md @@ -138,9 +138,7 @@ own test suites under the new profile. Promote the new Spark version from the compile-only job to the main test jobs in `.github/workflows/pr_build_linux.yml` (and `pr_build_macos.yml` if -capacity allows). Use `scan_impl: "auto"` so both `native_datafusion` and -`native_iceberg_compat` get exercised, matching how earlier versions are -configured. +capacity allows). Match how earlier versions are configured. ### Run the Suite Locally First @@ -256,14 +254,12 @@ new-version bring-up are: ### CI for the Spark SQL Tests Spark SQL tests do not run from the main PR build workflows. They have -their own dedicated workflow files: +their own dedicated workflow file: - `.github/workflows/spark_sql_test.yml` -- `.github/workflows/spark_sql_test_native_iceberg_compat.yml` -Add the new version to the matrix in each of these files (`spark-short`, -`spark-full`, `java`, `scan-impl`). Use the closest existing entry as a -template. +Add the new version to the matrix (`spark-short`, `spark-full`, `java`). +Use the closest existing entry as a template. Before merging, run `make format`, run clippy (`cd native && cargo clippy --all-targets --workspace -- -D warnings`), and diff --git a/docs/source/contributor-guide/bug_triage.md b/docs/source/contributor-guide/bug_triage.md index 9e51f44637..477829bae4 100644 --- a/docs/source/contributor-guide/bug_triage.md +++ b/docs/source/contributor-guide/bug_triage.md @@ -73,8 +73,7 @@ help contributors find bugs in their area of expertise. | `area:ffi` | Arrow FFI / JNI boundary | | `area:ci` | CI/CD, GitHub Actions, build tooling | -The following pre-existing labels also serve as area indicators: `native_datafusion`, -`native_iceberg_compat`, `spark 4`, `spark sql tests`. +The following pre-existing labels also serve as area indicators: `spark 4`, `spark sql tests`. ## Triage Process @@ -109,9 +108,8 @@ Periodically review open bugs to ensure priorities are still accurate: crashes, because crashes are at least visible. 2. **User-reported over test-only.** A bug hit by a real user on a real workload takes priority over one found only in test suites. -3. **Core path over experimental.** Bugs in the default scan mode (`native_comet`) or widely-used - expressions take priority over bugs in experimental features like `native_datafusion` or - `native_iceberg_compat`. +3. **Core path over experimental.** Bugs in widely-used expressions and operators take priority over + bugs in experimental features. 4. **Production safety over feature completeness.** Fixing a data corruption bug is more important than adding support for a new expression. diff --git a/docs/source/contributor-guide/release_process.md b/docs/source/contributor-guide/release_process.md index 23d2b91105..8490ec84cf 100644 --- a/docs/source/contributor-guide/release_process.md +++ b/docs/source/contributor-guide/release_process.md @@ -150,7 +150,7 @@ example generates a change log of all changes between the previous version and t ```shell export GITHUB_TOKEN= -python3 generate-changelog.py 0.12.0 HEAD 0.13.0 > ../changelog/0.13.0.md +python3 generate-changelog.py 0.12.0 HEAD 0.13.0 > ../../docs/source/changelog/0.13.0.md ``` Create a PR against the _main_ branch to add this change log and once this is approved and merged, cherry-pick the diff --git a/docs/source/contributor-guide/spark_expressions_support.md b/docs/source/contributor-guide/spark_expressions_support.md index cb27a439b9..65f941210a 100644 --- a/docs/source/contributor-guide/spark_expressions_support.md +++ b/docs/source/contributor-guide/spark_expressions_support.md @@ -215,12 +215,12 @@ ### datetime_funcs - [ ] add_months -- [ ] convert_timezone +- [x] convert_timezone - [ ] curdate - [ ] current_date - [ ] current_time - [ ] current_timestamp -- [ ] current_timezone +- [x] current_timezone - [x] date_add - [x] date_diff - [x] date_format @@ -238,10 +238,14 @@ - [x] dayofyear - [x] extract - [x] from_unixtime -- [ ] from_utc_timestamp +- [x] from_utc_timestamp + - Spark 3.4.3 (audited 2026-05-12): identical to 3.5.8. + - Spark 3.5.8 (audited 2026-05-12): baseline. + - Spark 4.0.1 (audited 2026-05-12): `inputTypes` widened to `StringTypeWithCollation`; behaviour unchanged for ASCII timezone strings. + - Known divergence: Comet's native timezone parser does not accept Spark's legacy zone forms (`GMT+1`, `UTC+1`, three-letter abbreviations like `PST`). Such timezones throw a native parse error at execution. - [x] hour - [x] last_day -- [ ] localtimestamp +- [x] localtimestamp - [x] make_date - [ ] make_dt_interval - [ ] make_interval @@ -270,7 +274,11 @@ - [ ] to_timestamp_ltz - [ ] to_timestamp_ntz - [ ] to_unix_timestamp -- [ ] to_utc_timestamp +- [x] to_utc_timestamp + - Spark 3.4.3 (audited 2026-05-12): identical to 3.5.8. + - Spark 3.5.8 (audited 2026-05-12): baseline. + - Spark 4.0.1 (audited 2026-05-12): `inputTypes` widened to `StringTypeWithCollation`; behaviour unchanged for ASCII timezone strings. + - Known divergence: Comet's native timezone parser does not accept Spark's legacy zone forms (`GMT+1`, `UTC+1`, three-letter abbreviations like `PST`). Such timezones throw a native parse error at execution. - [x] trunc - [ ] try_make_interval - [ ] try_make_timestamp @@ -371,13 +379,16 @@ - [x] cos - [x] cosh - [x] cot -- [ ] csc +- [x] csc - [x] degrees - [x] div - [ ] e - [x] exp - [x] expm1 -- [ ] factorial +- [x] factorial + - 3.4.3 (audited 2026-05-15): identical to v3.5.8. + - 3.5.8 (audited 2026-05-15): canonical reference; `extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant`. Returns NULL for NULL input or values outside `[0, 20]`. + - 4.0.1 (audited 2026-05-15): `NullIntolerant` trait replaced by `nullIntolerant: Boolean` method override; behavior unchanged. - [x] floor - [x] greatest - [x] hex @@ -400,9 +411,9 @@ - [x] randn - [ ] random - [ ] randstr -- [ ] rint +- [x] rint - [x] round -- [ ] sec +- [x] sec - [x] shiftleft - [x] sign - [x] signum @@ -517,7 +528,7 @@ - [ ] collation - [x] concat_ws - [x] contains -- [ ] decode +- [x] decode - [ ] elt - [ ] encode - [x] endswith @@ -585,7 +596,7 @@ ### url_funcs -- [ ] parse_url +- [x] parse_url (Incompatible: native diverges from Spark on edge cases) - [x] try_url_decode - 4.0.1, 2026-05-05 - [x] url_decode diff --git a/docs/source/index.md b/docs/source/index.md index 7424cffcc7..d1ea261824 100644 --- a/docs/source/index.md +++ b/docs/source/index.md @@ -107,6 +107,8 @@ shaping the future of Comet. Check out our User Guide Contributor Guide +Changelog Comparison with Gluten +Versioning Policy ASF Links ``` diff --git a/docs/source/user-guide/latest/compatibility/expressions/index.md b/docs/source/user-guide/latest/compatibility/expressions/index.md index b86e46cc0e..fcaa3bfbbc 100644 --- a/docs/source/user-guide/latest/compatibility/expressions/index.md +++ b/docs/source/user-guide/latest/compatibility/expressions/index.md @@ -36,5 +36,6 @@ math misc string struct +url cast ``` diff --git a/docs/source/user-guide/latest/compatibility/expressions/url.md b/docs/source/user-guide/latest/compatibility/expressions/url.md new file mode 100644 index 0000000000..765e12941f --- /dev/null +++ b/docs/source/user-guide/latest/compatibility/expressions/url.md @@ -0,0 +1,23 @@ + + +# URL Expressions + + + diff --git a/docs/source/user-guide/latest/compatibility/index.md b/docs/source/user-guide/latest/compatibility/index.md index 1ba9d9e181..59c6a906f1 100644 --- a/docs/source/user-guide/latest/compatibility/index.md +++ b/docs/source/user-guide/latest/compatibility/index.md @@ -23,7 +23,7 @@ Comet aims to provide consistent results with the version of Apache Spark that i This guide documents areas where Comet's behavior is known to differ from Spark. Topics are grouped by subsystem: -- **Parquet**: limitations when reading Parquet files (both scan implementations, shared and per-implementation). +- **Parquet**: limitations when reading Parquet files. - **Floating-point comparison**: NaN and signed-zero handling in comparisons. - **Regular expressions**: differences between the Rust regexp crate and Java's regex engine. - **Operators**: operator-level compatibility notes, including window functions and round-robin partitioning. diff --git a/docs/source/user-guide/latest/compatibility/scans.md b/docs/source/user-guide/latest/compatibility/scans.md index d713dd4e8f..37524a829e 100644 --- a/docs/source/user-guide/latest/compatibility/scans.md +++ b/docs/source/user-guide/latest/compatibility/scans.md @@ -19,22 +19,13 @@ under the License. # Parquet Compatibility -Comet currently has two distinct implementations of the Parquet scan operator. +Comet's Parquet scan offloads decoding to native code and produces Arrow batches for the rest of +the plan. Comet falls back to Spark when the scan cannot be converted (for example, due to one of +the unsupported features listed below). -| Scan Implementation | Notes | -| ----------------------- | ---------------------- | -| `native_datafusion` | Fully native scan | -| `native_iceberg_compat` | Hybrid JVM/native scan | +## Parquet Scan Limitations -The configuration property `spark.comet.scan.impl` is used to select an implementation. The default setting is -`spark.comet.scan.impl=auto`, which attempts to use `native_datafusion` first, and falls back to Spark if the scan -cannot be converted (e.g., due to unsupported features). Most users should not need to change this setting. However, -it is possible to force Comet to use a particular implementation for all scan operations by setting this -configuration property to one of the following implementations. For example: `--conf spark.comet.scan.impl=native_datafusion`. - -## Shared Limitations - -The following features are not supported by either scan implementation, and Comet will fall back to Spark in these scenarios: +The following features are not supported and cause Comet to fall back to Spark: - Decimals encoded in binary format. - `ShortType` columns, by default. When reading Parquet files written by systems other than Spark that contain @@ -46,17 +37,30 @@ The following features are not supported by either scan implementation, and Come columns are always safe because they can only come from signed `INT8`, where truncation preserves the signed value. - Default values that are nested types (e.g., maps, arrays, structs). Literal default values are supported. - Spark's Datasource V2 API. When `spark.sql.sources.useV1SourceList` does not include `parquet`, Spark uses the - V2 API for Parquet scans. The DataFusion-based implementations only support the V1 API. + V2 API for Parquet scans. Comet's Parquet scan only supports the V1 API. - Spark metadata columns (e.g., `_metadata.file_path`) +- No support for row indexes +- No support for `input_file_name()`, `input_file_block_start()`, or `input_file_block_length()` SQL functions. + Comet's Parquet scan does not use Spark's `FileScanRDD`, so these functions cannot populate their values. +- No support for `ignoreMissingFiles` or `ignoreCorruptFiles` being set to `true` +- Duplicate field names in case-insensitive mode (e.g., a Parquet file with both `B` and `b` columns) + are detected at read time and raise a `SparkRuntimeException` with error class `_LEGACY_ERROR_TEMP_2093`, + matching Spark's behavior. +- `spark.sql.parquet.enableVectorizedReader=false`. Disabling the vectorized reader opts into + Spark's parquet-mr semantics (silent overflow, null-on-narrowing), which Comet's native reader + does not replicate. By default Comet falls back to Spark in this case. Set + `spark.comet.scan.allowDisabledParquetVectorizedReader=true` to opt in to running the + Comet Parquet scan regardless. See + [#4352](https://github.com/apache/datafusion-comet/issues/4352). -The following shared limitation may produce incorrect results without falling back to Spark: +The following limitation may produce incorrect results without falling back to Spark: - No support for datetime rebasing. When reading Parquet files containing dates or timestamps written before Spark 3.0 (which used a hybrid Julian/Gregorian calendar), dates/timestamps will be read as if they were written using the Proleptic Gregorian calendar. This may produce incorrect results for dates before October 15, 1582. -The following shared limitation raises an error at scan time rather than falling back to Spark: +The following limitation raises an error at scan time rather than falling back to Spark: - Invalid UTF-8 bytes in `STRING` columns. Spark permits arbitrary byte sequences in a `STRING` column (for example from `CAST(X'C1' AS STRING)`), but Comet's native execution path is built on @@ -65,22 +69,7 @@ The following shared limitation raises an error at scan time rather than falling query, or cast the column to `BINARY` before persisting, if you need to preserve non-UTF-8 bytes. See [#4121](https://github.com/apache/datafusion-comet/issues/4121). -## `native_datafusion` Limitations - -The `native_datafusion` scan has some additional limitations, mostly related to Parquet metadata. All of these -cause Comet to fall back to Spark (including when using `auto` mode). Note that the `native_datafusion` scan -requires `spark.comet.exec.enabled=true` because the scan node must be wrapped by `CometExecRule`. - -- No support for row indexes -- No support for reading Parquet field IDs -- No support for `input_file_name()`, `input_file_block_start()`, or `input_file_block_length()` SQL functions. - The `native_datafusion` scan does not use Spark's `FileScanRDD`, so these functions cannot populate their values. -- No support for `ignoreMissingFiles` or `ignoreCorruptFiles` being set to `true` -- Duplicate field names in case-insensitive mode (e.g., a Parquet file with both `B` and `b` columns) - are detected at read time and raise a `SparkRuntimeException` with error class `_LEGACY_ERROR_TEMP_2093`, - matching Spark's behavior. - -The following `native_datafusion` limitations may produce incorrect results on Spark versions prior to 4.0 +The following limitation may produce incorrect results on Spark versions prior to 4.0 without falling back to Spark: - Reading `TimestampLTZ` as `TimestampNTZ`. On Spark 3.x, Spark raises an error per @@ -91,18 +80,42 @@ without falling back to Spark: ([SPARK-47447](https://issues.apache.org/jira/browse/SPARK-47447)) and Comet matches Spark's behavior. See [#4219](https://github.com/apache/datafusion-comet/issues/4219). -- Unsupported Parquet type conversions. Spark raises schema incompatibility errors for certain conversions - (e.g., reading INT32 as BIGINT, reading BINARY as TIMESTAMP, unsupported decimal precision changes). - The `native_datafusion` scan may not detect these mismatches and could return unexpected values instead - of raising an error. See [#3720](https://github.com/apache/datafusion-comet/issues/3720). - -## `native_iceberg_compat` Limitations - -The `native_iceberg_compat` scan has the following additional limitation that may produce incorrect results -without falling back to Spark: - -- Some Spark configuration values are hard-coded to their defaults rather than respecting user-specified values. - This may produce incorrect results when non-default values are set. The affected configurations are - `spark.sql.parquet.binaryAsString`, `spark.sql.parquet.int96AsTimestamp`, `spark.sql.caseSensitive`, - `spark.sql.parquet.inferTimestampNTZ.enabled`, and `spark.sql.legacy.parquet.nanosAsLong`. See - [issue #1816](https://github.com/apache/datafusion-comet/issues/1816) for more details. +### Schema Mismatch Handling + +The issues in this subsection apply only when the requested read schema differs from the schema written +to the Parquet file. They do **not** affect a plain `spark.read.parquet(path)` that infers the schema +from file metadata, because in that case the requested schema and file schema match by construction. +Schema mismatch happens in two real-world scenarios: + +1. The user provides an explicit read schema: `spark.read.schema().parquet(path)` (or the + equivalent DataFrame API). +2. **Schema evolution / partitioned reads** where files in a single dataset were written at different + times with different types, or a table-format catalog (Iceberg, Delta) records a logical schema + that has evolved past one or more underlying Parquet files. Spark coerces the file types to the + table types at read time. + +Spark's vectorized Parquet reader fully validates these conversions in `ParquetVectorUpdaterFactory.getUpdater` +and throws `SchemaColumnConvertNotSupportedException` for unsupported pairs. Comet's Parquet scan +mirrors that validation in its schema adapter; the entries below are the remaining gaps. + +Note that the exact set of accepted conversions has changed between Spark versions +(for example, Spark 3.x's `schemaEvolution.enabled` flag gates `INT32 → INT64`, `FLOAT → DOUBLE`, +and `INT32 → DOUBLE` widening that Spark 4.0+ accepts unconditionally; `TimestampLTZ → TimestampNTZ` +is rejected by Spark 3.x but accepted by Spark 4.0+). Comet aims to follow the per-version Spark +behavior. + +- **`ParquetSchemaConvert` errors do not include the file path**. The mismatch itself is detected and + rejected correctly, but the resulting Spark error message reads + `Encountered error while reading file . Data type mismatches…` (note the empty path). Behavior is + consistent across Spark versions. See + [#4316](https://github.com/apache/datafusion-comet/issues/4316). +- **Spark 3.x: extra `SparkException` layer in the cause chain**. The native error is translated to a + `SparkException` whose cause is `SchemaColumnConvertNotSupportedException` (matching what Spark would + throw); on Spark 3.x the executor / task error handling re-wraps this once more on the way back to + the driver, producing a two-level chain (`SparkException → SparkException → +SchemaColumnConvertNotSupportedException`) instead of the one-level chain Spark's own vectorized + reader produces. Code that catches `SparkException` and inspects only the immediate cause via + `e.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]` will see the inner + `SparkException` instead. Walk the cause chain to recover the + `SchemaColumnConvertNotSupportedException`. Spark 4.0+ produces a single-level chain, matching + vanilla Spark. See [#4354](https://github.com/apache/datafusion-comet/issues/4354). diff --git a/docs/source/user-guide/latest/compatibility/spark-versions.md b/docs/source/user-guide/latest/compatibility/spark-versions.md index 8c17fd6b13..4856cf5a1b 100644 --- a/docs/source/user-guide/latest/compatibility/spark-versions.md +++ b/docs/source/user-guide/latest/compatibility/spark-versions.md @@ -31,13 +31,13 @@ Spark 3.4.3 is supported with Java 11/17 and Scala 2.12/2.13. ### Known Limitations - **Reading `TimestampLTZ` as `TimestampNTZ`**: Spark 3.4 raises an error for this operation - (SPARK-36182), but Comet's `native_datafusion` scan silently returns the raw UTC value instead. - See [Parquet Compatibility](scans.md#native_datafusion-limitations) for details. + (SPARK-36182), but Comet's Parquet scan silently returns the raw UTC value instead. + See [Parquet Compatibility](scans.md#parquet-scan-limitations) for details. - **Unsupported Parquet type conversions**: Spark 3.4 raises schema incompatibility errors for certain type mismatches (e.g., reading INT32 as BIGINT, decimal precision changes), but Comet's - `native_datafusion` scan may not detect these and could return unexpected values. - See [Parquet Compatibility](scans.md#native_datafusion-limitations) for details. + Comet's Parquet scan may not detect these and could return unexpected values. + See [Parquet Compatibility](scans.md#parquet-scan-limitations) for details. ## Spark 3.5 @@ -46,13 +46,13 @@ Spark 3.5.8 is supported with Java 11/17 and Scala 2.12/2.13. ### Known Limitations - **Reading `TimestampLTZ` as `TimestampNTZ`**: Spark 3.5 raises an error for this operation - (SPARK-36182), but Comet's `native_datafusion` scan silently returns the raw UTC value instead. - See [Parquet Compatibility](scans.md#native_datafusion-limitations) for details. + (SPARK-36182), but Comet's Parquet scan silently returns the raw UTC value instead. + See [Parquet Compatibility](scans.md#parquet-scan-limitations) for details. - **Unsupported Parquet type conversions**: Spark 3.5 raises schema incompatibility errors for certain type mismatches (e.g., reading INT32 as BIGINT, decimal precision changes), but Comet's - `native_datafusion` scan may not detect these and could return unexpected values. - See [Parquet Compatibility](scans.md#native_datafusion-limitations) for details. + Comet's Parquet scan may not detect these and could return unexpected values. + See [Parquet Compatibility](scans.md#parquet-scan-limitations) for details. ## Spark 4.0 diff --git a/docs/source/user-guide/latest/datasources.md b/docs/source/user-guide/latest/datasources.md index 065b719ba5..f1d20fa3a4 100644 --- a/docs/source/user-guide/latest/datasources.md +++ b/docs/source/user-guide/latest/datasources.md @@ -61,11 +61,9 @@ Comet supports most standard storage systems, such as local file system and obje Apache DataFusion Comet native reader seamlessly scans files from remote HDFS for [supported formats](#supported-spark-data-sources) -### Using experimental native DataFusion reader +### Building Comet with HDFS support -Unlike to native Comet reader the Datafusion reader fully supports nested types processing. This reader is currently experimental only - -To build Comet with native DataFusion reader and remote HDFS support it is required to have a JDK installed +To build Comet with remote HDFS support it is required to have a JDK installed. Example: Build a Comet for `spark-4.1` provide a JDK path in `JAVA_HOME` @@ -76,11 +74,10 @@ export JAVA_HOME="/opt/homebrew/opt/openjdk@17" make release PROFILES="-Pspark-4.1" COMET_FEATURES=hdfs RUSTFLAGS="-L $JAVA_HOME/libexec/openjdk.jdk/Contents/Home/lib/server" ``` -Start Comet with experimental reader and HDFS support as [described](installation.md/#run-spark-shell-with-comet-enabled) +Start Comet with HDFS support as [described](installation.md/#run-spark-shell-with-comet-enabled) and add additional parameters ```shell ---conf spark.comet.scan.impl=native_datafusion \ --conf spark.hadoop.fs.defaultFS="hdfs://namenode:9000" \ --conf spark.hadoop.dfs.client.use.datanode.hostname = true \ --conf dfs.client.use.datanode.hostname = true @@ -158,7 +155,6 @@ JAVA_HOME="/opt/homebrew/opt/openjdk@17" make release PROFILES="-Pspark-4.1" COM withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", "fs.defaultFS" -> "hdfs://namenode:9000", "dfs.client.use.datanode.hostname" -> "true") { @@ -169,14 +165,15 @@ JAVA_HOME="/opt/homebrew/opt/openjdk@17" make release PROFILES="-Pspark-4.1" COM } ``` -Or use `spark-shell` with HDFS support as described [above](#using-experimental-native-datafusion-reader) +Or use `spark-shell` with HDFS support as described [above](#building-comet-with-hdfs-support) ## S3 -The `native_datafusion` and `native_iceberg_compat` Parquet scan implementations completely offload data loading -to native code. They use the [`object_store` crate](https://crates.io/crates/object_store) to read data from S3 and -support configuring S3 access using standard [Hadoop S3A configurations](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#General_S3A_Client_configuration) by translating them to -the `object_store` crate's format. +Comet's Parquet scan completely offloads data loading to native code. It uses the +[`object_store` crate](https://crates.io/crates/object_store) to read data from S3 and supports +configuring S3 access using standard +[Hadoop S3A configurations](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#General_S3A_Client_configuration) +by translating them to the `object_store` crate's format. This implementation maintains compatibility with existing Hadoop S3A configurations, so existing code will continue to work as long as the configurations are supported and can be translated without loss of functionality. @@ -210,8 +207,7 @@ Multiple credential providers can be specified in a comma-separated list using t ### Additional S3 Configuration Options -Beyond credential providers, the `native_datafusion` and `native_iceberg_compat` implementations support additional -S3 configuration options: +Beyond credential providers, Comet's Parquet scan supports additional S3 configuration options: | Option | Description | | ------------------------------- | -------------------------------------------------------------------------------------------------- | @@ -224,8 +220,7 @@ All configuration options support bucket-specific overrides using the pattern `f ### Examples -The following examples demonstrate how to configure S3 access with the `native_datafusion` and `native_iceberg_compat` -Parquet scan implementations using different authentication methods. +The following examples demonstrate how to configure S3 access using different authentication methods. **Example 1: Simple Credentials** @@ -234,7 +229,6 @@ This example shows how to access a private S3 bucket using an access key and sec ```shell $SPARK_HOME/bin/spark-shell \ ... ---conf spark.comet.scan.impl=native_datafusion \ --conf spark.hadoop.fs.s3a.access.key=my-access-key \ --conf spark.hadoop.fs.s3a.secret.key=my-secret-key ... @@ -247,7 +241,6 @@ This example demonstrates using an assumed role credential to access a private S ```shell $SPARK_HOME/bin/spark-shell \ ... ---conf spark.comet.scan.impl=native_datafusion \ --conf spark.hadoop.fs.s3a.aws.credentials.provider=org.apache.hadoop.fs.s3a.auth.AssumedRoleCredentialProvider \ --conf spark.hadoop.fs.s3a.assumed.role.arn=arn:aws:iam::123456789012:role/my-role \ --conf spark.hadoop.fs.s3a.assumed.role.session.name=my-session \ @@ -257,7 +250,7 @@ $SPARK_HOME/bin/spark-shell \ ### Limitations -The S3 support of `native_datafusion` and `native_iceberg_compat` has the following limitations: +Comet's S3 support has the following limitations: 1. **Partial Hadoop S3A configuration support**: Not all Hadoop S3A configurations are currently supported. Only the configurations listed in the tables above are translated and applied to the underlying `object_store` crate. diff --git a/docs/source/user-guide/latest/expressions.md b/docs/source/user-guide/latest/expressions.md index 668081d257..a610a83cea 100644 --- a/docs/source/user-guide/latest/expressions.md +++ b/docs/source/user-guide/latest/expressions.md @@ -63,6 +63,7 @@ of expressions that be disabled. | Concat | | ConcatWs | | Contains | +| Decode | | EndsWith | | InitCap | | Left | @@ -100,6 +101,8 @@ of expressions that be disabled. | Expression | SQL | | ---------------- | ---------------------------- | +| ConvertTimezone | `convert_timezone` | +| CurrentTimeZone | `current_timezone` | | DateAdd | `date_add` | | DateDiff | `datediff` | | DateFormat | `date_format` | @@ -111,7 +114,9 @@ of expressions that be disabled. | FromUnixTime | `from_unixtime` | | Hour | `hour` | | LastDay | `last_day` | +| LocalTimestamp | `localtimestamp` | | MakeDate | `make_date` | +| MakeTime | `make_time` | | Minute | `minute` | | NextDay | `next_day` | | Second | `second` | @@ -128,6 +133,8 @@ of expressions that be disabled. | DayOfYear | `dayofyear` | | WeekOfYear | `weekofyear` | | Quarter | `quarter` | +| ToTime | `to_time` | +| TryToTime | `try_to_time` | ## Math Expressions @@ -149,9 +156,11 @@ of expressions that be disabled. | Cos | `cos` | | Cosh | `cosh` | | Cot | `cot` | +| Csc | `csc` | | Divide | `/` | | Exp | `exp` | | Expm1 | `expm1` | +| Factorial | `factorial` | | Floor | `floor` | | Hex | `hex` | | IntegralDivide | `div` | @@ -165,7 +174,9 @@ of expressions that be disabled. | Rand | `rand` | | Randn | `randn` | | Remainder | `%` | +| Rint | `rint` | | Round | `round` | +| Sec | `sec` | | Signum | `signum` | | Sin | `sin` | | Sinh | `sinh` | diff --git a/docs/source/user-guide/latest/iceberg.md b/docs/source/user-guide/latest/iceberg.md index 5c63ae9ad6..f22180ec77 100644 --- a/docs/source/user-guide/latest/iceberg.md +++ b/docs/source/user-guide/latest/iceberg.md @@ -146,6 +146,24 @@ The following scenarios will fall back to Spark's native Iceberg reader: - Dynamic Partition Pruning under Adaptive Query Execution (non-AQE DPP is supported); see [#3510](https://github.com/apache/datafusion-comet/issues/3510) +### Iceberg UDFs + +Iceberg ships several `ScalaUDF`s that surface in user queries and maintenance actions: + +- `IcebergSpark.registerBucketUDF` and `registerTruncateUDF` register `bucket(N, col)` and + `truncate(W, col)` for use in `SELECT` / `JOIN` / `WHERE` predicates that align with hidden + partitioning. +- `RewriteDataFiles` with `sort-strategy=zorder` builds a tree of per-type ordered-bytes UDFs + (`INT_ORDERED_BYTES`, `LONG_ORDERED_BYTES`, ..., `INTERLEAVE_BYTES`) over the sort key columns + during compaction. + +By default these UDFs cause the enclosing operator to fall back to Spark, which forces a +columnar-to-row roundtrip and demotes the surrounding shuffle from `CometExchange` to +`CometColumnarExchange`. Enabling the experimental +[Scala UDF and Java UDF Support](scala_java_udfs.md) feature +(`spark.comet.exec.scalaUDF.codegen.enabled=true`) routes these UDFs through native execution so +the project, exchange, and sort operators around them stay on the Comet path end-to-end. + ### Task input metrics The native Iceberg reader populates Spark's task-level `inputMetrics.bytesRead` (visible in the Spark UI Stages tab) using the `bytes_read` counter from iceberg-rust's `ScanMetrics`. This counter includes bytes read from both data files and delete files. diff --git a/docs/source/user-guide/latest/index.rst b/docs/source/user-guide/latest/index.rst index 314a0a51bd..9587b2ee03 100644 --- a/docs/source/user-guide/latest/index.rst +++ b/docs/source/user-guide/latest/index.rst @@ -43,6 +43,7 @@ to read more. Supported Data Types Supported Operators Supported Expressions + ScalaUDF and Java UDF Support Configuration Settings Compatibility Guide Understanding Comet Plans diff --git a/docs/source/user-guide/latest/operators.md b/docs/source/user-guide/latest/operators.md index 1b8f78d9c6..495141f9bb 100644 --- a/docs/source/user-guide/latest/operators.md +++ b/docs/source/user-guide/latest/operators.md @@ -30,7 +30,7 @@ not supported by Comet will fall back to regular Spark execution. | ExpandExec | Yes | | | FileSourceScanExec | Yes | Supports Parquet files. See the [Comet Compatibility Guide] for more information. | | FilterExec | Yes | | -| GenerateExec | Yes | Supports `explode` generator only. | +| GenerateExec | Yes | Supports `explode` and `posexplode` generators (arrays only, `_outer` variants are incompatible). | | GlobalLimitExec | Yes | | | HashAggregateExec | Yes | | | InsertIntoHadoopFsRelationCommand | No | Experimental support for native Parquet writes. Disabled by default. | diff --git a/docs/source/user-guide/latest/scala_java_udfs.md b/docs/source/user-guide/latest/scala_java_udfs.md new file mode 100644 index 0000000000..e8163e494c --- /dev/null +++ b/docs/source/user-guide/latest/scala_java_udfs.md @@ -0,0 +1,61 @@ + + +# Scala UDF and Java UDF Support + +Comet executes Spark's Scala and Java [scalar user-defined functions (UDFs)](https://spark.apache.org/docs/latest/sql-ref-functions-udf-scalar.html) on the native Comet path. The presence of a UDF does not force the enclosing operator off the native path; surrounding native operators stay native. + +This page covers Spark's `ScalaUDF` (Scala `udf(...)`, `spark.udf.register(...)` over Scala or Java functional interfaces, and SQL `CREATE FUNCTION ... AS 'com.example.MyUDF'`). Other UDF kinds (Python / Pandas, Hive, aggregate) are out of scope and continue to fall back to Spark. + +This feature is experimental and disabled by default. + +## Configuration + +| Key | Default | Description | +| ------------------------------------------- | ------- | ------------------------------------------------------------------------------------------------------------------ | +| `spark.comet.exec.scalaUDF.codegen.enabled` | `false` | When `true`, eligible `ScalaUDF`s run on the Comet path. When `false`, the enclosing operator falls back to Spark. | + +## Supported + +- User functions registered via `udf(...)`, `spark.udf.register(...)` (Scala or Java functional interfaces), or SQL `CREATE FUNCTION ... AS 'com.example.MyUDF'`. +- Scalar input/output types: `Boolean`, `Byte`, `Short`, `Int`, `Long`, `Float`, `Double`, `Decimal`, `String`, `Binary`, `Date`, `Timestamp`, `TimestampNTZ`. +- Complex input/output types with arbitrary nesting: `ArrayType`, `StructType`, `MapType`. +- Composition with other Catalyst expressions inside the argument tree (e.g. `myUdf(upper(s))` runs as one native unit). +- Higher-order functions (`transform`, `filter`, `exists`, `aggregate`, `zip_with`, `map_filter`, `map_zip_with`, etc.) inside the argument tree. + +## Not supported + +- Aggregate UDFs (`ScalaAggregator`, `TypedImperativeAggregate`, the legacy `UserDefinedAggregateFunction`). +- Table UDFs and generators. +- Python `@udf` and Pandas `@pandas_udf`. +- Hive `GenericUDF` and `SimpleUDF`. +- `CalendarIntervalType`, `NullType`, and `UserDefinedType` arguments and return types. UDT-typed columns fall back to Spark; for native execution, store and read the underlying representation directly (e.g. write MLlib `Vector` outputs as `Struct, values: Array>` rather than `VectorUDT`). +- Trees whose total nested-field count (output plus all input columns the UDF tree references) exceeds `spark.sql.codegen.maxFields` (default 100). Comet refuses these at plan time and the operator falls back to Spark. + +When a UDF is rejected, the reason surfaces through Comet's standard fallback diagnostics; the query still runs on Spark. + +## Behavior + +- Non-deterministic expressions referenced from the argument tree (`rand`, `uuid`, `monotonically_increasing_id`) produce per-partition sequences consistent with Spark. +- `TaskContext.get()` inside the user function returns the driving Spark task's context. +- The user function must be closure-serializable; the same function that works with Spark's executor execution works here. + +## Known limitations + +- Each query containing a ScalaUDF pays a one-time codegen cost on its first batch and reuses the compiled kernel for subsequent batches, matching Spark's whole-stage codegen behavior. Bytecode is deduped JVM-wide via the same `CodeGenerator` cache, so structurally identical queries across a session share the compiled class. diff --git a/docs/source/user-guide/latest/understanding-comet-plans.md b/docs/source/user-guide/latest/understanding-comet-plans.md index f66f385207..7fb93c8f53 100644 --- a/docs/source/user-guide/latest/understanding-comet-plans.md +++ b/docs/source/user-guide/latest/understanding-comet-plans.md @@ -145,34 +145,33 @@ by role. Names match what is shown in the plan output. ### Scans -| Node | Description | -| ------------------------ | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| `CometScan` | V1 Parquet scan driven by Spark's file-source path through Comet's Parquet reader. Decoding runs in native code; the resulting Arrow batches cross JNI into the native plan. The active scan implementation is shown in brackets, e.g. `CometScan [native_iceberg_compat]`. | -| `CometBatchScan` | DataSource V2 scan, including Iceberg Parquet, that produces Arrow batches consumed by Comet. | -| `CometNativeScan` | Fully native Parquet scan that runs entirely in DataFusion (no JVM Parquet reader involvement). | -| `CometIcebergNativeScan` | Fully native Iceberg Parquet scan. | -| `CometCsvNativeScan` | Fully native CSV scan (experimental). | +| Node | Description | +| ------------------------ | --------------------------------------------------------------------------------------------- | +| `CometBatchScan` | DataSource V2 scan, including Iceberg Parquet, that produces Arrow batches consumed by Comet. | +| `CometNativeScan` | Fully native Parquet scan that runs entirely in DataFusion. | +| `CometIcebergNativeScan` | Fully native Iceberg Parquet scan. | +| `CometCsvNativeScan` | Fully native CSV scan (experimental). | ### Native Execution Operators These run natively in DataFusion. When several appear consecutively in a plan, they execute as a single fused native block. -| Node | Spark equivalent | -| ---------------------------- | ---------------------------------------------- | -| `CometProject` | `ProjectExec` | -| `CometFilter` | `FilterExec` | -| `CometSort` | `SortExec` | -| `CometLocalLimit` | `LocalLimitExec` | -| `CometGlobalLimit` | `GlobalLimitExec` | -| `CometExpand` | `ExpandExec` | -| `CometExplode` | `GenerateExec` (for `explode` only) | -| `CometHashAggregate` | `HashAggregateExec`, `ObjectHashAggregateExec` | -| `CometHashJoin` | `ShuffledHashJoinExec` | -| `CometBroadcastHashJoin` | `BroadcastHashJoinExec` | -| `CometSortMergeJoin` | `SortMergeJoinExec` | -| `CometWindow` | `WindowExec` | -| `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | +| Node | Spark equivalent | +| ---------------------------- | ----------------------------------------------- | +| `CometProject` | `ProjectExec` | +| `CometFilter` | `FilterExec` | +| `CometSort` | `SortExec` | +| `CometLocalLimit` | `LocalLimitExec` | +| `CometGlobalLimit` | `GlobalLimitExec` | +| `CometExpand` | `ExpandExec` | +| `CometExplode` | `GenerateExec` (for `explode` and `posexplode`) | +| `CometHashAggregate` | `HashAggregateExec`, `ObjectHashAggregateExec` | +| `CometHashJoin` | `ShuffledHashJoinExec` | +| `CometBroadcastHashJoin` | `BroadcastHashJoinExec` | +| `CometSortMergeJoin` | `SortMergeJoinExec` | +| `CometWindow` | `WindowExec` | +| `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | ### JVM-Side Operators diff --git a/native/Cargo.lock b/native/Cargo.lock index df3c3b03c0..097d08509b 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -230,9 +230,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "607e64bb911ee4f90483e044fe78f175989148c2892e659a2cd25429e782ec54" +checksum = "378530e55cd479eda3c14eb345310799717e6f76d0c332041e8487022166b471" dependencies = [ "arrow-arith", "arrow-array", @@ -251,9 +251,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e754319ed8a85d817fe7adf183227e0b5308b82790a737b426c1124626b48118" +checksum = "a0ab212d2c1886e802f51c5212d78ebbcbb0bec980fff9dadc1eb8d45cd0b738" dependencies = [ "arrow-array", "arrow-buffer", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "841321891f247aa86c6112c80d83d89cb36e0addd020fa2425085b8eb6c3f579" +checksum = "cfd33d3e92f207444098c75b42de99d329562be0cf686b307b097cc52b4e999e" dependencies = [ "ahash", "arrow-buffer", @@ -276,7 +276,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.17.0", + "hashbrown 0.17.1", "num-complex", "num-integer", "num-traits", @@ -284,9 +284,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f955dfb73fae000425f49c8226d2044dab60fb7ad4af1e24f961756354d996c9" +checksum = "0c6cd424c2693bcdbc150d843dc9d4d137dd2de4782ce6df491ad11a3a0416c0" dependencies = [ "bytes", "half", @@ -296,9 +296,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca5e686972523798f76bef355145bc1ae25a84c731e650268d31ab763c701663" +checksum = "4c5aefb56a2c02e9e2b30746241058b85f8983f0fcff2ba0c6d09006e1cded7f" dependencies = [ "arrow-array", "arrow-buffer", @@ -318,9 +318,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86c276756867fc8186ec380c72c290e6e3b23a1d4fb05df6b1d62d2e62666d48" +checksum = "e94e8cf7e517657a52b91ea1263acf38c4ca62a84655d72458a3359b12ab97de" dependencies = [ "arrow-array", "arrow-cast", @@ -333,9 +333,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3b5846209775b6dc8056d77ff9a032b27043383dd5488abd0b663e265b9373" +checksum = "3c88210023a2bfee1896af366309a3028fc3bcbd6515fa29a7990ee1baa08ee0" dependencies = [ "arrow-buffer", "arrow-schema", @@ -346,9 +346,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd8907ddd8f9fbabf91ec2c85c1d81fe2874e336d2443eb36373595e28b98dd5" +checksum = "238438f0834483703d88896db6fe5a7138b2230debc31b34c0336c2996e3c64f" dependencies = [ "arrow-array", "arrow-buffer", @@ -357,13 +357,14 @@ dependencies = [ "arrow-select", "flatbuffers", "lz4_flex", + "zstd", ] [[package]] name = "arrow-json" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4518c59acc501f10d7dcae397fe12b8db3d81bc7de94456f8a58f9165d6f502" +checksum = "205ca2119e6d679d5c133c6f30e68f027738d95ed948cf77677ea69c7800036b" dependencies = [ "arrow-array", "arrow-buffer", @@ -386,9 +387,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efa70d9d6b1356f1fb9f1f651b84a725b7e0abb93f188cf7d31f14abfa2f2e6f" +checksum = "1bffd8fd2579286a5d63bac898159873e5094a79009940bcb42bbfce4f19f1d0" dependencies = [ "arrow-array", "arrow-buffer", @@ -399,9 +400,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faec88a945338192beffbbd4be0def70135422930caa244ac3cec0cd213b26b4" +checksum = "bab5994731204603c73ba69267616c50f80780774c6bb0476f1f830625115e0c" dependencies = [ "arrow-array", "arrow-buffer", @@ -412,9 +413,9 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18aa020f6bc8e5201dcd2d4b7f98c68f8a410ef37128263243e6ff2a47a67d4f" +checksum = "f633dbfdf39c039ada1bf9e34c694816eb71fbb7dc78f613993b7245e078a1ed" dependencies = [ "bitflags 2.11.1", "serde_core", @@ -423,9 +424,9 @@ dependencies = [ [[package]] name = "arrow-select" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a657ab5132e9c8ca3b24eb15a823d0ced38017fe3930ff50167466b02e2d592c" +checksum = "8cd065c54172ac787cf3f2f8d4107e0d3fdc26edba76fdf4f4cc170258942222" dependencies = [ "ahash", "arrow-array", @@ -437,9 +438,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "58.2.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6de2efbbd1a9f9780ceb8d1ff5d20421b35863b361e3386b4f571f1fc69fcb8" +checksum = "29dd7cda3ab9692f43a2e4acc444d760cc17b12bb6d8232ddf64e9bab7c06b42" dependencies = [ "arrow-array", "arrow-buffer", @@ -647,7 +648,7 @@ dependencies = [ "fastrand", "hex", "http 1.4.0", - "sha1", + "sha1 0.10.6", "time", "tokio", "tracing", @@ -1294,9 +1295,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.61" +version = "1.2.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d16d90359e986641506914ba71350897565610e87ce0ad9e6f28569db3dd5c6d" +checksum = "a1dce859f0832a7d088c4f1119888ab94ef4b5d6795d1ce05afb7fe159d79f98" dependencies = [ "find-msvc-tools", "jobserver", @@ -1859,14 +1860,12 @@ dependencies = [ [[package]] name = "datafusion" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93db0e623840612f7f2cd757f7e8a8922064192363732c88692e0870016e141b" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "arrow-schema", "async-trait", - "bytes", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1894,13 +1893,12 @@ dependencies = [ "datafusion-session", "datafusion-sql", "futures", + "indexmap 2.14.0", "itertools 0.14.0", "log", "object_store", "parking_lot", "parquet", - "rand 0.9.4", - "regex", "sqlparser", "tempfile", "tokio", @@ -1910,9 +1908,8 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37cefde60b26a7f4ff61e9d2ff2833322f91df2b568d7238afe67bde5bdffb66" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-trait", @@ -1935,9 +1932,8 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17e112307715d6a7a331111a4c2330ff54bc237183511c319e3708a4cff431fb" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-trait", @@ -2131,16 +2127,16 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d72a11ca44a95e1081870d3abb80c717496e8a7acb467a1d3e932bb636af5cc2" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ - "ahash", "arrow", "arrow-ipc", + "arrow-schema", "chrono", + "foldhash 0.2.0", "half", - "hashbrown 0.16.1", + "hashbrown 0.17.1", "hex", "indexmap 2.14.0", "itertools 0.14.0", @@ -2148,17 +2144,16 @@ dependencies = [ "log", "object_store", "parquet", - "paste", "sqlparser", "tokio", + "uuid", "web-time", ] [[package]] name = "datafusion-common-runtime" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89f4afaed29670ec4fd6053643adc749fe3f4bc9d1ce1b8c5679b22c67d12def" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "futures", "log", @@ -2167,9 +2162,8 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9fb386e1691355355a96419978a0022b7947b44d4a24a6ea99f00b6b485cbb6" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-compression", @@ -2193,6 +2187,7 @@ dependencies = [ "liblzma", "log", "object_store", + "parking_lot", "rand 0.9.4", "tokio", "tokio-util", @@ -2202,9 +2197,8 @@ dependencies = [ [[package]] name = "datafusion-datasource-arrow" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffa6c52cfed0734c5f93754d1c0175f558175248bf686c944fb05c373e5fc096" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "arrow-ipc", @@ -2226,9 +2220,8 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "503f29e0582c1fc189578d665ff57d9300da1f80c282777d7eb67bb79fb8cdca" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-trait", @@ -2249,9 +2242,8 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e33804749abc8d0c8cb7473228483cb8070e524c6f6086ee1b85a64debe2b3d2" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-trait", @@ -2266,16 +2258,14 @@ dependencies = [ "datafusion-session", "futures", "object_store", - "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" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-trait", @@ -2285,6 +2275,7 @@ dependencies = [ "datafusion-datasource", "datafusion-execution", "datafusion-expr", + "datafusion-functions", "datafusion-functions-aggregate-common", "datafusion-physical-expr", "datafusion-physical-expr-adapter", @@ -2303,20 +2294,17 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de6ac0df1662b9148ad3c987978b32cbec7c772f199b1d53520c8fa764a87ee" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" [[package]] name = "datafusion-execution" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c03c7fbdaefcca4ef6ffe425a5fc2325763bfb426599bb0bf4536466efabe709" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "arrow-buffer", "async-trait", - "chrono", "dashmap", "datafusion-common", "datafusion-expr", @@ -2333,11 +2321,11 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "574b9b6977fedbd2a611cbff12e5caf90f31640ad9dc5870f152836d94bad0dd" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", + "arrow-schema", "async-trait", "chrono", "datafusion-common", @@ -2348,29 +2336,25 @@ dependencies = [ "datafusion-physical-expr-common", "indexmap 2.14.0", "itertools 0.14.0", - "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" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "datafusion-common", "indexmap 2.14.0", "itertools 0.14.0", - "paste", ] [[package]] name = "datafusion-functions" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f28aa4e10384e782774b10e72aca4d93ef7b31aa653095d9d4536b0a3dbc51b6" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "arrow-buffer", @@ -2385,26 +2369,24 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-macros", + "datafusion-physical-expr-common", "hex", "itertools 0.14.0", "log", - "md-5", + "md-5 0.11.0", "memchr", "num-traits", "rand 0.9.4", "regex", - "sha2 0.10.9", - "unicode-segmentation", + "sha2 0.11.0", "uuid", ] [[package]] name = "datafusion-functions-aggregate" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00aa6217e56098ba84e0a338176fe52f0a84cca398021512c6c8c5eff806d0ad" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ - "ahash", "arrow", "datafusion-common", "datafusion-doc", @@ -2414,19 +2396,17 @@ dependencies = [ "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", + "foldhash 0.2.0", "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" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ - "ahash", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2435,9 +2415,8 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef13a858e20d50f0a9bb5e96e7ac82b4e7597f247515bccca4fdd2992df0212a" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "arrow-ord", @@ -2451,34 +2430,32 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-macros", "datafusion-physical-expr-common", - "hashbrown 0.16.1", + "hashbrown 0.17.1", "itertools 0.14.0", "itoa", "log", - "paste", + "memchr", ] [[package]] name = "datafusion-functions-table" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b40d3f5bbb3905f9ccb1ce9485a9595c77b69758a7c24d3ba79e334ff51e7e" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "async-trait", "datafusion-catalog", "datafusion-common", "datafusion-expr", + "datafusion-physical-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" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "datafusion-common", @@ -2489,14 +2466,12 @@ dependencies = [ "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" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -2504,9 +2479,8 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e367e6a71051d0ebdd29b2f85d12059b38b1d1f172c6906e80016da662226bd" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "datafusion-doc", "quote", @@ -2515,9 +2489,8 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e929015451a67f77d9d8b727b2bf3a40c4445fdef6cdc53281d7d97c76888ace" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "chrono", @@ -2534,11 +2507,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4b1e68aba7a4b350401cfdf25a3d6f989ad898a7410164afe9ca52080244cb59" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ - "ahash", "arrow", "datafusion-common", "datafusion-expr", @@ -2546,20 +2517,18 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-physical-expr-common", "half", - "hashbrown 0.16.1", + "hashbrown 0.17.1", "indexmap 2.14.0", "itertools 0.14.0", "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" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "datafusion-common", @@ -2572,26 +2541,24 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b04b45ea8ad3ac2d78f2ea2a76053e06591c9629c7a603eda16c10649ecf4362" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ - "ahash", "arrow", "chrono", "datafusion-common", "datafusion-expr-common", - "hashbrown 0.16.1", + "hashbrown 0.17.1", "indexmap 2.14.0", "itertools 0.14.0", "parking_lot", + "pin-project", ] [[package]] name = "datafusion-physical-optimizer" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7cb13397809a425918f608dfe8653f332015a3e330004ab191b4404187238b95" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "datafusion-common", @@ -2607,12 +2574,12 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5edc023675791af9d5fb4cc4c24abf5f7bd3bd4dcf9e5bd90ea1eff6976dcc79" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ - "ahash", "arrow", + "arrow-data", + "arrow-ipc", "arrow-ord", "arrow-schema", "async-trait", @@ -2627,7 +2594,7 @@ dependencies = [ "datafusion-physical-expr-common", "futures", "half", - "hashbrown 0.16.1", + "hashbrown 0.17.1", "indexmap 2.14.0", "itertools 0.14.0", "log", @@ -2639,9 +2606,8 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac8c76860e355616555081cab5968cec1af7a80701ff374510860bcd567e365a" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "datafusion-common", @@ -2650,15 +2616,13 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", - "itertools 0.14.0", "log", ] [[package]] name = "datafusion-session" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5412111aa48e2424ba926112e192f7a6b7e4ccb450145d25ce5ede9f19dc491e" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "async-trait", "datafusion-common", @@ -2670,9 +2634,8 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e059dcf8544da0d6598d0235be3cc29c209094a5976b2e4822e4a2cf91c2b5c5" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "bigdecimal", @@ -2685,21 +2648,23 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", "datafusion-functions-nested", "log", + "num-traits", "percent-encoding", "rand 0.9.4", "serde_json", - "sha1", - "sha2 0.10.9", + "sha1 0.11.0", + "sha2 0.11.0", + "twox-hash", "url", ] [[package]] name = "datafusion-sql" -version = "53.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa0d133ddf8b9b3b872acac900157f783e7b879fe9a6bccf389abebbfac45ec1" +version = "54.0.0" +source = "git+https://github.com/apache/datafusion?branch=branch-54#c8dddb8a03f098b92e0295118d12390381d23fa8" dependencies = [ "arrow", "bigdecimal", @@ -3319,21 +3284,15 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.16.1" +version = "0.17.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "841d1cc9bed7f9236f321df977030373f4a4163ae1a7dbfe1a51a2c1a51d9100" +checksum = "ed5909b6e89a2db4456e54cd5f673791d7eca6732202bbf2a9cc504fe2f9b84a" dependencies = [ "allocator-api2", "equivalent", "foldhash 0.2.0", ] -[[package]] -name = "hashbrown" -version = "0.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f467dd6dccf739c208452f8014c75c18bb8301b050ad1cfb27153803edb0f51" - [[package]] name = "hdfs-sys" version = "0.3.0" @@ -3771,7 +3730,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d466e9454f08e4a911e14806c24e16fba1b4c121d1ea474396f396069cf949d9" dependencies = [ "equivalent", - "hashbrown 0.17.0", + "hashbrown 0.17.1", "serde", "serde_core", ] @@ -4265,9 +4224,9 @@ checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" [[package]] name = "lz4_flex" -version = "0.13.0" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db9a0d582c2874f68138a16ce1867e0ffde6c0bb0a0df85e1f36d04146db488a" +checksum = "7ef0d4ed8669f8f8826eb00dc878084aa8f253506c4fd5e8f58f5bce72ddb97e" dependencies = [ "twox-hash", ] @@ -4282,6 +4241,16 @@ dependencies = [ "digest 0.10.7", ] +[[package]] +name = "md-5" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69b6441f590336821bb897fb28fc622898ccceb1d6cea3fde5ea86b090c4de98" +dependencies = [ + "cfg-if", + "digest 0.11.2", +] + [[package]] name = "mea" version = "0.6.3" @@ -4537,7 +4506,7 @@ dependencies = [ "humantime", "hyper", "itertools 0.14.0", - "md-5", + "md-5 0.10.6", "parking_lot", "percent-encoding", "quick-xml 0.39.2", @@ -4615,7 +4584,7 @@ dependencies = [ "http-body 1.0.1", "jiff", "log", - "md-5", + "md-5 0.10.6", "percent-encoding", "quick-xml 0.38.4", "reqsign", @@ -4656,7 +4625,7 @@ dependencies = [ "http-body 1.0.1", "jiff", "log", - "md-5", + "md-5 0.10.6", "mea", "percent-encoding", "quick-xml 0.38.4", @@ -4799,9 +4768,9 @@ dependencies = [ [[package]] name = "parquet" -version = "58.1.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d3f9f2205199603564127932b89695f52b62322f541d0fc7179d57c2e1c9877" +checksum = "5dafa7d01085b62a47dd0c1829550a0a36710ea9c4fe358a05a85477cec8a908" dependencies = [ "ahash", "arrow-array", @@ -4817,7 +4786,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.16.1", + "hashbrown 0.17.1", "lz4_flex", "num-bigint", "num-integer", @@ -4839,23 +4808,25 @@ dependencies = [ [[package]] name = "parquet-variant" -version = "58.1.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bf493f3c9ddd984d0efb019f67343e4aa4bab893931f6a14b82083065dc3d28" +checksum = "74c8db065291f088a2aad8ab831853eae1871c0d311c8d0b83bbc3b7e735d0fc" dependencies = [ + "arrow", "arrow-schema", "chrono", "half", "indexmap 2.14.0", + "num-traits", "simdutf8", "uuid", ] [[package]] name = "parquet-variant-compute" -version = "58.1.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ac038d46a503a7d563b4f5df5802c4315d5343d009feab195d15ac512b4cb27" +checksum = "a530e8d5b5e14efcb39c9a6ec55432ad11f6afb7dc4455a79be0dc615fe3cc31" dependencies = [ "arrow", "arrow-schema", @@ -4870,9 +4841,9 @@ dependencies = [ [[package]] name = "parquet-variant-json" -version = "58.1.0" +version = "58.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "015a09c2ffe5108766c7c1235c307b8a3c2ea64eca38455ba1a7f3a7f32f16e2" +checksum = "00ed89908289f67caa2ca078f9ff9aacd6229a313ec92b12bf4f48f613dc2b97" dependencies = [ "arrow-schema", "base64", @@ -5564,7 +5535,7 @@ dependencies = [ "rust-ini", "serde", "serde_json", - "sha1", + "sha1 0.10.6", "sha2 0.10.9", "tokio", ] @@ -5586,7 +5557,7 @@ dependencies = [ "jiff", "log", "percent-encoding", - "sha1", + "sha1 0.10.6", "sha2 0.10.9", "windows-sys 0.61.2", ] @@ -6131,6 +6102,17 @@ dependencies = [ "digest 0.10.7", ] +[[package]] +name = "sha1" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aacc4cc499359472b4abe1bf11d0b12e688af9a805fa5e3016f9a386dc2d0214" +dependencies = [ + "cfg-if", + "cpufeatures 0.3.0", + "digest 0.11.2", +] + [[package]] name = "sha2" version = "0.10.9" @@ -6274,9 +6256,9 @@ dependencies = [ [[package]] name = "sqlparser" -version = "0.61.0" +version = "0.62.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbf5ea8d4d7c808e1af1cbabebca9a2abe603bcefc22294c5b95018d53200cb7" +checksum = "13c6d1b651dc4edf07eead2a0c6c78016ce971bc2c10da5266861b13f25e7cec" dependencies = [ "log", "sqlparser_derive", @@ -6591,9 +6573,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.52.2" +version = "1.52.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "110a78583f19d5cdb2c5ccf321d1290344e71313c6c37d43520d386027d18386" +checksum = "8fc7f01b389ac15039e4dc9531aa973a135d7a4135281b12d7c1bc79fd57fffe" dependencies = [ "bytes", "libc", diff --git a/native/Cargo.toml b/native/Cargo.toml index d1b5c74af9..c545016792 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,14 +34,14 @@ edition = "2021" rust-version = "1.88" [workspace.dependencies] -arrow = { version = "58.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "58.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.11.1" } -parquet = { version = "58.1.0", default-features = false, features = ["experimental"] } -datafusion = { version = "53.1.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } -datafusion-datasource = { version = "53.1.0" } -datafusion-physical-expr-adapter = { version = "53.1.0" } -datafusion-spark = { version = "53.1.0", features = ["core"] } +parquet = { version = "58.3.0", default-features = false, features = ["experimental"] } +datafusion = { git = "https://github.com/apache/datafusion", branch = "branch-54", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion-datasource = { git = "https://github.com/apache/datafusion", branch = "branch-54" } +datafusion-physical-expr-adapter = { git = "https://github.com/apache/datafusion", branch = "branch-54" } +datafusion-spark = { git = "https://github.com/apache/datafusion", branch = "branch-54", features = ["core"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-common = { path = "common" } datafusion-comet-jni-bridge = { path = "jni-bridge" } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4fb3ed4c5d..6505e31cf0 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -91,7 +91,7 @@ jni = { version = "0.22.4", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { version = "53.1.0" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", branch = "branch-54" } [features] backtrace = ["datafusion/backtrace"] @@ -109,18 +109,6 @@ name = "comet" # "rlib" is for benchmarking with criterion. crate-type = ["cdylib", "rlib"] -[[bench]] -name = "parquet_read" -harness = false - -[[bench]] -name = "bit_util" -harness = false - -[[bench]] -name = "parquet_decode" -harness = false - [[bench]] name = "array_element_append" harness = false diff --git a/native/core/benches/bit_util.rs b/native/core/benches/bit_util.rs deleted file mode 100644 index 92ae497a56..0000000000 --- a/native/core/benches/bit_util.rs +++ /dev/null @@ -1,216 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{mem::size_of, time::Duration}; - -use rand::{rng, RngExt}; - -use arrow::buffer::Buffer; -use comet::common::bit::{ - log2, read_num_bytes_u32, read_num_bytes_u64, read_u32, read_u64, set_bits, trailing_bits, - BitReader, BitWriter, -}; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; -use std::hint::black_box; - -/// Benchmark to measure bit_util performance. -/// To run this benchmark: -/// `cd core && cargo bench --bench bit_util` -/// Results will be written to "core/target/criterion/bit_util/" -fn criterion_benchmark(c: &mut Criterion) { - let mut group = c.benchmark_group("bit_util"); - - const N: usize = 1024 * 1024; - let mut writer: BitWriter = BitWriter::new(N * 10); - for _ in 0..N { - if !writer.put_vlq_int(rng().random::()) { - break; - } - } - let buffer = writer.consume(); - let buffer = Buffer::from(buffer.as_slice()); - - // log2 - for bits in (0..64).step_by(3) { - let x = 1u64 << bits; - group.bench_with_input(BenchmarkId::new("log2", bits), &x, |b, &x| { - b.iter(|| log2(black_box(x))); - }); - } - - // set_bits - for offset in (0..16).step_by(3) { - for length in (0..16).step_by(3) { - let x = (offset, length); - group.bench_with_input( - BenchmarkId::new("set_bits", format!("offset_{}_length_{}", x.0, x.1)), - &x, - |b, &x| { - b.iter(|| set_bits(&mut [0u8; 4], black_box(x.0), black_box(x.1))); - }, - ); - } - } - - // get_vlq_int - group.bench_function("get_vlq_int", |b| { - b.iter(|| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - bench_get_vlq_int(&mut reader) - }) - }); - - // get_bits - for offset in (0..32).step_by(17) { - for num_bits in (1..5).step_by(1) { - let x = (offset, num_bits); - group.bench_with_input( - BenchmarkId::new("get_bits", format!("offset_{}_num_bits_{}", x.0, x.1)), - &x, - |b, &x| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - b.iter(|| reader.get_bits(&mut [0u8; 4], black_box(x.0), black_box(x.1))); - }, - ); - } - } - - // get_aligned - for num_bytes in (1..=size_of::()).step_by(3) { - let x = num_bytes; - group.bench_with_input( - BenchmarkId::new("get_aligned", format!("u8_num_bytes_{x}")), - &x, - |b, &x| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - b.iter(|| reader.get_aligned::(black_box(x))); - }, - ); - } - for num_bytes in (1..=size_of::()).step_by(3) { - let x = num_bytes; - group.bench_with_input( - BenchmarkId::new("get_aligned", format!("u32_num_bytes_{x}")), - &x, - |b, &x| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - b.iter(|| reader.get_aligned::(black_box(x))); - }, - ); - } - for num_bytes in (1..=size_of::()).step_by(3) { - let x = num_bytes; - group.bench_with_input( - BenchmarkId::new("get_aligned", format!("i32_num_bytes_{x}")), - &x, - |b, &x| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - b.iter(|| reader.get_aligned::(black_box(x))); - }, - ); - } - - // get_value - for num_bytes in (1..=size_of::()).step_by(3) { - let x = num_bytes * 8; - group.bench_with_input( - BenchmarkId::new("get_value", format!("i32_num_bits_{x}")), - &x, - |b, &x| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - b.iter(|| reader.get_value::(black_box(x))); - }, - ); - } - - // read_num_bytes_u64 - for num_bytes in (1..=8).step_by(7) { - let x = num_bytes; - group.bench_with_input( - BenchmarkId::new("read_num_bytes_u64", format!("num_bytes_{x}")), - &x, - |b, &x| { - b.iter(|| read_num_bytes_u64(black_box(x), black_box(buffer.as_slice()))); - }, - ); - } - - // read_num_bytes_u32 - for num_bytes in (1..=4).step_by(3) { - let x = num_bytes; - group.bench_with_input( - BenchmarkId::new("read_num_bytes_u32", format!("num_bytes_{x}")), - &x, - |b, &x| { - b.iter(|| read_num_bytes_u32(black_box(x), black_box(buffer.as_slice()))); - }, - ); - } - - // trailing_bits - for length in (0..=64).step_by(32) { - let x = length; - group.bench_with_input( - BenchmarkId::new("trailing_bits", format!("num_bits_{x}")), - &x, - |b, &x| { - b.iter(|| trailing_bits(black_box(1234567890), black_box(x))); - }, - ); - } - - // read_u64 - group.bench_function("read_u64", |b| { - b.iter(|| read_u64(black_box(&[0u8; 8]))); - }); - - // read_u32 - group.bench_function("read_u32", |b| { - b.iter(|| read_u32(black_box(&[0u8; 4]))); - }); - - // get_u32_value - group.bench_function("get_u32_value", |b| { - b.iter(|| { - let mut reader: BitReader = BitReader::new_all(buffer.slice(0)); - for _ in 0..(buffer.len() * 8 / 31) { - black_box(reader.get_u32_value(black_box(31))); - } - }) - }); - - group.finish(); -} - -fn bench_get_vlq_int(reader: &mut BitReader) { - while let Some(v) = reader.get_vlq_int() { - black_box(v); - } -} - -fn config() -> Criterion { - Criterion::default() - .measurement_time(Duration::from_millis(500)) - .warm_up_time(Duration::from_millis(500)) -} - -criterion_group! { - name = benches; - config = config(); - targets = criterion_benchmark -} -criterion_main!(benches); diff --git a/native/core/benches/parquet_decode.rs b/native/core/benches/parquet_decode.rs deleted file mode 100644 index d3976aefc3..0000000000 --- a/native/core/benches/parquet_decode.rs +++ /dev/null @@ -1,56 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::datatypes::ToByteSlice; -use comet::parquet::read::values::{copy_i32_to_i16, copy_i32_to_u16, copy_i64_to_i64}; -use criterion::{criterion_group, criterion_main, Criterion}; - -fn criterion_benchmark(c: &mut Criterion) { - let num = 1000; - let source = vec![78_i8; num * 8]; - let mut group = c.benchmark_group("parquet_decode"); - group.bench_function("decode_i32_to_i16", |b| { - let mut dest: Vec = vec![b' '; num * 2]; - b.iter(|| { - copy_i32_to_i16(source.to_byte_slice(), dest.as_mut_slice(), num); - }); - }); - group.bench_function("decode_i32_to_u16", |b| { - let mut dest: Vec = vec![b' '; num * 4]; - b.iter(|| { - copy_i32_to_u16(source.to_byte_slice(), dest.as_mut_slice(), num); - }); - }); - group.bench_function("decode_i64_to_i64", |b| { - let mut dest: Vec = vec![b' '; num * 8]; - b.iter(|| { - copy_i64_to_i64(source.to_byte_slice(), dest.as_mut_slice(), num); - }); - }); -} - -// Create UTF8 batch with strings representing ints, floats, nulls -fn config() -> Criterion { - Criterion::default() -} - -criterion_group! { - name = benches; - config = config(); - targets = criterion_benchmark -} -criterion_main!(benches); diff --git a/native/core/benches/parquet_read.rs b/native/core/benches/parquet_read.rs deleted file mode 100644 index 52ab96eb1b..0000000000 --- a/native/core/benches/parquet_read.rs +++ /dev/null @@ -1,216 +0,0 @@ -// 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. - -mod perf; - -use std::sync::Arc; - -use arrow::{array::ArrayData, buffer::Buffer}; -use comet::parquet::{read::ColumnReader, util::jni::TypePromotionInfo}; -use criterion::{criterion_group, criterion_main, Criterion}; -use parquet::{ - basic::{Encoding, Type as PhysicalType}, - column::page::{PageIterator, PageReader}, - data_type::Int32Type, - schema::types::{ - ColumnDescPtr, ColumnDescriptor, ColumnPath, PrimitiveTypeBuilder, SchemaDescPtr, TypePtr, - }, -}; - -use comet::parquet::util::test_common::page_util::{ - DataPageBuilder, DataPageBuilderImpl, InMemoryPageIterator, -}; - -use perf::FlamegraphProfiler; -use rand::{prelude::StdRng, RngExt, SeedableRng}; - -fn bench(c: &mut Criterion) { - let expected_num_values: usize = NUM_PAGES * VALUES_PER_PAGE; - let mut group = c.benchmark_group("comet_parquet_read"); - let schema = build_test_schema(); - - let pages = build_plain_int32_pages(schema.column(0), 0.0); - group.bench_function("INT/PLAIN/NOT_NULL", |b| { - let t = TypePtr::new( - PrimitiveTypeBuilder::new("f", PhysicalType::INT32) - .with_length(4) - .build() - .unwrap(), - ); - b.iter(|| { - let cd = ColumnDescriptor::new(t.clone(), 0, 0, ColumnPath::from(Vec::new())); - let promotion_info = TypePromotionInfo::new(PhysicalType::INT32, -1, -1, 32); - let mut column_reader = TestColumnReader::new( - cd, - promotion_info, - BATCH_SIZE, - pages.clone(), - expected_num_values, - ); - - let mut total = 0; - for batch in column_reader.by_ref() { - total += batch.len(); - ::std::mem::forget(batch); - } - assert_eq!(total, expected_num_values); - }); - }); -} - -fn profiled() -> Criterion { - Criterion::default().with_profiler(FlamegraphProfiler::new(100)) -} - -criterion_group! { - name = benches; - config = profiled(); - targets = bench -} -criterion_main!(benches); - -fn build_test_schema() -> SchemaDescPtr { - use parquet::schema::{parser::parse_message_type, types::SchemaDescriptor}; - let message_type = " - message test_schema { - REQUIRED INT32 c1; - OPTIONAL INT32 c2; - } - "; - parse_message_type(message_type) - .map(|t| Arc::new(SchemaDescriptor::new(Arc::new(t)))) - .unwrap() -} - -fn seedable_rng() -> StdRng { - StdRng::seed_from_u64(42) -} - -// test data params -const NUM_PAGES: usize = 1000; -const VALUES_PER_PAGE: usize = 10_000; -const BATCH_SIZE: usize = 4096; - -fn build_plain_int32_pages( - column_desc: ColumnDescPtr, - null_density: f32, -) -> impl PageIterator + Clone { - let max_def_level = column_desc.max_def_level(); - let max_rep_level = column_desc.max_rep_level(); - let rep_levels = vec![0; VALUES_PER_PAGE]; - let mut rng = seedable_rng(); - let mut pages: Vec = Vec::new(); - let mut int32_value = 0; - for _ in 0..NUM_PAGES { - // generate page - let mut values = Vec::with_capacity(VALUES_PER_PAGE); - let mut def_levels = Vec::with_capacity(VALUES_PER_PAGE); - for _ in 0..VALUES_PER_PAGE { - let def_level = if rng.random::() < null_density { - max_def_level - 1 - } else { - max_def_level - }; - if def_level == max_def_level { - int32_value += 1; - values.push(int32_value); - } - def_levels.push(def_level); - } - let mut page_builder = - DataPageBuilderImpl::new(column_desc.clone(), values.len() as u32, true); - page_builder.add_rep_levels(max_rep_level, &rep_levels); - page_builder.add_def_levels(max_def_level, &def_levels); - page_builder.add_values::(Encoding::PLAIN, &values); - pages.push(page_builder.consume()); - } - - // Since `InMemoryPageReader` is not exposed from parquet crate, here we use - // `InMemoryPageIterator` instead which is a Iter>. - InMemoryPageIterator::new(vec![pages]) -} - -struct TestColumnReader { - inner: ColumnReader, - pages: Box, - batch_size: usize, - total_num_values: usize, - total_num_values_read: usize, - first_page_loaded: bool, -} - -impl TestColumnReader { - pub fn new( - cd: ColumnDescriptor, - promotion_info: TypePromotionInfo, - batch_size: usize, - mut page_iter: impl PageIterator + 'static, - total_num_values: usize, - ) -> Self { - let reader = ColumnReader::get(cd, promotion_info, batch_size, false, false); - let first = page_iter.next().unwrap().unwrap(); - Self { - inner: reader, - pages: first, - batch_size, - total_num_values, - total_num_values_read: 0, - first_page_loaded: false, - } - } - - fn load_page(&mut self) { - if let Some(page) = self.pages.get_next_page().unwrap() { - let num_values = page.num_values() as usize; - let buffer = Buffer::from_slice_ref(page.buffer()); - self.inner.set_page_v1(num_values, buffer, page.encoding()); - } - } -} - -impl Iterator for TestColumnReader { - type Item = ArrayData; - - fn next(&mut self) -> Option { - if self.total_num_values_read >= self.total_num_values { - return None; - } - - if !self.first_page_loaded { - self.load_page(); - self.first_page_loaded = true; - } - - self.inner.reset_batch(); - let total = ::std::cmp::min( - self.batch_size, - self.total_num_values - self.total_num_values_read, - ); - - let mut left = total; - while left > 0 { - let (num_read, _) = self.inner.read_batch(left, 0); - if num_read < left { - self.load_page(); - } - left -= num_read; - } - self.total_num_values_read += total; - - Some(self.inner.current_batch().unwrap()) - } -} diff --git a/native/core/src/common/bit.rs b/native/core/src/common/bit.rs deleted file mode 100644 index 459c38a52b..0000000000 --- a/native/core/src/common/bit.rs +++ /dev/null @@ -1,1602 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{cmp::min, mem::size_of}; - -use crate::{ - errors::CometResult as Result, - parquet::{data_type::AsBytes, util::bit_packing::unpack32}, -}; -use arrow::buffer::Buffer; -use datafusion_comet_spark_expr::utils::{likely, unlikely}; - -#[inline] -pub fn from_ne_slice(bs: &[u8]) -> T { - let mut b = T::Buffer::default(); - { - let b = b.as_mut(); - let bs = &bs[..b.len()]; - b.copy_from_slice(bs); - } - T::from_ne_bytes(b) -} - -pub trait FromBytes: Sized { - type Buffer: AsMut<[u8]> + Default; - fn from_le_bytes(bs: Self::Buffer) -> Self; - fn from_be_bytes(bs: Self::Buffer) -> Self; - fn from_ne_bytes(bs: Self::Buffer) -> Self; - fn from(v: u64) -> Self; -} - -macro_rules! from_le_bytes { - ($($ty: ty),*) => { - $( - impl FromBytes for $ty { - type Buffer = [u8; size_of::()]; - fn from_le_bytes(bs: Self::Buffer) -> Self { - <$ty>::from_le_bytes(bs) - } - fn from_be_bytes(bs: Self::Buffer) -> Self { - <$ty>::from_be_bytes(bs) - } - fn from_ne_bytes(bs: Self::Buffer) -> Self { - <$ty>::from_ne_bytes(bs) - } - fn from(v: u64) -> Self { - v as $ty - } - } - )* - }; -} - -impl FromBytes for bool { - type Buffer = [u8; 1]; - fn from_le_bytes(bs: Self::Buffer) -> Self { - Self::from_ne_bytes(bs) - } - fn from_be_bytes(bs: Self::Buffer) -> Self { - Self::from_ne_bytes(bs) - } - fn from_ne_bytes(bs: Self::Buffer) -> Self { - match bs[0] { - 0 => false, - 1 => true, - _ => panic!("Invalid byte when reading bool"), - } - } - fn from(v: u64) -> Self { - (v & 1) == 1 - } -} - -// TODO: support f32 and f64 in the future, but there is no use case right now -// f32/f64::from(v: u64) will be like `from_ne_slice(v.as_bytes()))` and that is -// expensive as it involves copying buffers -from_le_bytes! { u8, u16, u32, u64, i8, i16, i32, i64 } - -/// Reads `$size` of bytes from `$src`, and reinterprets them as type `$ty`, in -/// little-endian order. `$ty` must implement the `Default` trait. Otherwise this won't -/// compile. -/// This is copied and modified from byteorder crate. -macro_rules! read_num_bytes { - ($ty:ty, $size:expr, $src:expr) => {{ - debug_assert!($size <= $src.len()); - let mut buffer = <$ty as $crate::common::bit::FromBytes>::Buffer::default(); - buffer.as_mut()[..$size].copy_from_slice(&$src[..$size]); - <$ty>::from_ne_bytes(buffer) - }}; -} - -/// u64 specific version of read_num_bytes! -/// This is faster than read_num_bytes! because this method avoids buffer copies. -#[inline] -pub fn read_num_bytes_u64(size: usize, src: &[u8]) -> u64 { - debug_assert!(size <= src.len()); - if unlikely(src.len() < 8) { - return read_num_bytes!(u64, size, src); - } - let in_ptr = src as *const [u8] as *const u8 as *const u64; - let v = unsafe { in_ptr.read_unaligned() }; - trailing_bits(v, size * 8) -} - -/// u32 specific version of read_num_bytes! -/// This is faster than read_num_bytes! because this method avoids buffer copies. -#[inline] -pub fn read_num_bytes_u32(size: usize, src: &[u8]) -> u32 { - debug_assert!(size <= src.len()); - if unlikely(src.len() < 4) { - return read_num_bytes!(u32, size, src); - } - let in_ptr = src as *const [u8] as *const u8 as *const u32; - let v = unsafe { in_ptr.read_unaligned() }; - trailing_bits(v as u64, size * 8) as u32 -} - -#[inline] -pub fn read_u64(src: &[u8]) -> u64 { - let in_ptr = src.as_ptr() as *const u64; - unsafe { in_ptr.read_unaligned() } -} - -#[inline] -pub fn read_u32(src: &[u8]) -> u32 { - let in_ptr = src.as_ptr() as *const u32; - unsafe { in_ptr.read_unaligned() } -} - -#[inline] -pub fn memcpy(source: &[u8], target: &mut [u8]) { - debug_assert!(target.len() >= source.len(), "Copying from source to target is not possible. Source has {} bytes but target has {} bytes", source.len(), target.len()); - // Originally `target[..source.len()].copy_from_slice(source)` - // We use the unsafe copy method to avoid some expensive bounds checking/ - unsafe { std::ptr::copy_nonoverlapping(source.as_ptr(), target.as_mut_ptr(), source.len()) } -} - -#[inline] -pub fn memcpy_value(source: &T, num_bytes: usize, target: &mut [u8]) -where - T: ?Sized + AsBytes, -{ - debug_assert!( - target.len() >= num_bytes, - "Not enough space. Only had {} bytes but need to put {} bytes", - target.len(), - num_bytes - ); - memcpy(&source.as_bytes()[..num_bytes], target) -} - -/// Returns ceil(log2(x)) -#[inline] -pub fn log2(mut x: u64) -> u32 { - if x == 1 { - return 0; - } - x -= 1; - 64u32 - x.leading_zeros() -} - -/// Returns the `num_bits` least-significant bits of `v` -#[inline] -pub fn trailing_bits(v: u64, num_bits: usize) -> u64 { - if unlikely(num_bits == 0) { - return 0; - } - if unlikely(num_bits >= 64) { - return v; - } - v & ((1 << num_bits) - 1) -} - -#[inline] -pub fn set_bit(bits: &mut [u8], i: usize) { - bits[i / 8] |= 1 << (i % 8); -} - -/// Set the bit value at index `i`, for buffer `bits`. -/// -/// # Safety -/// This doesn't check bounds, the caller must ensure that `i` is in (0, bits.len() * 8) -#[inline] -pub unsafe fn set_bit_raw(bits: *mut u8, i: usize) { - *bits.add(i / 8) |= 1 << (i % 8); -} - -#[inline] -pub fn unset_bit(bits: &mut [u8], i: usize) { - bits[i / 8] &= !(1 << (i % 8)); -} - -#[inline] -pub fn set_bits(bits: &mut [u8], offset: usize, length: usize) { - let mut byte_i = offset / 8; - let offset_r = offset % 8; - let end = offset + length; - let end_byte_i = end / 8; - let end_r = end % 8; - - // if the offset starts in the middle of a byte, update the byte first - if offset_r != 0 { - let num_bits = min(length, 7); - bits[byte_i] |= ((1u8 << num_bits) - 1) << offset_r; - byte_i += 1; - } - - // See if there is an opportunity to do a bulk byte write - if byte_i < end_byte_i { - unsafe { - bits.as_mut_ptr() - .add(byte_i) - .write_bytes(255, end_byte_i - byte_i); - } - byte_i = end_byte_i; - } - - // take care of the last byte - if end_r > 0 && (byte_i == end_byte_i) { - bits[byte_i] |= (1u8 << end_r) - 1; - } -} - -#[inline(always)] -pub fn mix_hash(lower: u64, upper: u64) -> u64 { - let hash = (17 * 37u64).wrapping_add(lower); - hash.wrapping_mul(37).wrapping_add(upper) -} - -static BIT_MASK: [u8; 8] = [1, 2, 4, 8, 16, 32, 64, 128]; - -/// Returns whether bit at position `i` in `data` is set or not -#[inline] -pub fn get_bit(data: &[u8], i: usize) -> bool { - (data[i >> 3] & BIT_MASK[i & 7]) != 0 -} - -/// Returns the boolean value at index `i`. -/// -/// # Safety -/// This doesn't check bounds, the caller must ensure that index < self.len() -#[inline] -pub unsafe fn get_bit_raw(ptr: *const u8, i: usize) -> bool { - (*ptr.add(i >> 3) & BIT_MASK[i & 7]) != 0 -} - -/// Utility class for writing bit/byte streams. This class can write data in either -/// bit packed or byte aligned fashion. -pub struct BitWriter { - buffer: Vec, - max_bytes: usize, - buffered_values: u64, - byte_offset: usize, - bit_offset: usize, - start: usize, -} - -impl BitWriter { - pub fn new(max_bytes: usize) -> Self { - Self { - buffer: vec![0; max_bytes], - max_bytes, - buffered_values: 0, - byte_offset: 0, - bit_offset: 0, - start: 0, - } - } - - /// Initializes the writer from the existing buffer `buffer` and starting - /// offset `start`. - pub fn new_from_buf(buffer: Vec, start: usize) -> Self { - debug_assert!(start < buffer.len()); - let len = buffer.len(); - Self { - buffer, - max_bytes: len, - buffered_values: 0, - byte_offset: start, - bit_offset: 0, - start, - } - } - - /// Extend buffer size by `increment` bytes - #[inline] - pub fn extend(&mut self, increment: usize) { - self.max_bytes += increment; - let extra = vec![0; increment]; - self.buffer.extend(extra); - } - - /// Report buffer size, in bytes - #[inline] - pub fn capacity(&mut self) -> usize { - self.max_bytes - } - - /// Consumes and returns the current buffer. - #[inline] - pub fn consume(mut self) -> Vec { - self.flush(); - self.buffer.truncate(self.byte_offset); - self.buffer - } - - /// Flushes the internal buffered bits and returns the buffer's content. - /// This is a borrow equivalent of `consume` method. - #[inline] - pub fn flush_buffer(&mut self) -> &[u8] { - self.flush(); - &self.buffer()[0..self.byte_offset] - } - - /// Clears the internal state so the buffer can be reused. - #[inline] - pub fn clear(&mut self) { - self.buffered_values = 0; - self.byte_offset = self.start; - self.bit_offset = 0; - } - - /// Flushes the internal buffered bits and the align the buffer to the next byte. - #[inline] - pub fn flush(&mut self) { - let num_bytes = self.bit_offset.div_ceil(8); - debug_assert!(self.byte_offset + num_bytes <= self.max_bytes); - memcpy_value( - &self.buffered_values, - num_bytes, - &mut self.buffer[self.byte_offset..], - ); - self.buffered_values = 0; - self.bit_offset = 0; - self.byte_offset += num_bytes; - } - - /// Advances the current offset by skipping `num_bytes`, flushing the internal bit - /// buffer first. - /// This is useful when you want to jump over `num_bytes` bytes and come back later - /// to fill these bytes. - /// - /// Returns error if `num_bytes` is beyond the boundary of the internal buffer. - /// Otherwise, returns the old offset. - #[inline] - pub fn skip(&mut self, num_bytes: usize) -> Result { - self.flush(); - debug_assert!(self.byte_offset <= self.max_bytes); - if unlikely(self.byte_offset + num_bytes > self.max_bytes) { - return Err(general_err!( - "Not enough bytes left in BitWriter. Need {} but only have {}", - self.byte_offset + num_bytes, - self.max_bytes - )); - } - let result = self.byte_offset; - self.byte_offset += num_bytes; - Ok(result) - } - - /// Returns a slice containing the next `num_bytes` bytes starting from the current - /// offset, and advances the underlying buffer by `num_bytes`. - /// This is useful when you want to jump over `num_bytes` bytes and come back later - /// to fill these bytes. - #[inline] - pub fn get_next_byte_ptr(&mut self, num_bytes: usize) -> Result<&mut [u8]> { - let offset = self.skip(num_bytes)?; - Ok(&mut self.buffer[offset..offset + num_bytes]) - } - - #[inline] - pub fn bytes_written(&self) -> usize { - self.byte_offset - self.start + self.bit_offset.div_ceil(8) - } - - #[inline] - pub fn buffer(&self) -> &[u8] { - &self.buffer[self.start..] - } - - #[inline] - pub fn byte_offset(&self) -> usize { - self.byte_offset - } - - /// Returns the internal buffer length. This is the maximum number of bytes that this - /// writer can write. User needs to call `consume` to consume the current buffer - /// before more data can be written. - #[inline] - pub fn buffer_len(&self) -> usize { - self.max_bytes - } - - /// Writes the entire byte `value` at the byte `offset` - pub fn write_at(&mut self, offset: usize, value: u8) { - self.buffer[offset] = value; - } - - /// Writes the `num_bits` LSB of value `v` to the internal buffer of this writer. - /// The `num_bits` must not be greater than 64. This is bit packed. - /// - /// Returns false if there's not enough room left. True otherwise. - #[inline] - #[allow(clippy::unnecessary_cast)] - pub fn put_value(&mut self, v: u64, num_bits: usize) -> bool { - debug_assert!(num_bits <= 64); - debug_assert_eq!(v.checked_shr(num_bits as u32).unwrap_or(0), 0); // covers case v >> 64 - - let num_bytes = self.byte_offset * 8 + self.bit_offset + num_bits; - if unlikely(num_bytes > self.max_bytes as usize * 8) { - return false; - } - - self.buffered_values |= v << self.bit_offset; - self.bit_offset += num_bits; - if self.bit_offset >= 64 { - memcpy_value( - &self.buffered_values, - 8, - &mut self.buffer[self.byte_offset..], - ); - self.byte_offset += 8; - self.bit_offset -= 64; - self.buffered_values = 0; - // Perform checked right shift: v >> offset, where offset < 64, otherwise we - // shift all bits - self.buffered_values = v - .checked_shr((num_bits - self.bit_offset) as u32) - .unwrap_or(0); - } - debug_assert!(self.bit_offset < 64); - true - } - - /// Writes `val` of `num_bytes` bytes to the next aligned byte. If size of `T` is - /// larger than `num_bytes`, extra higher ordered bytes will be ignored. - /// - /// Returns false if there's not enough room left. True otherwise. - #[inline] - pub fn put_aligned(&mut self, val: T, num_bytes: usize) -> bool { - let result = self.get_next_byte_ptr(num_bytes); - if unlikely(result.is_err()) { - // TODO: should we return `Result` for this func? - return false; - } - let ptr = result.unwrap(); - memcpy_value(&val, num_bytes, ptr); - true - } - - /// Writes `val` of `num_bytes` bytes at the designated `offset`. The `offset` is the - /// offset starting from the beginning of the internal buffer that this writer - /// maintains. Note that this will overwrite any existing data between `offset` and - /// `offset + num_bytes`. Also that if size of `T` is larger than `num_bytes`, extra - /// higher ordered bytes will be ignored. - /// - /// Returns false if there's not enough room left, or the `pos` is not valid. - /// True otherwise. - #[inline] - pub fn put_aligned_offset( - &mut self, - val: T, - num_bytes: usize, - offset: usize, - ) -> bool { - if unlikely(num_bytes + offset > self.max_bytes) { - return false; - } - memcpy_value( - &val, - num_bytes, - &mut self.buffer[offset..offset + num_bytes], - ); - true - } - - /// Writes a VLQ encoded integer `v` to this buffer. The value is byte aligned. - /// - /// Returns false if there's not enough room left. True otherwise. - #[inline] - pub fn put_vlq_int(&mut self, mut v: u64) -> bool { - let mut result = true; - while v & 0xFFFFFFFFFFFFFF80 != 0 { - result &= self.put_aligned::(((v & 0x7F) | 0x80) as u8, 1); - v >>= 7; - } - result &= self.put_aligned::((v & 0x7F) as u8, 1); - result - } - - /// Writes a zigzag-VLQ encoded (in little endian order) int `v` to this buffer. - /// Zigzag-VLQ is a variant of VLQ encoding where negative and positive - /// numbers are encoded in a zigzag fashion. - /// See: https://developers.google.com/protocol-buffers/docs/encoding - /// - /// Returns false if there's not enough room left. True otherwise. - #[inline] - pub fn put_zigzag_vlq_int(&mut self, v: i64) -> bool { - let u: u64 = ((v << 1) ^ (v >> 63)) as u64; - self.put_vlq_int(u) - } -} - -/// Maximum byte length for a VLQ encoded integer -/// MAX_VLQ_BYTE_LEN = 5 for i32, and MAX_VLQ_BYTE_LEN = 10 for i64 -pub const MAX_VLQ_BYTE_LEN: usize = 10; - -pub struct BitReader { - /// The byte buffer to read from, passed in by client - buffer: Buffer, // TODO: generalize this - - /// Bytes are memcpy'd from `buffer` and values are read from this variable. - /// This is faster than reading values byte by byte directly from `buffer` - buffered_values: u64, - - /// - /// End Start - /// |............|B|B|B|B|B|B|B|B|..............| - /// ^ ^ - /// bit_offset byte_offset - /// - /// Current byte offset in `buffer` - byte_offset: usize, - - /// Current bit offset in `buffered_values` - bit_offset: usize, - - /// Total number of bytes in `buffer` - total_bytes: usize, -} - -/// Utility class to read bit/byte stream. This class can read bits or bytes that are -/// either byte aligned or not. -impl BitReader { - pub fn new(buf: Buffer, len: usize) -> Self { - let buffered_values = if size_of::() > len { - read_num_bytes_u64(len, buf.as_slice()) - } else { - read_u64(buf.as_slice()) - }; - BitReader { - buffer: buf, - buffered_values, - byte_offset: 0, - bit_offset: 0, - total_bytes: len, - } - } - - pub fn new_all(buf: Buffer) -> Self { - let len = buf.len(); - Self::new(buf, len) - } - - pub fn reset(&mut self, buf: Buffer) { - self.buffer = buf; - self.total_bytes = self.buffer.len(); - self.buffered_values = if size_of::() > self.total_bytes { - read_num_bytes_u64(self.total_bytes, self.buffer.as_slice()) - } else { - read_u64(self.buffer.as_slice()) - }; - self.byte_offset = 0; - self.bit_offset = 0; - } - - /// Gets the current byte offset - #[inline] - pub fn get_byte_offset(&self) -> usize { - self.byte_offset + self.bit_offset.div_ceil(8) - } - - /// Reads a value of type `T` and of size `num_bits`. - /// - /// Returns `None` if there's not enough data available. `Some` otherwise. - pub fn get_value(&mut self, num_bits: usize) -> Option { - debug_assert!(num_bits <= 64); - debug_assert!(num_bits <= size_of::() * 8); - - if unlikely(self.byte_offset * 8 + self.bit_offset + num_bits > self.total_bytes * 8) { - return None; - } - - let v = self.get_u64_value(num_bits); - Some(T::from(v)) - } - - /// Reads a `u32` value encoded using `num_bits` of bits. - /// - /// # Safety - /// - /// This method asusumes the following: - /// - /// - the `num_bits` is <= 64 - /// - the remaining number of bits to read in this reader is >= `num_bits`. - /// - /// Undefined behavior will happen if any of the above assumptions is violated. - #[inline] - pub fn get_u32_value(&mut self, num_bits: usize) -> u32 { - self.get_u64_value(num_bits) as u32 - } - - #[inline(always)] - fn get_u64_value(&mut self, num_bits: usize) -> u64 { - if unlikely(num_bits == 0) { - 0 - } else { - let v = self.buffered_values >> self.bit_offset; - let mask = u64::MAX >> (64 - num_bits); - self.bit_offset += num_bits; - if self.bit_offset < 64 { - v & mask - } else { - self.byte_offset += 8; - self.bit_offset -= 64; - self.reload_buffer_values(); - ((self.buffered_values << (num_bits - self.bit_offset)) | v) & mask - } - } - } - - /// Gets at most `num` bits from this reader, and append them to the `dst` byte slice, starting - /// at bit offset `offset`. - /// - /// Returns the actual number of bits appended. In case either the `dst` slice doesn't have - /// enough space or the current reader doesn't have enough bits to consume, the returned value - /// will be less than the input `num_bits`. - /// - /// # Preconditions - /// * `offset` MUST < dst.len() * 8 - pub fn get_bits(&mut self, dst: &mut [u8], offset: usize, num_bits: usize) -> usize { - debug_assert!(offset < dst.len() * 8); - - let remaining_bits = (self.total_bytes - self.byte_offset) * 8 - self.bit_offset; - let num_bits_to_read = min(remaining_bits, min(num_bits, dst.len() * 8 - offset)); - let mut i = 0; - - // First consume all the remaining bits from the `buffered_values` if there're any. - if likely(self.bit_offset != 0) { - i += self.get_bits_buffered(dst, offset, num_bits_to_read); - } - - debug_assert!(self.bit_offset == 0 || i == num_bits_to_read); - - // Check if there's opportunity to directly copy bytes using `memcpy`. - if (offset + i).is_multiple_of(8) && i < num_bits_to_read { - let num_bytes = (num_bits_to_read - i) / 8; - let dst_byte_offset = (offset + i) / 8; - if num_bytes > 0 { - memcpy( - &self.buffer[self.byte_offset..self.byte_offset + num_bytes], - &mut dst[dst_byte_offset..], - ); - i += num_bytes * 8; - self.byte_offset += num_bytes; - self.reload_buffer_values(); - } - } - - debug_assert!(!(offset + i).is_multiple_of(8) || num_bits_to_read - i < 8); - - // Now copy the remaining bits if there's any. - while i < num_bits_to_read { - i += self.get_bits_buffered(dst, offset + i, num_bits_to_read - i); - } - - num_bits_to_read - } - - /// Consume at most `n` bits from `buffered_values`. Returns the actual number of bits consumed. - /// - /// # Postcondition - /// - either bits from `buffered_values` are completely drained (i.e., `bit_offset` == 0) - /// - OR the `num_bits` is < the number of remaining bits in `buffered_values` and thus the - /// returned value is < `num_bits`. - /// - /// Either way, the returned value is in range [0, 64]. - #[inline] - fn get_bits_buffered(&mut self, dst: &mut [u8], offset: usize, num_bits: usize) -> usize { - if unlikely(num_bits == 0) { - return 0; - } - - let n = min(num_bits, 64 - self.bit_offset); - let offset_i = offset / 8; - let offset_r = offset % 8; - - // Extract the value to read out of the buffer - let mut v = trailing_bits(self.buffered_values >> self.bit_offset, n); - - // Read the first byte always because n > 0 - dst[offset_i] |= (v << offset_r) as u8; - v >>= 8 - offset_r; - - // Read the rest of the bytes - ((offset_i + 1)..(offset_i + usize::div_ceil(n + offset_r, 8))).for_each(|i| { - dst[i] |= v as u8; - v >>= 8; - }); - - self.bit_offset += n; - if self.bit_offset == 64 { - self.byte_offset += 8; - self.bit_offset -= 64; - self.reload_buffer_values(); - } - - n - } - - /// Skips at most `num` bits from this reader. - /// - /// Returns the actual number of bits skipped. - pub fn skip_bits(&mut self, num_bits: usize) -> usize { - let remaining_bits = (self.total_bytes - self.byte_offset) * 8 - self.bit_offset; - let num_bits_to_read = min(remaining_bits, num_bits); - let mut i = 0; - - // First skip all the remaining bits by updating the offsets of `buffered_values`. - if likely(self.bit_offset != 0) { - let n = 64 - self.bit_offset; - if num_bits_to_read < n { - self.bit_offset += num_bits_to_read; - i = num_bits_to_read; - } else { - self.byte_offset += 8; - self.bit_offset = 0; - i = n; - } - } - - // Check if there's opportunity to skip by byte - if i + 7 < num_bits_to_read { - let num_bytes = (num_bits_to_read - i) / 8; - i += num_bytes * 8; - self.byte_offset += num_bytes; - } - - if self.bit_offset == 0 { - self.reload_buffer_values(); - } - - // Now skip the remaining bits if there's any. - if i < num_bits_to_read { - self.bit_offset += num_bits_to_read - i; - } - - num_bits_to_read - } - - /// Reads a batch of `u32` values encoded using `num_bits` of bits, into `dst`. - /// - /// # Safety - /// - /// This method asusumes the following: - /// - /// - the `num_bits` is <= 64 - /// - the remaining number of bits to read in this reader is >= `total * num_bits`. - /// - /// Undefined behavior will happen if any of the above assumptions is violated. - /// - /// Unlike `[get_batch]`, this method removes a few checks such as checking the remaining number - /// of bits as well as checking the bit width for the element type in `dst`. Therefore, it is - /// more efficient. - pub unsafe fn get_u32_batch(&mut self, mut dst: *mut u32, total: usize, num_bits: usize) { - let mut i = 0; - - // First align bit offset to byte offset - if likely(self.bit_offset != 0) { - while i < total && self.bit_offset != 0 { - *dst = self.get_u32_value(num_bits); - dst = dst.offset(1); - i += 1; - } - } - - let in_buf = &self.buffer.as_slice()[self.byte_offset..]; - let mut in_ptr = in_buf as *const [u8] as *const u8 as *const u32; - while total - i >= 32 { - in_ptr = unpack32(in_ptr, dst, num_bits); - self.byte_offset += 4 * num_bits; - dst = dst.offset(32); - i += 32; - } - - self.reload_buffer_values(); - while i < total { - *dst = self.get_u32_value(num_bits); - dst = dst.offset(1); - i += 1; - } - } - - pub fn get_batch(&mut self, batch: &mut [T], num_bits: usize) -> usize { - debug_assert!(num_bits <= 32); - debug_assert!(num_bits <= size_of::() * 8); - - let mut values_to_read = batch.len(); - let needed_bits = num_bits * values_to_read; - let remaining_bits = (self.total_bytes - self.byte_offset) * 8 - self.bit_offset; - if remaining_bits < needed_bits { - values_to_read = remaining_bits / num_bits; - } - - let mut i = 0; - - // First align bit offset to byte offset - if likely(self.bit_offset != 0) { - while i < values_to_read && self.bit_offset != 0 { - batch[i] = self - .get_value(num_bits) - .expect("expected to have more data"); - i += 1; - } - } - - unsafe { - let in_buf = &self.buffer.as_slice()[self.byte_offset..]; - let mut in_ptr = in_buf as *const [u8] as *const u8 as *const u32; - // FIXME assert!(memory::is_ptr_aligned(in_ptr)); - if size_of::() == 4 { - while values_to_read - i >= 32 { - let out_ptr = &mut batch[i..] as *mut [T] as *mut T as *mut u32; - in_ptr = unpack32(in_ptr, out_ptr, num_bits); - self.byte_offset += 4 * num_bits; - i += 32; - } - } else { - let mut out_buf = [0u32; 32]; - let out_ptr = &mut out_buf as &mut [u32] as *mut [u32] as *mut u32; - while values_to_read - i >= 32 { - in_ptr = unpack32(in_ptr, out_ptr, num_bits); - self.byte_offset += 4 * num_bits; - for n in 0..32 { - // We need to copy from smaller size to bigger size to avoid - // overwriting other memory regions. - if size_of::() > size_of::() { - std::ptr::copy_nonoverlapping( - out_buf[n..].as_ptr(), - &mut batch[i] as *mut T as *mut u32, - 1, - ); - } else { - std::ptr::copy_nonoverlapping( - out_buf[n..].as_ptr() as *const T, - &mut batch[i] as *mut T, - 1, - ); - } - i += 1; - } - } - } - } - - debug_assert!(values_to_read - i < 32); - - self.reload_buffer_values(); - while i < values_to_read { - batch[i] = self - .get_value(num_bits) - .expect("expected to have more data"); - i += 1; - } - - values_to_read - } - - /// Reads a `num_bytes`-sized value from this buffer and return it. - /// `T` needs to be a little-endian native type. The value is assumed to be byte - /// aligned so the bit reader will be advanced to the start of the next byte before - /// reading the value. - /// Returns `Some` if there's enough bytes left to form a value of `T`. - /// Otherwise `None`. - pub fn get_aligned(&mut self, num_bytes: usize) -> Option { - debug_assert!(8 >= size_of::()); - debug_assert!(num_bytes <= size_of::()); - - let bytes_read = self.bit_offset.div_ceil(8); - if unlikely(self.byte_offset + bytes_read + num_bytes > self.total_bytes) { - return None; - } - - if bytes_read + num_bytes > 8 { - // There may be still unread bytes in buffered_values; however, just reloading seems to - // be faster than stitching the buffer with the next buffer based on micro benchmarks - // because reloading logic can be simpler - - // Advance byte_offset to next unread byte - self.byte_offset += bytes_read; - // Reset buffered_values - self.reload_buffer_values(); - self.bit_offset = 0 - } else { - // Advance bit_offset to next unread byte - self.bit_offset = bytes_read * 8; - } - - let v = T::from(trailing_bits( - self.buffered_values >> self.bit_offset, - num_bytes * 8, - )); - self.bit_offset += num_bytes * 8; - - if self.bit_offset == 64 { - self.byte_offset += 8; - self.bit_offset -= 64; - self.reload_buffer_values(); - } - - Some(v) - } - - /// Reads a VLQ encoded (in little endian order) int from the stream. - /// The encoded int must start at the beginning of a byte. - /// - /// Returns `None` if there's not enough bytes in the stream. `Some` otherwise. - pub fn get_vlq_int(&mut self) -> Option { - let mut shift = 0; - let mut v: i64 = 0; - while let Some(byte) = self.get_aligned::(1) { - v |= ((byte & 0x7F) as i64) << shift; - shift += 7; - debug_assert!( - shift <= MAX_VLQ_BYTE_LEN * 7, - "Num of bytes exceed MAX_VLQ_BYTE_LEN ({MAX_VLQ_BYTE_LEN})" - ); - if likely(byte & 0x80 == 0) { - return Some(v); - } - } - None - } - - /// Reads a zigzag-VLQ encoded (in little endian order) int from the stream - /// Zigzag-VLQ is a variant of VLQ encoding where negative and positive numbers are - /// encoded in a zigzag fashion. - /// See: https://developers.google.com/protocol-buffers/docs/encoding - /// - /// Note: the encoded int must start at the beginning of a byte. - /// - /// Returns `None` if the number of bytes there's not enough bytes in the stream. - /// `Some` otherwise. - #[inline] - pub fn get_zigzag_vlq_int(&mut self) -> Option { - self.get_vlq_int().map(|v| { - let u = v as u64; - (u >> 1) as i64 ^ -((u & 1) as i64) - }) - } - - fn reload_buffer_values(&mut self) { - let bytes_to_read = self.total_bytes - self.byte_offset; - self.buffered_values = if 8 > bytes_to_read { - read_num_bytes_u64(bytes_to_read, &self.buffer.as_slice()[self.byte_offset..]) - } else { - read_u64(&self.buffer.as_slice()[self.byte_offset..]) - }; - } -} - -impl From> for BitReader { - #[inline] - fn from(vec: Vec) -> Self { - let len = vec.len(); - BitReader::new(Buffer::from(vec.as_slice()), len) - } -} - -/// Returns the nearest multiple of `factor` that is `>=` than `num`. Here `factor` must -/// be a power of 2. -/// -/// Copied from the arrow crate to make arrow optional -pub fn round_upto_power_of_2(num: usize, factor: usize) -> usize { - debug_assert!(factor > 0 && (factor & (factor - 1)) == 0); - (num + (factor - 1)) & !(factor - 1) -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::parquet::util::test_common::*; - - use rand::{ - distr::{Distribution, StandardUniform}, - RngExt, - }; - use std::fmt::Debug; - - #[test] - fn test_read_num_bytes_u64() { - let buffer: Vec = vec![0, 1, 2, 3, 4, 5, 6, 7]; - for size in 0..buffer.len() { - assert_eq!( - read_num_bytes_u64(size, &buffer), - read_num_bytes!(u64, size, &buffer), - ); - } - } - - #[test] - fn test_read_u64() { - let buffer: Vec = vec![0, 1, 2, 3, 4, 5, 6, 7]; - assert_eq!(read_u64(&buffer), read_num_bytes!(u64, 8, &buffer),); - } - - #[test] - fn test_read_num_bytes_u32() { - let buffer: Vec = vec![0, 1, 2, 3]; - for size in 0..buffer.len() { - assert_eq!( - read_num_bytes_u32(size, &buffer), - read_num_bytes!(u32, size, &buffer), - ); - } - } - - #[test] - fn test_read_u32() { - let buffer: Vec = vec![0, 1, 2, 3]; - assert_eq!(read_u32(&buffer), read_num_bytes!(u32, 4, &buffer),); - } - - #[test] - fn test_bit_reader_get_byte_offset() { - let buffer = vec![255; 10]; - let mut bit_reader = BitReader::from(buffer); - assert_eq!(bit_reader.get_byte_offset(), 0); // offset (0 bytes, 0 bits) - bit_reader.get_value::(6); - assert_eq!(bit_reader.get_byte_offset(), 1); // offset (0 bytes, 6 bits) - bit_reader.get_value::(10); - assert_eq!(bit_reader.get_byte_offset(), 2); // offset (0 bytes, 16 bits) - bit_reader.get_value::(20); - assert_eq!(bit_reader.get_byte_offset(), 5); // offset (0 bytes, 36 bits) - bit_reader.get_value::(30); - assert_eq!(bit_reader.get_byte_offset(), 9); // offset (8 bytes, 2 bits) - } - - #[test] - fn test_bit_reader_get_value() { - let buffer = vec![255, 0]; - let mut bit_reader = BitReader::from(buffer); - assert_eq!(bit_reader.get_value::(1), Some(1)); - assert_eq!(bit_reader.get_value::(2), Some(3)); - assert_eq!(bit_reader.get_value::(3), Some(7)); - assert_eq!(bit_reader.get_value::(4), Some(3)); - } - - #[test] - fn test_bit_reader_get_value_boundary() { - let buffer = vec![10, 0, 0, 0, 20, 0, 30, 0, 0, 0, 40, 0]; - let mut bit_reader = BitReader::from(buffer); - assert_eq!(bit_reader.get_value::(32), Some(10)); - assert_eq!(bit_reader.get_value::(16), Some(20)); - assert_eq!(bit_reader.get_value::(32), Some(30)); - assert_eq!(bit_reader.get_value::(16), Some(40)); - } - - #[test] - fn test_bit_reader_get_aligned() { - // 01110101 11001011 - let buffer = Buffer::from(&[0x75, 0xCB]); - let mut bit_reader = BitReader::new_all(buffer.clone()); - assert_eq!(bit_reader.get_value::(3), Some(5)); - assert_eq!(bit_reader.get_aligned::(1), Some(203)); - assert_eq!(bit_reader.get_value::(1), None); - bit_reader.reset(buffer); - assert_eq!(bit_reader.get_aligned::(3), None); - } - - #[test] - fn test_bit_reader_get_vlq_int() { - // 10001001 00000001 11110010 10110101 00000110 - let buffer: Vec = vec![0x89, 0x01, 0xF2, 0xB5, 0x06]; - let mut bit_reader = BitReader::from(buffer); - assert_eq!(bit_reader.get_vlq_int(), Some(137)); - assert_eq!(bit_reader.get_vlq_int(), Some(105202)); - } - - #[test] - fn test_bit_reader_get_zigzag_vlq_int() { - let buffer: Vec = vec![0, 1, 2, 3]; - let mut bit_reader = BitReader::from(buffer); - assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(0)); - assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(-1)); - assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(1)); - assert_eq!(bit_reader.get_zigzag_vlq_int(), Some(-2)); - } - - #[test] - fn test_set_bit() { - let mut buffer = vec![0, 0, 0]; - set_bit(&mut buffer[..], 1); - assert_eq!(buffer, vec![2, 0, 0]); - set_bit(&mut buffer[..], 4); - assert_eq!(buffer, vec![18, 0, 0]); - unset_bit(&mut buffer[..], 1); - assert_eq!(buffer, vec![16, 0, 0]); - set_bit(&mut buffer[..], 10); - assert_eq!(buffer, vec![16, 4, 0]); - set_bit(&mut buffer[..], 10); - assert_eq!(buffer, vec![16, 4, 0]); - set_bit(&mut buffer[..], 11); - assert_eq!(buffer, vec![16, 12, 0]); - unset_bit(&mut buffer[..], 10); - assert_eq!(buffer, vec![16, 8, 0]); - } - - #[test] - fn test_set_bits() { - for offset in 0..=16 { - for length in 0..=16 { - let mut actual = vec![0, 0, 0, 0]; - set_bits(&mut actual[..], offset, length); - let mut expected = vec![0, 0, 0, 0]; - for i in 0..length { - set_bit(&mut expected, offset + i); - } - assert_eq!(actual, expected); - } - } - } - - #[test] - fn test_get_bit() { - // 00001101 - assert!(get_bit(&[0b00001101], 0)); - assert!(!get_bit(&[0b00001101], 1)); - assert!(get_bit(&[0b00001101], 2)); - assert!(get_bit(&[0b00001101], 3)); - - // 01001001 01010010 - assert!(get_bit(&[0b01001001, 0b01010010], 0)); - assert!(!get_bit(&[0b01001001, 0b01010010], 1)); - assert!(!get_bit(&[0b01001001, 0b01010010], 2)); - assert!(get_bit(&[0b01001001, 0b01010010], 3)); - assert!(!get_bit(&[0b01001001, 0b01010010], 4)); - assert!(!get_bit(&[0b01001001, 0b01010010], 5)); - assert!(get_bit(&[0b01001001, 0b01010010], 6)); - assert!(!get_bit(&[0b01001001, 0b01010010], 7)); - assert!(!get_bit(&[0b01001001, 0b01010010], 8)); - assert!(get_bit(&[0b01001001, 0b01010010], 9)); - assert!(!get_bit(&[0b01001001, 0b01010010], 10)); - assert!(!get_bit(&[0b01001001, 0b01010010], 11)); - assert!(get_bit(&[0b01001001, 0b01010010], 12)); - assert!(!get_bit(&[0b01001001, 0b01010010], 13)); - assert!(get_bit(&[0b01001001, 0b01010010], 14)); - assert!(!get_bit(&[0b01001001, 0b01010010], 15)); - } - - #[test] - fn test_log2() { - assert_eq!(log2(1), 0); - assert_eq!(log2(2), 1); - assert_eq!(log2(3), 2); - assert_eq!(log2(4), 2); - assert_eq!(log2(5), 3); - assert_eq!(log2(5), 3); - assert_eq!(log2(6), 3); - assert_eq!(log2(7), 3); - assert_eq!(log2(8), 3); - assert_eq!(log2(9), 4); - } - - #[test] - fn test_skip() { - let mut writer = BitWriter::new(5); - let old_offset = writer.skip(1).expect("skip() should return OK"); - writer.put_aligned(42, 4); - writer.put_aligned_offset(0x10, 1, old_offset); - let result = writer.consume(); - assert_eq!(result.as_ref(), [0x10, 42, 0, 0, 0]); - - writer = BitWriter::new(4); - let result = writer.skip(5); - assert!(result.is_err()); - } - - #[test] - fn test_get_next_byte_ptr() { - let mut writer = BitWriter::new(5); - { - let first_byte = writer - .get_next_byte_ptr(1) - .expect("get_next_byte_ptr() should return OK"); - first_byte[0] = 0x10; - } - writer.put_aligned(42, 4); - let result = writer.consume(); - assert_eq!(result.as_ref(), [0x10, 42, 0, 0, 0]); - } - - #[test] - fn test_consume_flush_buffer() { - let mut writer1 = BitWriter::new(3); - let mut writer2 = BitWriter::new(3); - for i in 1..10 { - writer1.put_value(i, 4); - writer2.put_value(i, 4); - } - let res1 = writer1.flush_buffer(); - let res2 = writer2.consume(); - assert_eq!(res1, &res2[..]); - } - - #[test] - fn test_put_get_bool() { - let len = 8; - let mut writer = BitWriter::new(len); - - for i in 0..8 { - let result = writer.put_value(i % 2, 1); - assert!(result); - } - - writer.flush(); - { - let buffer = writer.buffer(); - assert_eq!(buffer[0], 0b10101010); - } - - // Write 00110011 - for i in 0..8 { - let result = match i { - 0 | 1 | 4 | 5 => writer.put_value(false as u64, 1), - _ => writer.put_value(true as u64, 1), - }; - assert!(result); - } - writer.flush(); - { - let buffer = writer.buffer(); - assert_eq!(buffer[0], 0b10101010); - assert_eq!(buffer[1], 0b11001100); - } - - let mut reader = BitReader::from(writer.consume()); - - for i in 0..8 { - let val = reader - .get_value::(1) - .expect("get_value() should return OK"); - assert_eq!(val, i % 2); - } - - for i in 0..8 { - let val = reader - .get_value::(1) - .expect("get_value() should return OK"); - match i { - 0 | 1 | 4 | 5 => assert!(!val), - _ => assert!(val), - } - } - } - - #[test] - fn test_put_value_roundtrip() { - test_put_value_rand_numbers(32, 2); - test_put_value_rand_numbers(32, 3); - test_put_value_rand_numbers(32, 4); - test_put_value_rand_numbers(32, 5); - test_put_value_rand_numbers(32, 6); - test_put_value_rand_numbers(32, 7); - test_put_value_rand_numbers(32, 8); - test_put_value_rand_numbers(64, 16); - test_put_value_rand_numbers(64, 24); - test_put_value_rand_numbers(64, 32); - } - - fn test_put_value_rand_numbers(total: usize, num_bits: usize) { - assert!(num_bits < 64); - let num_bytes = num_bits.div_ceil(8); - let mut writer = BitWriter::new(num_bytes * total); - let values: Vec = random_numbers::(total) - .iter() - .map(|v| v & ((1 << num_bits) - 1)) - .collect(); - (0..total).for_each(|i| { - assert!( - writer.put_value(values[i], num_bits), - "[{i}]: put_value() failed" - ); - }); - - let mut reader = BitReader::from(writer.consume()); - (0..total).for_each(|i| { - let v = reader - .get_value::(num_bits) - .expect("get_value() should return OK"); - assert_eq!( - v, values[i], - "[{}]: expected {} but got {}", - i, values[i], v - ); - }); - } - - #[test] - fn test_get_bits() { - const NUM_BYTES: usize = 100; - - let mut vec = vec![0; NUM_BYTES]; - let total_num_bits = NUM_BYTES * 8; - let v = random_bools(total_num_bits); - (0..total_num_bits).for_each(|i| { - if v[i] { - set_bit(&mut vec, i); - } else { - unset_bit(&mut vec, i); - } - }); - - let expected = vec.clone(); - - // test reading the first time from a buffer - for &(offset, num_bits) in [(0, 10), (2, 10), (8, 16), (25, 40), (7, 64)].iter() { - let mut reader = BitReader::from(vec.clone()); - let mut buffer = vec![0; NUM_BYTES]; - - let actual_bits_read = reader.get_bits(&mut buffer, offset, num_bits); - let expected_bits_read = ::std::cmp::min(buffer.len() * 8 - offset, num_bits); - assert_eq!(expected_bits_read, actual_bits_read); - - for i in 0..actual_bits_read { - assert_eq!(get_bit(&expected, i), get_bit(&buffer, offset + i)); - } - } - - // test reading consecutively from a buffer - let mut reader = BitReader::from(vec); - let mut buffer = vec![0; NUM_BYTES]; - let mut rng = rand::rng(); - let mut bits_read = 0; - - loop { - if bits_read >= total_num_bits { - break; - } - let n: u64 = rng.random(); - let num_bits = n % 20; - bits_read += reader.get_bits(&mut buffer, bits_read, num_bits as usize); - } - - assert_eq!(total_num_bits, bits_read); - assert_eq!(&expected, &buffer); - } - - #[test] - fn test_skip_bits() { - const NUM_BYTES: usize = 100; - - let mut vec = vec![0; NUM_BYTES]; - let total_num_bits = NUM_BYTES * 8; - let v = random_bools(total_num_bits); - (0..total_num_bits).for_each(|i| { - if v[i] { - set_bit(&mut vec, i); - } else { - unset_bit(&mut vec, i); - } - }); - - let expected = vec.clone(); - - // test skipping and check the next value - let mut reader = BitReader::from(vec); - let mut bits_read = 0; - for &num_bits in [10, 60, 8].iter() { - let actual_bits_read = reader.skip_bits(num_bits); - assert_eq!(num_bits, actual_bits_read); - - bits_read += num_bits; - assert_eq!(Some(get_bit(&expected, bits_read)), reader.get_value(1)); - bits_read += 1; - } - - // test skipping consecutively - let mut rng = rand::rng(); - loop { - if bits_read >= total_num_bits { - break; - } - let n: u64 = rng.random(); - let num_bits = n % 20; - bits_read += reader.skip_bits(num_bits as usize); - } - - assert_eq!(total_num_bits, bits_read); - } - - #[test] - fn test_get_batch() { - const SIZE: &[usize] = &[1, 31, 32, 33, 128, 129]; - for s in SIZE { - for i in 0..33 { - match i { - 0..=8 => test_get_batch_helper::(*s, i), - 9..=16 => test_get_batch_helper::(*s, i), - _ => test_get_batch_helper::(*s, i), - } - } - } - } - - fn test_get_batch_helper(total: usize, num_bits: usize) - where - T: FromBytes + Default + Clone + Debug + Eq, - { - assert!(num_bits <= 32); - let num_bytes = num_bits.div_ceil(8); - let mut writer = BitWriter::new(num_bytes * total); - - let values: Vec = random_numbers::(total) - .iter() - .map(|v| v & ((1u64 << num_bits) - 1) as u32) - .collect(); - - // Generic values used to check against actual values read from `get_batch`. - let expected_values: Vec = values.iter().map(|v| from_ne_slice(v.as_bytes())).collect(); - - (0..total).for_each(|i| { - assert!(writer.put_value(values[i] as u64, num_bits)); - }); - - let buf = writer.consume(); - let mut reader = BitReader::from(buf); - let mut batch = vec![T::default(); values.len()]; - let values_read = reader.get_batch::(&mut batch, num_bits); - assert_eq!(values_read, values.len()); - for i in 0..batch.len() { - assert_eq!( - batch[i], expected_values[i], - "num_bits = {num_bits}, index = {i}" - ); - } - } - - #[test] - fn test_get_u32_batch() { - const SIZE: &[usize] = &[1, 31, 32, 33, 128, 129]; - for total in SIZE { - for num_bits in 1..33 { - let num_bytes = usize::div_ceil(num_bits, 8); - let mut writer = BitWriter::new(num_bytes * total); - - let values: Vec = random_numbers::(*total) - .iter() - .map(|v| v & ((1u64 << num_bits) - 1) as u32) - .collect(); - - (0..*total).for_each(|i| { - assert!(writer.put_value(values[i] as u64, num_bits)); - }); - - let buf = writer.consume(); - let mut reader = BitReader::from(buf); - let mut batch = vec![0u32; values.len()]; - unsafe { - reader.get_u32_batch(batch.as_mut_ptr(), *total, num_bits); - } - for i in 0..batch.len() { - assert_eq!(batch[i], values[i], "num_bits = {num_bits}, index = {i}"); - } - } - } - } - - #[test] - fn test_put_aligned_roundtrip() { - test_put_aligned_rand_numbers::(4, 3); - test_put_aligned_rand_numbers::(16, 5); - test_put_aligned_rand_numbers::(32, 7); - test_put_aligned_rand_numbers::(32, 9); - test_put_aligned_rand_numbers::(32, 11); - test_put_aligned_rand_numbers::(32, 13); - test_put_aligned_rand_numbers::(32, 17); - test_put_aligned_rand_numbers::(32, 23); - } - - fn test_put_aligned_rand_numbers(total: usize, num_bits: usize) - where - T: Copy + FromBytes + AsBytes + Debug + PartialEq, - StandardUniform: Distribution, - { - assert!(num_bits <= 32); - assert_eq!(total % 2, 0); - - let aligned_value_byte_width = std::mem::size_of::(); - let value_byte_width = num_bits.div_ceil(8); - let mut writer = - BitWriter::new((total / 2) * (aligned_value_byte_width + value_byte_width)); - let values: Vec = random_numbers::(total / 2) - .iter() - .map(|v| v & ((1 << num_bits) - 1)) - .collect(); - let aligned_values = random_numbers::(total / 2); - - for i in 0..total { - let j = i / 2; - if i % 2 == 0 { - assert!( - writer.put_value(values[j] as u64, num_bits), - "[{i}]: put_value() failed" - ); - } else { - assert!( - writer.put_aligned::(aligned_values[j], aligned_value_byte_width), - "[{i}]: put_aligned() failed" - ); - } - } - - let mut reader = BitReader::from(writer.consume()); - for i in 0..total { - let j = i / 2; - if i % 2 == 0 { - let v = reader - .get_value::(num_bits) - .expect("get_value() should return OK"); - assert_eq!( - v, values[j] as u64, - "[{}]: expected {} but got {}", - i, values[j], v - ); - } else { - let v = reader - .get_aligned::(aligned_value_byte_width) - .expect("get_aligned() should return OK"); - assert_eq!( - v, aligned_values[j], - "[{}]: expected {:?} but got {:?}", - i, aligned_values[j], v - ); - } - } - } - - #[test] - fn test_put_vlq_int() { - let total = 64; - let mut writer = BitWriter::new(total * 32); - let values = random_numbers::(total); - (0..total).for_each(|i| { - assert!( - writer.put_vlq_int(values[i] as u64), - "[{i}]; put_vlq_int() failed" - ); - }); - - let mut reader = BitReader::from(writer.consume()); - (0..total).for_each(|i| { - let v = reader - .get_vlq_int() - .expect("get_vlq_int() should return OK"); - assert_eq!( - v as u32, values[i], - "[{}]: expected {} but got {}", - i, values[i], v - ); - }); - } - - #[test] - fn test_put_zigzag_vlq_int() { - let total = 64; - let mut writer = BitWriter::new(total * 32); - let values = random_numbers::(total); - (0..total).for_each(|i| { - assert!( - writer.put_zigzag_vlq_int(values[i] as i64), - "[{i}]; put_zigzag_vlq_int() failed" - ); - }); - - let mut reader = BitReader::from(writer.consume()); - (0..total).for_each(|i| { - let v = reader - .get_zigzag_vlq_int() - .expect("get_zigzag_vlq_int() should return OK"); - assert_eq!( - v as i32, values[i], - "[{}]: expected {} but got {}", - i, values[i], v - ); - }); - } -} diff --git a/native/core/src/common/buffer.rs b/native/core/src/common/buffer.rs deleted file mode 100644 index 8b437a9f18..0000000000 --- a/native/core/src/common/buffer.rs +++ /dev/null @@ -1,366 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use crate::common::bit; -use crate::execution::operators::ExecutionError; -use arrow::buffer::Buffer as ArrowBuffer; -use std::{ - alloc::{handle_alloc_error, Layout}, - ptr::NonNull, - sync::Arc, -}; - -/// A buffer implementation. This is very similar to Arrow's [`MutableBuffer`] implementation, -/// except that there are two modes depending on whether `owned` is true or false. -/// -/// If `owned` is true, this behaves the same way as a Arrow [`MutableBuffer`], and the struct is -/// the unique owner for the memory it wraps. The holder of this buffer can read or write the -/// buffer, and the buffer itself will be released when it goes out of scope. -/// -/// Also note that, in `owned` mode, the buffer is always filled with 0s, and its length is always -/// equal to its capacity. It's up to the caller to decide which part of the buffer contains valid -/// data. -/// -/// If `owned` is false, this buffer is an alias to another buffer. The buffer itself becomes -/// immutable and can only be read. -#[derive(Debug)] -pub struct CometBuffer { - data: NonNull, - len: usize, - capacity: usize, - /// Whether this buffer owns the data it points to. - owned: bool, - /// The allocation instance for this buffer. - allocation: Arc, -} - -unsafe impl Sync for CometBuffer {} -unsafe impl Send for CometBuffer {} - -/// All buffers are aligned to 64 bytes. -const ALIGNMENT: usize = 64; - -impl CometBuffer { - /// Initializes a owned buffer filled with 0. - pub fn new(capacity: usize) -> Self { - let aligned_capacity = bit::round_upto_power_of_2(capacity, ALIGNMENT); - unsafe { - let layout = Layout::from_size_align_unchecked(aligned_capacity, ALIGNMENT); - let ptr = std::alloc::alloc_zeroed(layout); - Self { - data: NonNull::new(ptr).unwrap_or_else(|| handle_alloc_error(layout)), - len: aligned_capacity, - capacity: aligned_capacity, - owned: true, - allocation: Arc::new(CometBufferAllocation::new()), - } - } - } - - /// Returns the capacity of this buffer. - pub fn capacity(&self) -> usize { - self.capacity - } - - /// Returns the length (i.e., number of bytes) in this buffer. - pub fn len(&self) -> usize { - self.len - } - - /// Whether this buffer is empty. - pub fn is_empty(&self) -> bool { - self.len == 0 - } - - /// Returns the data stored in this buffer as a slice. - pub fn as_slice(&self) -> &[u8] { - self - } - - /// Returns the data stored in this buffer as a mutable slice. - pub fn as_slice_mut(&mut self) -> &mut [u8] { - debug_assert!(self.owned, "cannot modify un-owned buffer"); - self - } - - /// Returns a raw pointer to this buffer's internal memory - /// This pointer is guaranteed to be aligned along cache-lines. - #[inline] - pub const fn as_ptr(&self) -> *const u8 { - self.data.as_ptr() - } - - /// Returns a mutable raw pointer to this buffer's internal memory - /// This pointer is guaranteed to be aligned along cache-lines. - #[inline] - pub fn as_mut_ptr(&mut self) -> *mut u8 { - debug_assert!(self.owned, "cannot modify un-owned buffer"); - self.data.as_ptr() - } - - /// Returns an immutable Arrow buffer on the content of this buffer. - /// - /// # Safety - /// - /// This function is highly unsafe since it leaks the raw pointer to the memory region that the - /// originally this buffer is tracking. Because of this, the caller of this function is - /// expected to make sure the returned immutable [`ArrowBuffer`] will never live longer than the - /// this buffer. Otherwise it will result to dangling pointers. - /// - /// In the particular case of the columnar reader, we'll guarantee the above since the reader - /// itself is closed at the very end, after the Spark task is completed (either successfully or - /// unsuccessfully) through task completion listener. - /// - /// When re-using [`MutableVector`] in Comet native operators, across multiple input batches, - /// because of the iterator-style pattern, the content of the original mutable buffer will only - /// be updated once upstream operators fully consumed the previous output batch. For breaking - /// operators, they are responsible for copying content out of the buffers. - pub unsafe fn to_arrow(&self) -> Result { - let ptr = NonNull::new_unchecked(self.data.as_ptr()); - self.check_reference()?; - Ok(ArrowBuffer::from_custom_allocation( - ptr, - self.len, - Arc::::clone(&self.allocation), - )) - } - - /// Checks if this buffer is exclusively owned by Comet. If not, an error is returned. - /// We run this check when we want to update the buffer. If the buffer is also shared by - /// other components, e.g. one DataFusion operator stores the buffer, Comet cannot safely - /// modify the buffer. - pub fn check_reference(&self) -> Result<(), ExecutionError> { - if Arc::strong_count(&self.allocation) > 1 { - Err(ExecutionError::GeneralError( - "Error on modifying a buffer which is not exclusively owned by Comet".to_string(), - )) - } else { - Ok(()) - } - } - - /// Resets this buffer by filling all bytes with zeros. - pub fn reset(&mut self) { - debug_assert!(self.owned, "cannot modify un-owned buffer"); - unsafe { - std::ptr::write_bytes(self.as_mut_ptr(), 0, self.len); - } - } - - /// Resize this buffer to the `new_capacity`. For additional bytes allocated, they are filled - /// with 0. if `new_capacity` is less than the current capacity of this buffer, this is a no-op. - #[inline(always)] - pub fn resize(&mut self, new_capacity: usize) { - assert!(self.owned, "cannot modify un-owned buffer"); - if new_capacity > self.len { - let (ptr, new_capacity) = - unsafe { Self::reallocate(self.data, self.capacity, new_capacity) }; - let diff = new_capacity - self.len; - self.data = ptr; - self.capacity = new_capacity; - // write the value - unsafe { self.data.as_ptr().add(self.len).write_bytes(0, diff) }; - self.len = new_capacity; - } - } - - unsafe fn reallocate( - ptr: NonNull, - old_capacity: usize, - new_capacity: usize, - ) -> (NonNull, usize) { - let new_capacity = bit::round_upto_power_of_2(new_capacity, ALIGNMENT); - let new_capacity = std::cmp::max(new_capacity, old_capacity * 2); - let raw_ptr = std::alloc::realloc( - ptr.as_ptr(), - Layout::from_size_align_unchecked(old_capacity, ALIGNMENT), - new_capacity, - ); - let ptr = NonNull::new(raw_ptr).unwrap_or_else(|| { - handle_alloc_error(Layout::from_size_align_unchecked(new_capacity, ALIGNMENT)) - }); - (ptr, new_capacity) - } -} - -impl Drop for CometBuffer { - fn drop(&mut self) { - if self.owned { - unsafe { - std::alloc::dealloc( - self.data.as_ptr(), - Layout::from_size_align_unchecked(self.capacity, ALIGNMENT), - ) - } - } - } -} - -impl PartialEq for CometBuffer { - fn eq(&self, other: &CometBuffer) -> bool { - if self.data.as_ptr() == other.data.as_ptr() { - return true; - } - if self.len != other.len { - return false; - } - if self.capacity != other.capacity { - return false; - } - self.as_slice() == other.as_slice() - } -} - -impl std::ops::Deref for CometBuffer { - type Target = [u8]; - - fn deref(&self) -> &[u8] { - unsafe { std::slice::from_raw_parts(self.as_ptr(), self.len) } - } -} - -impl std::ops::DerefMut for CometBuffer { - fn deref_mut(&mut self) -> &mut [u8] { - assert!(self.owned, "cannot modify un-owned buffer"); - unsafe { std::slice::from_raw_parts_mut(self.as_mut_ptr(), self.capacity) } - } -} - -#[derive(Debug)] -struct CometBufferAllocation {} - -impl CometBufferAllocation { - fn new() -> Self { - Self {} - } -} - -#[cfg(test)] -mod tests { - use super::*; - use arrow::buffer::Buffer as ArrowBuffer; - - impl CometBuffer { - pub fn from_ptr(ptr: *const u8, len: usize, capacity: usize) -> Self { - assert_eq!( - capacity % ALIGNMENT, - 0, - "input buffer is not aligned to {ALIGNMENT} bytes" - ); - Self { - data: NonNull::new(ptr as *mut u8).unwrap_or_else(|| { - panic!( - "cannot create CometBuffer from (ptr: {ptr:?}, len: {len}, capacity: {capacity}" - ) - }), - len, - capacity, - owned: false, - allocation: Arc::new(CometBufferAllocation::new()), - } - } - - /// Extends this buffer (must be an owned buffer) by appending bytes from `src`, - /// starting from `offset`. - pub fn extend_from_slice(&mut self, offset: usize, src: &[u8]) { - assert!(self.owned, "cannot modify un-owned buffer"); - assert!( - offset + src.len() <= self.capacity(), - "buffer overflow, offset = {}, src.len = {}, capacity = {}", - offset, - src.len(), - self.capacity() - ); - - unsafe { - let dst = self.data.as_ptr().add(offset); - std::ptr::copy_nonoverlapping(src.as_ptr(), dst, src.len()) - } - } - } - - #[test] - fn test_buffer_new() { - let buf = CometBuffer::new(63); - assert_eq!(64, buf.capacity()); - assert_eq!(64, buf.len()); - assert!(!buf.is_empty()); - } - - #[test] - fn test_resize() { - let mut buf = CometBuffer::new(1); - assert_eq!(64, buf.capacity()); - assert_eq!(64, buf.len()); - - buf.resize(100); - assert_eq!(128, buf.capacity()); - assert_eq!(128, buf.len()); - - // resize with less capacity is no-op - buf.resize(20); - assert_eq!(128, buf.capacity()); - assert_eq!(128, buf.len()); - } - - #[test] - fn test_extend_from_slice() { - let mut buf = CometBuffer::new(100); - buf.extend_from_slice(0, b"hello"); - assert_eq!(b"hello", &buf.as_slice()[0..5]); - - buf.extend_from_slice(5, b" world"); - assert_eq!(b"hello world", &buf.as_slice()[0..11]); - - buf.reset(); - buf.extend_from_slice(0, b"hello arrow"); - assert_eq!(b"hello arrow", &buf.as_slice()[0..11]); - } - - #[test] - fn test_to_arrow() { - let mut buf = CometBuffer::new(1); - - let str = b"aaaa bbbb cccc dddd"; - buf.extend_from_slice(0, str.as_slice()); - - assert_eq!(64, buf.len()); - assert_eq!(64, buf.capacity()); - assert_eq!(b"aaaa bbbb cccc dddd", &buf.as_slice()[0..str.len()]); - - unsafe { - let immutable_buf: ArrowBuffer = buf.to_arrow().unwrap(); - assert_eq!(64, immutable_buf.len()); - assert_eq!(str, &immutable_buf.as_slice()[0..str.len()]); - } - } - - #[test] - fn test_unowned() { - let arrow_buf = ArrowBuffer::from(b"hello comet"); - let buf = CometBuffer::from_ptr(arrow_buf.as_ptr(), arrow_buf.len(), arrow_buf.capacity()); - - assert_eq!(11, buf.len()); - assert_eq!(64, buf.capacity()); - assert_eq!(b"hello comet", &buf.as_slice()[0..11]); - - unsafe { - let arrow_buf2 = buf.to_arrow().unwrap(); - assert_eq!(arrow_buf, arrow_buf2); - } - } -} diff --git a/native/core/src/common/mod.rs b/native/core/src/common/mod.rs deleted file mode 100644 index dc539879fa..0000000000 --- a/native/core/src/common/mod.rs +++ /dev/null @@ -1,21 +0,0 @@ -// 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. - -#[macro_use] -pub mod bit; -mod buffer; -pub use buffer::*; diff --git a/native/core/src/debug/debug_batch_stream.rs b/native/core/src/debug/debug_batch_stream.rs index b59b6a36f7..37837a5daa 100644 --- a/native/core/src/debug/debug_batch_stream.rs +++ b/native/core/src/debug/debug_batch_stream.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; use std::fmt; use std::hash::{Hash, Hasher}; use std::sync::Arc; @@ -82,15 +81,20 @@ impl datafusion::physical_plan::ExecutionPlan for DebugExecutionDataStream { fn name(&self) -> &str { "DebugExecutionDataStream" } - fn as_any(&self) -> &dyn std::any::Any { - self - } fn properties(&self) -> &Arc { self.inner.properties() } fn children(&self) -> Vec<&Arc> { vec![&self.inner] } + fn apply_expressions( + &self, + _f: &mut dyn FnMut( + &dyn PhysicalExpr, + ) -> Result, + ) -> Result { + Ok(datafusion::common::tree_node::TreeNodeRecursion::Continue) + } fn with_new_children( self: Arc, children: Vec>, @@ -155,9 +159,6 @@ impl Hash for DebugExecutionDataPhyExpr { } impl PhysicalExpr for DebugExecutionDataPhyExpr { - fn as_any(&self) -> &dyn Any { - self - } fn data_type(&self, input_schema: &Schema) -> Result { self.inner.data_type(input_schema) } diff --git a/native/core/src/execution/columnar_to_row.rs b/native/core/src/execution/columnar_to_row.rs index ec2b633cc1..9a3616bef7 100644 --- a/native/core/src/execution/columnar_to_row.rs +++ b/native/core/src/execution/columnar_to_row.rs @@ -161,6 +161,7 @@ enum TypedArray<'a> { Float64(&'a Float64Array), Date32(&'a Date32Array), TimestampMicro(&'a TimestampMicrosecondArray), + Time64Nano(&'a Time64NanosecondArray), Decimal128(&'a Decimal128Array, u8), // array + precision String(&'a StringArray), LargeString(&'a LargeStringArray), @@ -200,6 +201,10 @@ impl<'a> TypedArray<'a> { DataType::Timestamp(TimeUnit::Microsecond, _) => Ok(TypedArray::TimestampMicro( downcast_array!(array, TimestampMicrosecondArray)?, )), + DataType::Time64(TimeUnit::Nanosecond) => Ok(TypedArray::Time64Nano(downcast_array!( + array, + Time64NanosecondArray + )?)), DataType::Decimal128(p, _) => Ok(TypedArray::Decimal128( downcast_array!(array, Decimal128Array)?, *p, @@ -267,6 +272,7 @@ impl<'a> TypedArray<'a> { Float64, Date32, TimestampMicro, + Time64Nano, Decimal128, String, LargeString, @@ -295,6 +301,7 @@ impl<'a> TypedArray<'a> { TypedArray::Float64(arr) => arr.value(row_idx).to_bits() as i64, TypedArray::Date32(arr) => arr.value(row_idx) as i64, TypedArray::TimestampMicro(arr) => arr.value(row_idx), + TypedArray::Time64Nano(arr) => arr.value(row_idx), TypedArray::Decimal128(arr, precision) if *precision <= MAX_LONG_DIGITS => { arr.value(row_idx) as i64 } @@ -317,7 +324,8 @@ impl<'a> TypedArray<'a> { | TypedArray::Float32(_) | TypedArray::Float64(_) | TypedArray::Date32(_) - | TypedArray::TimestampMicro(_) => false, + | TypedArray::TimestampMicro(_) + | TypedArray::Time64Nano(_) => false, TypedArray::Decimal128(_, precision) => *precision > MAX_LONG_DIGITS, _ => true, } @@ -380,6 +388,7 @@ enum TypedElements<'a> { Float64(&'a Float64Array), Date32(&'a Date32Array), TimestampMicro(&'a TimestampMicrosecondArray), + Time64Nano(&'a Time64NanosecondArray), Decimal128(&'a Decimal128Array, u8), String(&'a StringArray), LargeString(&'a LargeStringArray), @@ -418,6 +427,11 @@ impl<'a> TypedElements<'a> { return TypedElements::TimestampMicro(arr); } } + DataType::Time64(TimeUnit::Nanosecond) => { + if let Some(arr) = array.as_any().downcast_ref::() { + return TypedElements::Time64Nano(arr); + } + } DataType::Decimal128(p, _) => { if let Some(arr) = array.as_any().downcast_ref::() { return TypedElements::Decimal128(arr, *p); @@ -442,6 +456,7 @@ impl<'a> TypedElements<'a> { TypedElements::Int32(_) | TypedElements::Date32(_) | TypedElements::Float32(_) => 4, TypedElements::Int64(_) | TypedElements::TimestampMicro(_) + | TypedElements::Time64Nano(_) | TypedElements::Float64(_) => 8, TypedElements::Decimal128(_, p) if *p <= MAX_LONG_DIGITS => 8, _ => 8, // Variable-length uses 8 bytes for offset+length @@ -460,6 +475,7 @@ impl<'a> TypedElements<'a> { | TypedElements::Float64(_) | TypedElements::Date32(_) | TypedElements::TimestampMicro(_) + | TypedElements::Time64Nano(_) ) } @@ -479,6 +495,7 @@ impl<'a> TypedElements<'a> { Float64, Date32, TimestampMicro, + Time64Nano, Decimal128, String, LargeString, @@ -502,7 +519,8 @@ impl<'a> TypedElements<'a> { | TypedElements::Float32(_) | TypedElements::Float64(_) | TypedElements::Date32(_) - | TypedElements::TimestampMicro(_) => true, + | TypedElements::TimestampMicro(_) + | TypedElements::Time64Nano(_) => true, TypedElements::Decimal128(_, p) => *p <= MAX_LONG_DIGITS, _ => false, } @@ -521,6 +539,7 @@ impl<'a> TypedElements<'a> { TypedElements::Float64(arr) => arr.value(idx).to_bits() as i64, TypedElements::Date32(arr) => arr.value(idx) as i64, TypedElements::TimestampMicro(arr) => arr.value(idx), + TypedElements::Time64Nano(arr) => arr.value(idx), TypedElements::Decimal128(arr, _) => arr.value(idx) as i64, _ => 0, // Should not be called for variable-length types } @@ -655,6 +674,7 @@ impl<'a> TypedElements<'a> { TypedElements::Float64(arr) => bulk_copy_range!(arr, 8), TypedElements::Date32(arr) => bulk_copy_range!(arr, 4), TypedElements::TimestampMicro(arr) => bulk_copy_range!(arr, 8), + TypedElements::Time64Nano(arr) => bulk_copy_range!(arr, 8), _ => {} // Should not reach here due to supports_bulk_copy check } } @@ -827,7 +847,8 @@ fn is_fixed_width(data_type: &DataType) -> bool { | DataType::Float32 | DataType::Float64 | DataType::Date32 - | DataType::Timestamp(TimeUnit::Microsecond, _) => true, + | DataType::Timestamp(TimeUnit::Microsecond, _) + | DataType::Time64(TimeUnit::Nanosecond) => true, DataType::Decimal128(p, _) => *p <= MAX_LONG_DIGITS, _ => false, } @@ -1235,6 +1256,15 @@ impl ColumnarToRowContext { TimestampMicrosecondArray, |v: i64| v ), + DataType::Time64(TimeUnit::Nanosecond) => write_fixed_column_primitive!( + self, + array, + row_size, + field_offset_in_row, + num_rows, + Time64NanosecondArray, + |v: i64| v + ), DataType::Decimal128(precision, _) if *precision <= MAX_LONG_DIGITS => { write_fixed_column_primitive!( self, @@ -1360,6 +1390,9 @@ fn get_field_value(data_type: &DataType, array: &ArrayRef, row_idx: usize) -> Co DataType::Timestamp(TimeUnit::Microsecond, _) => { get_field_value_primitive!(array, row_idx, TimestampMicrosecondArray, |v: i64| v) } + DataType::Time64(TimeUnit::Nanosecond) => { + get_field_value_primitive!(array, row_idx, Time64NanosecondArray, |v: i64| v) + } DataType::Decimal128(precision, _) if *precision <= MAX_LONG_DIGITS => { get_field_value_primitive!(array, row_idx, Decimal128Array, |v: i128| v as i64) } diff --git a/native/core/src/execution/expressions/arithmetic.rs b/native/core/src/execution/expressions/arithmetic.rs index 320532d773..8d4c59a010 100644 --- a/native/core/src/execution/expressions/arithmetic.rs +++ b/native/core/src/execution/expressions/arithmetic.rs @@ -77,10 +77,6 @@ impl Hash for CheckedBinaryExpr { } impl PhysicalExpr for CheckedBinaryExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { self.child.fmt_sql(f) } diff --git a/native/core/src/execution/expressions/list_positions.rs b/native/core/src/execution/expressions/list_positions.rs new file mode 100644 index 0000000000..11643db4ce --- /dev/null +++ b/native/core/src/execution/expressions/list_positions.rs @@ -0,0 +1,135 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fmt::{Display, Formatter}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + +use arrow::array::{Array, ArrayRef, Int32Array, ListArray, RecordBatch}; +use arrow::datatypes::{DataType, Field, FieldRef, Schema}; +use datafusion::common::{exec_err, Result as DataFusionResult}; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_plan::ColumnarValue; + +/// A `PhysicalExpr` that takes a `List` input and produces a `List` where each row's +/// values are `[0, 1, ..., len - 1]`. Offsets and the null bitmap are inherited from the input, +/// so when the resulting list is unnested in parallel with the original list it produces the +/// `pos` column expected by Spark's `posexplode`. +#[derive(Debug, Clone)] +pub struct ListPositionsExpr { + child: Arc, + field: FieldRef, +} + +impl ListPositionsExpr { + pub fn new(child: Arc) -> Self { + let field = Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + )); + Self { child, field } + } +} + +impl Display for ListPositionsExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "list_positions({})", self.child) + } +} + +impl PartialEq for ListPositionsExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + } +} + +impl Eq for ListPositionsExpr {} + +impl Hash for ListPositionsExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + } +} + +impl PhysicalExpr for ListPositionsExpr { + fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + Display::fmt(self, f) + } + + fn data_type(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(self.field.data_type().clone()) + } + + fn nullable(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult { + let value = self.child.evaluate(batch)?; + let array = value.into_array(batch.num_rows())?; + + let list = match array.as_any().downcast_ref::() { + Some(list) => list, + None => { + return exec_err!( + "ListPositionsExpr expected List input, got {}", + array.data_type() + ); + } + }; + + let offsets = list.offsets(); + let total_len = *offsets.last().unwrap() as usize; + + let mut values: Vec = Vec::with_capacity(total_len); + for window in offsets.windows(2) { + let start = window[0]; + let end = window[1]; + for i in 0..(end - start) { + values.push(i); + } + } + + let element_field = Arc::new(Field::new("item", DataType::Int32, true)); + let result = ListArray::new( + element_field, + offsets.clone(), + Arc::new(Int32Array::from(values)), + list.nulls().cloned(), + ); + + Ok(ColumnarValue::Array(Arc::new(result) as ArrayRef)) + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> DataFusionResult> { + if children.len() != 1 { + return exec_err!( + "ListPositionsExpr expects exactly 1 child, got {}", + children.len() + ); + } + Ok(Arc::new(ListPositionsExpr::new(Arc::clone(&children[0])))) + } +} diff --git a/native/core/src/execution/expressions/mod.rs b/native/core/src/execution/expressions/mod.rs index c2b144b7dd..e174bd3747 100644 --- a/native/core/src/execution/expressions/mod.rs +++ b/native/core/src/execution/expressions/mod.rs @@ -20,6 +20,7 @@ pub mod arithmetic; pub mod bitwise; pub mod comparison; +pub mod list_positions; pub mod logical; pub mod nullcheck; pub mod partition; diff --git a/native/core/src/execution/expressions/strings.rs b/native/core/src/execution/expressions/strings.rs index 7219395963..4a7c44cc3e 100644 --- a/native/core/src/execution/expressions/strings.rs +++ b/native/core/src/execution/expressions/strings.rs @@ -91,7 +91,7 @@ impl ExpressionBuilder for RlikeBuilder { let left = planner.create_expr(expr.left.as_ref().unwrap(), Arc::clone(&input_schema))?; let right = planner.create_expr(expr.right.as_ref().unwrap(), input_schema)?; - match right.as_any().downcast_ref::().unwrap().value() { + match right.downcast_ref::().unwrap().value() { ScalarValue::Utf8(Some(pattern)) => Ok(Arc::new(RLike::try_new(left, pattern)?)), _ => Err(ExecutionError::GeneralError( "RLike only supports scalar patterns".to_string(), diff --git a/native/core/src/execution/expressions/subquery.rs b/native/core/src/execution/expressions/subquery.rs index 9272ede60c..fc7b8104d2 100644 --- a/native/core/src/execution/expressions/subquery.rs +++ b/native/core/src/execution/expressions/subquery.rs @@ -29,7 +29,6 @@ use jni::{ sys::{jboolean, jbyte, jint, jlong, jshort}, }; use std::{ - any::Any, fmt::{Display, Formatter}, hash::Hash, sync::Arc, @@ -63,10 +62,6 @@ impl Display for Subquery { } impl PhysicalExpr for Subquery { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index f5b04cc51d..97e3f851c5 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -48,20 +48,28 @@ use datafusion_spark::function::bitwise::bit_get::SparkBitGet; use datafusion_spark::function::bitwise::bitwise_not::SparkBitwiseNot; use datafusion_spark::function::datetime::date_add::SparkDateAdd; use datafusion_spark::function::datetime::date_sub::SparkDateSub; +use datafusion_spark::function::datetime::from_utc_timestamp::SparkFromUtcTimestamp; use datafusion_spark::function::datetime::last_day::SparkLastDay; use datafusion_spark::function::datetime::next_day::SparkNextDay; +use datafusion_spark::function::datetime::to_utc_timestamp::SparkToUtcTimestamp; use datafusion_spark::function::hash::crc32::SparkCrc32; use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; use datafusion_spark::function::map::map_from_entries::MapFromEntries; use datafusion_spark::function::map::str_to_map::SparkStrToMap; use datafusion_spark::function::math::expm1::SparkExpm1; +use datafusion_spark::function::math::factorial::SparkFactorial; use datafusion_spark::function::math::hex::SparkHex; +use datafusion_spark::function::math::rint::SparkRint; +use datafusion_spark::function::math::trigonometry::SparkCsc; +use datafusion_spark::function::math::trigonometry::SparkSec; use datafusion_spark::function::math::width_bucket::SparkWidthBucket; use datafusion_spark::function::string::char::CharFunc; use datafusion_spark::function::string::concat::SparkConcat; use datafusion_spark::function::string::luhn_check::SparkLuhnCheck; use datafusion_spark::function::string::space::SparkSpace; +use datafusion_spark::function::url::parse_url::ParseUrl as SparkParseUrl; +use datafusion_spark::function::url::try_parse_url::TryParseUrl as SparkTryParseUrl; use datafusion_spark::function::url::try_url_decode::TryUrlDecode as SparkTryUrlDecode; use datafusion_spark::function::url::url_decode::UrlDecode as SparkUrlDecode; use datafusion_spark::function::url::url_encode::UrlEncode as SparkUrlEncode; @@ -573,8 +581,10 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitGet::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateAdd::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateSub::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkFromUtcTimestamp::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkLastDay::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkNextDay::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkToUtcTimestamp::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default())); @@ -591,6 +601,12 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkUrlDecode::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkUrlEncode::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkTryUrlDecode::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkCsc::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkParseUrl::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkTryParseUrl::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkFactorial::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSec::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkRint::default())); } /// Prepares arrow arrays for output. diff --git a/native/core/src/execution/memory_pools/fair_pool.rs b/native/core/src/execution/memory_pools/fair_pool.rs index e4a7ceab54..5c4cacbefb 100644 --- a/native/core/src/execution/memory_pools/fair_pool.rs +++ b/native/core/src/execution/memory_pools/fair_pool.rs @@ -16,7 +16,7 @@ // under the License. use std::{ - fmt::{Debug, Formatter, Result as FmtResult}, + fmt::{Debug, Display, Formatter, Result as FmtResult}, sync::Arc, }; @@ -83,10 +83,20 @@ impl CometFairMemoryPool { } } +impl Display for CometFairMemoryPool { + fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { + write!(f, "CometFairMemoryPool") + } +} + unsafe impl Send for CometFairMemoryPool {} unsafe impl Sync for CometFairMemoryPool {} impl MemoryPool for CometFairMemoryPool { + fn name(&self) -> &str { + "CometFairMemoryPool" + } + fn register(&self, _: &MemoryConsumer) { let mut state = self.state.lock(); state.num = state diff --git a/native/core/src/execution/memory_pools/logging_pool.rs b/native/core/src/execution/memory_pools/logging_pool.rs index c23672d01a..b05f98c732 100644 --- a/native/core/src/execution/memory_pools/logging_pool.rs +++ b/native/core/src/execution/memory_pools/logging_pool.rs @@ -19,6 +19,7 @@ use datafusion::execution::memory_pool::{ MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation, }; use log::{info, warn}; +use std::fmt::{self, Display}; use std::sync::Arc; #[derive(Debug)] @@ -36,7 +37,17 @@ impl LoggingMemoryPool { } } +impl Display for LoggingMemoryPool { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "LoggingMemoryPool(task={})", self.task_attempt_id) + } +} + impl MemoryPool for LoggingMemoryPool { + fn name(&self) -> &str { + "LoggingMemoryPool" + } + fn register(&self, consumer: &MemoryConsumer) { info!( "[Task {}] MemoryPool[{}].register()", diff --git a/native/core/src/execution/memory_pools/unified_pool.rs b/native/core/src/execution/memory_pools/unified_pool.rs index f34418ee94..d26a57d35c 100644 --- a/native/core/src/execution/memory_pools/unified_pool.rs +++ b/native/core/src/execution/memory_pools/unified_pool.rs @@ -16,7 +16,7 @@ // under the License. use std::{ - fmt::{Debug, Formatter, Result as FmtResult}, + fmt::{Debug, Display, Formatter, Result as FmtResult}, sync::{ atomic::{AtomicUsize, Ordering::Relaxed}, Arc, @@ -93,7 +93,17 @@ impl Drop for CometUnifiedMemoryPool { unsafe impl Send for CometUnifiedMemoryPool {} unsafe impl Sync for CometUnifiedMemoryPool {} +impl Display for CometUnifiedMemoryPool { + fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { + write!(f, "CometUnifiedMemoryPool(task={})", self.task_attempt_id) + } +} + impl MemoryPool for CometUnifiedMemoryPool { + fn name(&self) -> &str { + "CometUnifiedMemoryPool" + } + fn grow(&self, reservation: &MemoryReservation, additional: usize) { self.try_grow(reservation, additional).unwrap(); } diff --git a/native/core/src/execution/merge_as_partial.rs b/native/core/src/execution/merge_as_partial.rs index 1c15ce8329..5ea26115bf 100644 --- a/native/core/src/execution/merge_as_partial.rs +++ b/native/core/src/execution/merge_as_partial.rs @@ -26,7 +26,6 @@ //! outputs state) but redirects `update_batch` calls to `merge_batch`, giving merge //! semantics with state output. -use std::any::Any; use std::fmt::Debug; use std::hash::{Hash, Hasher}; @@ -100,10 +99,6 @@ impl MergeAsPartialUDF { } impl AggregateUDFImpl for MergeAsPartialUDF { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { &self.name } diff --git a/native/core/src/execution/operators/expand.rs b/native/core/src/execution/operators/expand.rs index e06fab23ec..1617712a87 100644 --- a/native/core/src/execution/operators/expand.rs +++ b/native/core/src/execution/operators/expand.rs @@ -17,6 +17,7 @@ use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::SchemaRef; +use datafusion::common::tree_node::TreeNodeRecursion; use datafusion::common::DataFusionError; use datafusion::physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; @@ -29,7 +30,6 @@ use datafusion::{ }; use futures::{Stream, StreamExt}; use std::{ - any::Any, pin::Pin, sync::Arc, task::{Context, Poll}, @@ -91,10 +91,6 @@ impl DisplayAs for ExpandExec { } impl ExecutionPlan for ExpandExec { - fn as_any(&self) -> &dyn Any { - self - } - fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) } @@ -103,6 +99,19 @@ impl ExecutionPlan for ExpandExec { vec![&self.child] } + fn apply_expressions( + &self, + f: &mut dyn FnMut(&dyn PhysicalExpr) -> datafusion::common::Result, + ) -> datafusion::common::Result { + let mut tnr = TreeNodeRecursion::Continue; + for projection in &self.projections { + for expr in projection { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } + } + Ok(tnr) + } + fn with_new_children( self: Arc, children: Vec>, diff --git a/native/core/src/execution/operators/iceberg_scan.rs b/native/core/src/execution/operators/iceberg_scan.rs index 55bcbef349..406f5236c9 100644 --- a/native/core/src/execution/operators/iceberg_scan.rs +++ b/native/core/src/execution/operators/iceberg_scan.rs @@ -17,7 +17,6 @@ //! Native Iceberg table scan operator using iceberg-rust -use std::any::Any; use std::collections::HashMap; use std::fmt; use std::pin::Pin; @@ -26,6 +25,7 @@ use std::task::{Context, Poll}; use arrow::array::{ArrayRef, RecordBatch, RecordBatchOptions}; use arrow::datatypes::SchemaRef; +use datafusion::common::tree_node::TreeNodeRecursion; use datafusion::common::{DataFusionError, Result as DFResult}; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::expressions::Column; @@ -109,10 +109,6 @@ impl ExecutionPlan for IcebergScanExec { "IcebergScanExec" } - fn as_any(&self) -> &dyn Any { - self - } - fn schema(&self) -> SchemaRef { Arc::clone(&self.output_schema) } @@ -125,6 +121,13 @@ impl ExecutionPlan for IcebergScanExec { vec![] } + fn apply_expressions( + &self, + _f: &mut dyn FnMut(&dyn PhysicalExpr) -> DFResult, + ) -> DFResult { + Ok(TreeNodeRecursion::Continue) + } + fn with_new_children( self: Arc, _children: Vec>, diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 8ba79098d4..5303888186 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -18,7 +18,6 @@ //! Parquet writer operator for writing RecordBatches to Parquet files use std::{ - any::Any, collections::HashMap, fmt, fmt::{Debug, Formatter}, @@ -39,9 +38,10 @@ use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use async_trait::async_trait; use datafusion::{ + common::tree_node::TreeNodeRecursion, error::{DataFusionError, Result}, execution::context::TaskContext, - physical_expr::EquivalenceProperties, + physical_expr::{EquivalenceProperties, PhysicalExpr}, physical_plan::{ execution_plan::{Boundedness, EmissionType}, metrics::{ExecutionPlanMetricsSet, MetricsSet}, @@ -404,10 +404,6 @@ impl DisplayAs for ParquetWriterExec { #[async_trait] impl ExecutionPlan for ParquetWriterExec { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "ParquetWriterExec" } @@ -428,6 +424,13 @@ impl ExecutionPlan for ParquetWriterExec { vec![&self.input] } + fn apply_expressions( + &self, + _f: &mut dyn FnMut(&dyn PhysicalExpr) -> Result, + ) -> Result { + Ok(TreeNodeRecursion::Continue) + } + fn with_new_children( self: Arc, children: Vec>, diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index 90bb741b5e..f3b8f9ad1b 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -28,6 +28,7 @@ use arrow::compute::{cast_with_options, take, CastOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::ffi::FFI_ArrowArray; use arrow::ffi::FFI_ArrowSchema; +use datafusion::common::tree_node::TreeNodeRecursion; use datafusion::common::{arrow_datafusion_err, DataFusionError, Result as DataFusionResult}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{ @@ -43,7 +44,6 @@ use itertools::Itertools; use jni::objects::{Global, JObject, JValue}; use std::rc::Rc; use std::{ - any::Any, pin::Pin, sync::{Arc, Mutex}, task::{Context, Poll}, @@ -383,10 +383,6 @@ fn schema_from_data_types(data_types: &[DataType]) -> SchemaRef { } impl ExecutionPlan for ScanExec { - fn as_any(&self) -> &dyn Any { - self - } - fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) } @@ -395,6 +391,13 @@ impl ExecutionPlan for ScanExec { vec![] } + fn apply_expressions( + &self, + _f: &mut dyn FnMut(&dyn PhysicalExpr) -> DataFusionResult, + ) -> DataFusionResult { + Ok(TreeNodeRecursion::Continue) + } + fn with_new_children( self: Arc, _: Vec>, diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 92c4dc8780..c89814f2af 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -24,6 +24,7 @@ use crate::{ }; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::common::tree_node::TreeNodeRecursion; use datafusion::common::{arrow_datafusion_err, Result as DataFusionResult}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{ @@ -37,7 +38,6 @@ use datafusion::{ use futures::Stream; use jni::objects::{Global, JByteBuffer, JObject}; use std::{ - any::Any, pin::Pin, sync::{Arc, Mutex}, task::{Context, Poll}, @@ -221,10 +221,6 @@ fn schema_from_data_types(data_types: &[DataType]) -> SchemaRef { } impl ExecutionPlan for ShuffleScanExec { - fn as_any(&self) -> &dyn Any { - self - } - fn schema(&self) -> SchemaRef { Arc::clone(&self.schema) } @@ -233,6 +229,13 @@ impl ExecutionPlan for ShuffleScanExec { vec![] } + fn apply_expressions( + &self, + _f: &mut dyn FnMut(&dyn PhysicalExpr) -> DataFusionResult, + ) -> DataFusionResult { + Ok(TreeNodeRecursion::Continue) + } + fn with_new_children( self: Arc, _: Vec>, diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index b00f140026..0e33fe78ec 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -24,6 +24,7 @@ pub mod operator_registry; use crate::execution::operators::init_csv_datasource_exec; use crate::execution::operators::IcebergScanExec; use crate::execution::{ + expressions::list_positions::ListPositionsExpr, expressions::subquery::Subquery, operators::{ExecutionError, ExpandExec, ParquetWriterExec, ScanExec, ShuffleScanExec}, planner::expression_registry::ExpressionRegistry, @@ -210,9 +211,9 @@ impl PhysicalPlanner { self } - /// Attach the Spark `TaskContext` global reference captured at `createPlan` time. Cloned - /// into every `JvmScalarUdfExpr` the planner builds so the JNI bridge can install it as - /// the thread-local on the Tokio worker driving the UDF. + /// Attach a propagated Spark `TaskContext` global reference. Called by the JNI `executePlan` + /// entry with whatever was captured at `createPlan` time. The planner clones this `Option` + /// into every `JvmScalarUdfExpr` it builds. pub fn with_task_context( mut self, task_context: Option>>>, @@ -264,7 +265,6 @@ impl PhysicalPlanner { let literal = self.create_expr(partition_value, Arc::::clone(&empty_schema))?; literal - .as_any() .downcast_ref::() .ok_or_else(|| { GeneralError("Expected literal of partition value".to_string()) @@ -355,6 +355,9 @@ impl PhysicalPlanner { DataType::Map(f, s) => DataType::Map(f, s).try_into()?, DataType::List(f) => DataType::List(f).try_into()?, DataType::Null => ScalarValue::Null, + DataType::Time64(TimeUnit::Nanosecond) => { + ScalarValue::Time64Nanosecond(None) + } dt => { return Err(GeneralError(format!("{dt:?} is not supported in Comet"))) } @@ -455,11 +458,7 @@ impl PhysicalPlanner { // WideDecimalBinaryExpr already handles overflow — skip redundant check // but only if its output type matches CheckOverflow's declared type - if child - .as_any() - .downcast_ref::() - .is_some() - { + if child.downcast_ref::().is_some() { let child_type = child.data_type(&input_schema)?; if child_type == data_type { return Ok(child); @@ -468,7 +467,7 @@ impl PhysicalPlanner { // Fuse Cast(Decimal128→Decimal128) + CheckOverflow into single rescale+check // Only fuse when the Cast target type matches the CheckOverflow output type - if let Some(cast) = child.as_any().downcast_ref::() { + if let Some(cast) = child.downcast_ref::() { if let ( DataType::Decimal128(p_out, s_out), Ok(DataType::Decimal128(_p_in, s_in)), @@ -741,6 +740,13 @@ impl PhysicalPlanner { to_arrow_datatype(udf.return_type.as_ref().ok_or_else(|| { GeneralError("JvmScalarUdf missing return_type".to_string()) })?); + // Invariant: task_context is propagated for every JvmScalarUdfExpr built during + // normal execution. The TEST_EXEC_CONTEXT_ID path is the only context in which + // task_context may legitimately be None (unit tests, direct native driver runs). + debug_assert!( + self.task_context.is_some() || self.exec_context_id == TEST_EXEC_CONTEXT_ID, + "task_context must be set for non-test execution" + ); Ok(Arc::new(JvmScalarUdfExpr::new( udf.class_name.clone(), args, @@ -1308,12 +1314,10 @@ impl PhysicalPlanner { .iter() .map(|expr| { let literal = self.create_expr(expr, Arc::clone(&required_schema))?; - let df_literal = literal - .as_any() - .downcast_ref::() - .ok_or_else(|| { - GeneralError("Expected literal of default value.".to_string()) - })?; + let df_literal = + literal.downcast_ref::().ok_or_else(|| { + GeneralError("Expected literal of default value.".to_string()) + })?; Ok(df_literal.value().clone()) }) .collect(); @@ -1372,6 +1376,7 @@ impl PhysicalPlanner { common.session_timezone.as_str(), common.case_sensitive, common.return_null_struct_if_all_fields_missing, + common.allow_type_promotion, self.session_ctx(), common.encryption_enabled, common.use_field_id, @@ -1655,12 +1660,8 @@ impl PhysicalPlanner { .map(|expr| self.create_expr(expr, child.schema())) .collect::, _>>()?; - // For UnnestExec, we need to add a projection to put the columns in the right order: - // 1. First add all projection columns - // 2. Then add the array column to be exploded - // Then UnnestExec will unnest the last column - - // Use return_field() to get the proper column names from the expressions + // For posexplode, a parallel List positions column is added before the + // array column so UnnestExec can unnest both in parallel. let child_schema = child.schema(); let mut project_exprs: Vec<(Arc, String)> = projections .iter() @@ -1673,24 +1674,26 @@ impl PhysicalPlanner { }) .collect(); - // Add the array column as the last column let array_field = child_expr .return_field(&child_schema) .expect("Failed to get field from array expression"); let array_col_name = array_field.name().to_string(); + + if explode.position { + let positions_expr: Arc = + Arc::new(ListPositionsExpr::new(Arc::clone(&child_expr))); + project_exprs.push((positions_expr, "pos".to_string())); + } project_exprs.push((Arc::clone(&child_expr), array_col_name.clone())); - // Create a projection to arrange columns as needed let project_exec = Arc::new(ProjectionExec::try_new( project_exprs, Arc::clone(&child.native_plan), )?); - // Get the input schema from the projection let project_schema = project_exec.schema(); // Build the output schema for UnnestExec - // The output schema replaces the list column with its element type let mut output_fields: Vec = Vec::new(); // Add all projection columns (non-array columns) @@ -1698,9 +1701,17 @@ impl PhysicalPlanner { output_fields.push(project_schema.field(i).clone()); } - // Add the unnested array element field + let array_input_index = if explode.position { + // With outer=true, UnnestExec preserves rows whose array is empty or NULL + // and emits a NULL position for them, so pos must be nullable in that case. + output_fields.push(Field::new("pos", DataType::Int32, explode.outer)); + projections.len() + 1 + } else { + projections.len() + }; + // Extract the element type from the list/array type - let array_field = project_schema.field(projections.len()); + let array_field = project_schema.field(array_input_index); let element_type = match array_field.data_type() { DataType::List(field) => field.data_type().clone(), dt => { @@ -1711,8 +1722,6 @@ impl PhysicalPlanner { } }; - // The output column has the same name as the input array column - // but with the element type instead of the list type output_fields.push(Field::new( array_field.name(), element_type, @@ -1721,12 +1730,17 @@ impl PhysicalPlanner { let output_schema = Arc::new(Schema::new(output_fields)); - // Use UnnestExec to explode the last column (the array column) - // ListUnnest specifies which column to unnest and the depth (1 for single level) - let list_unnest = ListUnnest { - index_in_input_schema: projections.len(), // Index of the array column to unnest - depth: 1, // Unnest one level (explode single array) - }; + let mut list_unnests = Vec::with_capacity(2); + if explode.position { + list_unnests.push(ListUnnest { + index_in_input_schema: projections.len(), + depth: 1, + }); + } + list_unnests.push(ListUnnest { + index_in_input_schema: array_input_index, + depth: 1, + }); let unnest_options = UnnestOptions { preserve_nulls: explode.outer, @@ -1735,7 +1749,7 @@ impl PhysicalPlanner { let unnest_exec = Arc::new(UnnestExec::new( project_exec, - vec![list_unnest], + list_unnests, vec![], // No struct columns to unnest output_schema, unnest_options, @@ -1871,7 +1885,7 @@ impl PhysicalPlanner { hash_join.as_ref().swap_inputs(PartitionMode::Partitioned)?; let mut additional_native_plans = vec![]; - if swapped_hash_join.as_any().is::() { + if swapped_hash_join.is::() { // a projection was added to the hash join additional_native_plans.push(Arc::clone(swapped_hash_join.children()[0])); } @@ -2731,8 +2745,7 @@ impl PhysicalPlanner { &boundary_row.partition_bounds[col_idx], Arc::clone(&input_schema), )?; - let literal_expr = - expr.as_any().downcast_ref::().expect("Literal"); + let literal_expr = expr.downcast_ref::().expect("Literal"); col_values.push(literal_expr.value().clone()); } } @@ -2842,12 +2855,7 @@ impl PhysicalPlanner { // TODO this should try and find scalar let arguments = args .iter() - .map(|e| { - e.as_ref() - .as_any() - .downcast_ref::() - .map(|lit| lit.value()) - }) + .map(|e| e.downcast_ref::().map(|lit| lit.value())) .collect::>(); let args = ReturnFieldArgs { @@ -2951,7 +2959,7 @@ fn expr_to_columns( expr.apply(&mut |expr: &Arc| { Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { + if let Some(column) = expr.downcast_ref::() { if column.index() > left_field_len + right_field_len { return Err(DataFusionError::Internal(format!( "Column index {} out of range", @@ -3002,7 +3010,7 @@ impl TreeNodeRewriter for JoinFilterRewriter<'_> { 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(column) = node.downcast_ref::() { if column.index() < self.left_field_len { // left side let new_index = self diff --git a/native/core/src/execution/serde.rs b/native/core/src/execution/serde.rs index 5d60288f68..d6ec6be132 100644 --- a/native/core/src/execution/serde.rs +++ b/native/core/src/execution/serde.rs @@ -96,6 +96,7 @@ pub fn to_arrow_datatype(dt_value: &DataType) -> ArrowDataType { } DataTypeId::TimestampNtz => ArrowDataType::Timestamp(TimeUnit::Microsecond, None), DataTypeId::Date => ArrowDataType::Date32, + DataTypeId::Time => ArrowDataType::Time64(TimeUnit::Nanosecond), DataTypeId::Null => ArrowDataType::Null, DataTypeId::List => match dt_value .type_info diff --git a/native/core/src/lib.rs b/native/core/src/lib.rs index 7d0b6a5454..f4ae0b8834 100644 --- a/native/core/src/lib.rs +++ b/native/core/src/lib.rs @@ -65,8 +65,6 @@ pub mod jvm_bridge { use errors::{try_unwrap_or_throw, CometError, CometResult}; -#[macro_use] -pub mod common; pub mod execution; pub mod parquet; // this module is for non release only. Intended for debugging/profiling purposes diff --git a/native/core/src/parquet/cast_column.rs b/native/core/src/parquet/cast_column.rs index 67558b5266..1cc928d1d5 100644 --- a/native/core/src/parquet/cast_column.rs +++ b/native/core/src/parquet/cast_column.rs @@ -31,7 +31,6 @@ use datafusion::common::ScalarValue; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::{ - any::Any, fmt::{self, Display}, hash::Hash, sync::Arc, @@ -250,10 +249,6 @@ impl Display for CometCastColumnExpr { } impl PhysicalExpr for CometCastColumnExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, _input_schema: &Schema) -> DataFusionResult { Ok(self.target_field.data_type().clone()) } diff --git a/native/core/src/parquet/data_type.rs b/native/core/src/parquet/data_type.rs deleted file mode 100644 index 9a584d2359..0000000000 --- a/native/core/src/parquet/data_type.rs +++ /dev/null @@ -1,116 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use super::read::{PlainDecoding, PlainDictDecoding}; - -pub trait DataType: PlainDecoding + PlainDictDecoding + 'static {} - -macro_rules! make_type { - ($name:ident) => { - pub struct $name {} - impl DataType for $name {} - }; -} - -make_type!(BoolType); -make_type!(Int8Type); -make_type!(UInt8Type); -make_type!(Int16Type); -make_type!(Int16ToDoubleType); -make_type!(UInt16Type); -make_type!(Int32Type); -make_type!(Int32To64Type); -make_type!(Int32ToDecimal64Type); -make_type!(Int32ToDoubleType); -make_type!(UInt32Type); -make_type!(Int64Type); -make_type!(Int64ToDecimal64Type); -make_type!(UInt64Type); -make_type!(FloatType); -make_type!(DoubleType); -make_type!(FloatToDoubleType); -make_type!(ByteArrayType); -make_type!(StringType); -make_type!(Int32DecimalType); -make_type!(Int64DecimalType); -make_type!(FLBADecimalType); -make_type!(FLBADecimal32Type); -make_type!(FLBADecimal64Type); -make_type!(FLBAType); -make_type!(Int32DateType); -make_type!(Int32TimestampMicrosType); -make_type!(Int64TimestampMillisType); -make_type!(Int64TimestampMicrosType); -make_type!(Int96TimestampMicrosType); - -pub trait AsBytes { - /// Returns the slice of bytes for an instance of this data type. - fn as_bytes(&self) -> &[u8]; -} - -impl AsBytes for Vec { - fn as_bytes(&self) -> &[u8] { - self.as_slice() - } -} - -impl AsBytes for &str { - fn as_bytes(&self) -> &[u8] { - (self as &str).as_bytes() - } -} - -impl AsBytes for [u8] { - fn as_bytes(&self) -> &[u8] { - self - } -} - -impl AsBytes for str { - fn as_bytes(&self) -> &[u8] { - (self as &str).as_bytes() - } -} - -macro_rules! make_as_bytes { - ($source_ty:ident) => { - impl AsBytes for $source_ty { - #[allow(clippy::size_of_in_element_count)] - fn as_bytes(&self) -> &[u8] { - unsafe { - ::std::slice::from_raw_parts( - self as *const $source_ty as *const u8, - ::std::mem::size_of::<$source_ty>(), - ) - } - } - } - }; -} - -make_as_bytes!(bool); -make_as_bytes!(i8); -make_as_bytes!(u8); -make_as_bytes!(i16); -make_as_bytes!(u16); -make_as_bytes!(i32); -make_as_bytes!(u32); -make_as_bytes!(i64); -make_as_bytes!(u64); -make_as_bytes!(f32); -make_as_bytes!(f64); -make_as_bytes!(i128); diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index 5de14aa610..806800cd59 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -15,18 +15,13 @@ // specific language governing permissions and limitations // under the License. -pub mod data_type; pub mod encryption_support; -pub mod mutable_vector; -pub use mutable_vector::*; -#[macro_use] -pub mod util; pub mod parquet_exec; pub mod parquet_read_cached_factory; pub mod parquet_support; -pub mod read; pub mod schema_adapter; +pub mod util; mod cast_column; mod objectstore; @@ -44,7 +39,6 @@ use jni::{ Env, EnvUnowned, }; -use self::util::jni::TypePromotionInfo; use crate::execution::jni_api::get_runtime; use crate::execution::metrics::utils::update_comet_metric; use crate::execution::operators::ExecutionError; @@ -53,281 +47,19 @@ use crate::execution::serde; use crate::execution::spark_plan::SparkPlan; use crate::execution::utils::SparkArrowConvert; use crate::jvm_bridge::JVMClasses; -use crate::parquet::data_type::AsBytes; use crate::parquet::encryption_support::{CometEncryptionFactory, ENCRYPTION_FACTORY_ID}; use crate::parquet::parquet_exec::init_datasource_exec; use crate::parquet::parquet_support::prepare_object_store_with_configs; use arrow::array::{Array, RecordBatch}; -use arrow::buffer::MutableBuffer; use datafusion::datasource::listing::PartitionedFile; use datafusion::execution::SendableRecordBatchStream; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{SessionConfig, SessionContext}; use futures::{poll, StreamExt}; -use jni::objects::{JByteArray, JLongArray, JMap, JObject, JObjectArray, JString, ReleaseMode}; -use jni::sys::{jintArray, JNI_FALSE}; +use jni::objects::{JByteArray, JLongArray, JMap, JObject, JString, ReleaseMode}; +use jni::sys::JNI_FALSE; use object_store::path::Path; -use read::ColumnReader; -use util::jni::{convert_column_descriptor, convert_encoding, deserialize_schema}; - -/// Parquet read context maintained across multiple JNI calls. -struct Context { - pub column_reader: ColumnReader, -} - -#[no_mangle] -pub extern "system" fn Java_org_apache_comet_parquet_Native_initColumnReader( - e: EnvUnowned, - _jclass: JClass, - primitive_type: jint, - logical_type: jint, - read_primitive_type: jint, - jni_path: JObjectArray, - max_dl: jint, - max_rl: jint, - bit_width: jint, - read_bit_width: jint, - is_signed: jboolean, - type_length: jint, - precision: jint, - read_precision: jint, - scale: jint, - read_scale: jint, - time_unit: jint, - is_adjusted_utc: jboolean, - batch_size: jint, - use_decimal_128: jboolean, - use_legacy_date_timestamp: jboolean, -) -> jlong { - try_unwrap_or_throw(&e, |env| { - let desc = convert_column_descriptor( - env, - primitive_type, - logical_type, - max_dl, - max_rl, - bit_width, - is_signed, - type_length, - precision, - scale, - time_unit, - is_adjusted_utc, - jni_path, - )?; - let promotion_info = TypePromotionInfo::new_from_jni( - read_primitive_type, - read_precision, - read_scale, - read_bit_width, - ); - let ctx = Context { - column_reader: ColumnReader::get( - desc, - promotion_info, - batch_size as usize, - use_decimal_128, - use_legacy_date_timestamp, - ), - }; - let res = Box::new(ctx); - Ok(Box::into_raw(res) as i64) - }) -} - -/// # Safety -/// This function is inheritly unsafe since it deals with raw pointers passed from JNI. -#[no_mangle] -pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setDictionaryPage( - e: EnvUnowned, - _jclass: JClass, - handle: jlong, - page_value_count: jint, - page_data: JByteArray, - encoding: jint, -) { - try_unwrap_or_throw(&e, |env| { - let reader = get_reader(handle)?; - - // convert value encoding ordinal to the native encoding definition - let encoding = convert_encoding(encoding); - - // copy the input on-heap buffer to native - let page_len = page_data.len(env)?; - let mut buffer = MutableBuffer::from_len_zeroed(page_len); - page_data.get_region(env, 0, from_u8_slice(buffer.as_slice_mut()))?; - - reader.set_dictionary_page(page_value_count as usize, buffer.into(), encoding); - Ok(()) - }) -} - -/// # Safety -/// This function is inheritly unsafe since it deals with raw pointers passed from JNI. -#[no_mangle] -pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setPageV1( - e: EnvUnowned, - _jclass: JClass, - handle: jlong, - page_value_count: jint, - page_data: JByteArray, - value_encoding: jint, -) { - try_unwrap_or_throw(&e, |env| { - let reader = get_reader(handle)?; - - // convert value encoding ordinal to the native encoding definition - let encoding = convert_encoding(value_encoding); - - // copy the input on-heap buffer to native - let page_len = page_data.len(env)?; - let mut buffer = MutableBuffer::from_len_zeroed(page_len); - page_data.get_region(env, 0, from_u8_slice(buffer.as_slice_mut()))?; - - reader.set_page_v1(page_value_count as usize, buffer.into(), encoding); - Ok(()) - }) -} - -/// # Safety -/// This function is inheritly unsafe since it deals with raw pointers passed from JNI. -#[no_mangle] -pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_setPageV2( - e: EnvUnowned, - _jclass: JClass, - handle: jlong, - page_value_count: jint, - def_level_data: JByteArray, - rep_level_data: JByteArray, - value_data: JByteArray, - value_encoding: jint, -) { - try_unwrap_or_throw(&e, |env| { - let reader = get_reader(handle)?; - - // convert value encoding ordinal to the native encoding definition - let encoding = convert_encoding(value_encoding); - - // copy the input on-heap buffer to native - let dl_len = def_level_data.len(env)?; - let mut dl_buffer = MutableBuffer::from_len_zeroed(dl_len); - def_level_data.get_region(env, 0, from_u8_slice(dl_buffer.as_slice_mut()))?; - - let rl_len = rep_level_data.len(env)?; - let mut rl_buffer = MutableBuffer::from_len_zeroed(rl_len); - rep_level_data.get_region(env, 0, from_u8_slice(rl_buffer.as_slice_mut()))?; - - let v_len = value_data.len(env)?; - let mut v_buffer = MutableBuffer::from_len_zeroed(v_len); - value_data.get_region(env, 0, from_u8_slice(v_buffer.as_slice_mut()))?; - - reader.set_page_v2( - page_value_count as usize, - dl_buffer.into(), - rl_buffer.into(), - v_buffer.into(), - encoding, - ); - Ok(()) - }) -} - -#[no_mangle] -pub extern "system" fn Java_org_apache_comet_parquet_Native_resetBatch( - env: EnvUnowned, - _jclass: JClass, - handle: jlong, -) { - try_unwrap_or_throw(&env, |_| { - let reader = get_reader(handle)?; - reader.reset_batch(); - Ok(()) - }) -} - -#[no_mangle] -pub extern "system" fn Java_org_apache_comet_parquet_Native_readBatch( - e: EnvUnowned, - _jclass: JClass, - handle: jlong, - batch_size: jint, - null_pad_size: jint, -) -> jintArray { - try_unwrap_or_throw(&e, |env| { - let reader = get_reader(handle)?; - let (num_values, num_nulls) = - reader.read_batch(batch_size as usize, null_pad_size as usize); - let res = env.new_int_array(2)?; - let buf: [i32; 2] = [num_values as i32, num_nulls as i32]; - res.set_region(env, 0, &buf)?; - Ok(res.into_raw()) - }) -} - -#[no_mangle] -pub extern "system" fn Java_org_apache_comet_parquet_Native_skipBatch( - env: EnvUnowned, - _jclass: JClass, - handle: jlong, - batch_size: jint, - discard: jboolean, -) -> jint { - try_unwrap_or_throw(&env, |_| { - let reader = get_reader(handle)?; - Ok(reader.skip_batch(batch_size as usize, discard) as jint) - }) -} - -#[no_mangle] -pub extern "system" fn Java_org_apache_comet_parquet_Native_currentBatch( - e: EnvUnowned, - _jclass: JClass, - handle: jlong, - array_addr: jlong, - schema_addr: jlong, -) { - try_unwrap_or_throw(&e, |_env| { - let ctx = get_context(handle)?; - let reader = &mut ctx.column_reader; - let data = reader.current_batch()?; - data.move_to_spark(array_addr, schema_addr) - .map_err(|e| e.into()) - }) -} - -#[inline] -fn get_context<'a>(handle: jlong) -> Result<&'a mut Context, CometError> { - unsafe { - (handle as *mut Context) - .as_mut() - .ok_or_else(|| CometError::NullPointer("null context handle".to_string())) - } -} - -#[inline] -fn get_reader<'a>(handle: jlong) -> Result<&'a mut ColumnReader, CometError> { - Ok(&mut get_context(handle)?.column_reader) -} - -#[no_mangle] -pub extern "system" fn Java_org_apache_comet_parquet_Native_closeColumnReader( - env: EnvUnowned, - _jclass: JClass, - handle: jlong, -) { - try_unwrap_or_throw(&env, |_| { - unsafe { - let ctx = get_context(handle)?; - let _ = Box::from_raw(ctx); - }; - Ok(()) - }) -} - -fn from_u8_slice(src: &mut [u8]) -> &mut [i8] { - let raw_ptr = src.as_mut_ptr() as *mut i8; - unsafe { std::slice::from_raw_parts_mut(raw_ptr, src.len()) } -} +use util::jni::deserialize_schema; // TODO: (ARROW NATIVE) remove this if not needed. enum ParquetReaderState { @@ -397,31 +129,6 @@ pub fn get_object_store_options( Ok(collected_map) } -/// # Safety -/// This function is inherently unsafe since it deals with raw pointers passed from JNI. -#[no_mangle] -pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_validateObjectStoreConfig( - e: EnvUnowned, - _jclass: JClass, - file_path: JString, - object_store_options: JObject, -) { - try_unwrap_or_throw(&e, |env| { - let session_config = SessionConfig::new(); - let planner = - PhysicalPlanner::new(Arc::new(SessionContext::new_with_config(session_config)), 0); - let session_ctx = planner.session_ctx(); - let path: String = file_path.try_to_string(env).unwrap(); - let object_store_config = get_object_store_options(env, object_store_options)?; - let (_, _) = prepare_object_store_with_configs( - session_ctx.runtime_env(), - path.clone(), - &object_store_config, - )?; - Ok(()) - }) -} - /// # Safety /// This function is inherently unsafe since it deals with raw pointers passed from JNI. #[no_mangle] @@ -460,10 +167,10 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBat )?; let required_schema_buffer = env.convert_byte_array(&required_schema)?; - let required_schema = Arc::new(deserialize_schema(required_schema_buffer.as_bytes())?); + let required_schema = Arc::new(deserialize_schema(&required_schema_buffer)?); let data_schema_buffer = env.convert_byte_array(&data_schema)?; - let data_schema = Arc::new(deserialize_schema(data_schema_buffer.as_bytes())?); + let data_schema = Arc::new(deserialize_schema(&data_schema_buffer)?); let data_filters = if !filter.is_null() { let filter_buffer = env.convert_byte_array(&filter)?; @@ -513,6 +220,7 @@ pub unsafe extern "system" fn Java_org_apache_comet_parquet_Native_initRecordBat session_timezone.as_str(), case_sensitive != JNI_FALSE, return_null_struct_if_all_fields_missing != JNI_FALSE, + true, // allow_type_promotion: JVM side already validated via TypeUtil.checkParquetType session_ctx, encryption_enabled, // The iceberg-compat path resolves IDs in the JVM via NativeBatchReader, diff --git a/native/core/src/parquet/mutable_vector.rs b/native/core/src/parquet/mutable_vector.rs deleted file mode 100644 index 5fe6aa1f3b..0000000000 --- a/native/core/src/parquet/mutable_vector.rs +++ /dev/null @@ -1,243 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::{array::ArrayData, datatypes::DataType as ArrowDataType}; - -use crate::common::{bit, CometBuffer}; -use crate::execution::operators::ExecutionError; - -const DEFAULT_ARRAY_LEN: usize = 4; - -/// A mutable vector that can be re-used across batches, for Parquet read. -/// -/// Note this class is similar to [`MutableVector`](crate::common::MutableVector). However, the -/// latter has functionalities such as `ValueGetter`, `ValueSetter`. In addition, it represents -/// String and Binary data using [`StringView`](crate::data_type::StringView), while this struct -/// uses Arrow format to represent them. -/// -/// TODO: unify the two structs in future -#[derive(Debug)] -pub struct ParquetMutableVector { - /// The Arrow type for the elements of this vector. - pub(crate) arrow_type: ArrowDataType, - - /// The number of total elements in this vector. - pub(crate) num_values: usize, - - /// The number of null elements in this vector, must <= `num_values`. - pub(crate) num_nulls: usize, - - /// The validity buffer of this Arrow vector. A bit set at position `i` indicates the `i`th - /// element is not null. Otherwise, an unset bit at position `i` indicates the `i`th element is - /// null. - pub(crate) validity_buffer: CometBuffer, - - /// The value buffer of this Arrow vector. This could store either offsets if the vector - /// is of list or struct type, or actual values themselves otherwise. - pub(crate) value_buffer: CometBuffer, - - /// Child vectors for non-primitive types (e.g., list, struct). - pub(crate) children: Vec, - - /// Dictionary (i.e., values) associated with this vector. Only set if using dictionary - /// encoding. - pub(crate) dictionary: Option>, -} - -impl ParquetMutableVector { - pub fn new(capacity: usize, arrow_type: &ArrowDataType) -> Self { - let bit_width = Self::bit_width(arrow_type); - Self::new_with_bit_width(capacity, arrow_type.clone(), bit_width) - } - - pub fn new_with_bit_width( - capacity: usize, - arrow_type: ArrowDataType, - bit_width: usize, - ) -> Self { - let validity_len = capacity.div_ceil(8); - let validity_buffer = CometBuffer::new(validity_len); - - let mut value_capacity = capacity; - if Self::is_binary_type(&arrow_type) { - // Arrow offset array needs to have one extra slot - value_capacity += 1; - } - // Make sure the capacity is positive - let len = usize::div_ceil(value_capacity * bit_width, 8); - let mut value_buffer = CometBuffer::new(len); - - let mut children = Vec::new(); - - match arrow_type { - ArrowDataType::Binary | ArrowDataType::Utf8 => { - children.push(ParquetMutableVector::new_with_bit_width( - capacity, - ArrowDataType::Int8, - DEFAULT_ARRAY_LEN * 8, - )); - } - _ => {} - } - - if Self::is_binary_type(&arrow_type) { - // Setup the first offset which is always 0. - let zero: u32 = 0; - bit::memcpy_value(&zero, 4, &mut value_buffer); - } - - Self { - arrow_type, - num_values: 0, - num_nulls: 0, - validity_buffer, - value_buffer, - children, - dictionary: None, - } - } - - /// Whether the given value at `idx` of this vector is null. - #[inline] - pub fn is_null(&self, idx: usize) -> bool { - unsafe { !bit::get_bit_raw(self.validity_buffer.as_ptr(), idx) } - } - - /// Resets this vector to the initial state. - #[inline] - pub fn reset(&mut self) { - self.num_values = 0; - self.num_nulls = 0; - self.validity_buffer.reset(); - if Self::is_binary_type(&self.arrow_type) { - // Reset the first offset to 0 - let zero: u32 = 0; - bit::memcpy_value(&zero, 4, &mut self.value_buffer); - // Also reset the child value vector - let child = &mut self.children[0]; - child.reset(); - } else if Self::should_reset_value_buffer(&self.arrow_type) { - self.value_buffer.reset(); - } - } - - /// Appends a new null value to the end of this vector. - #[inline] - pub fn put_null(&mut self) { - self.put_nulls(1) - } - - /// Appends `n` null values to the end of this vector. - #[inline] - pub fn put_nulls(&mut self, n: usize) { - // We need to update offset buffer for binary. - if Self::is_binary_type(&self.arrow_type) { - let mut offset = self.num_values * 4; - let prev_offset_value = bit::read_u32(&self.value_buffer[offset..]); - offset += 4; - (0..n).for_each(|_| { - bit::memcpy_value(&prev_offset_value, 4, &mut self.value_buffer[offset..]); - offset += 4; - }); - } - - self.num_nulls += n; - self.num_values += n; - } - - /// Returns the number of total values (including both null and non-null) of this vector. - #[inline] - pub fn num_values(&self) -> usize { - self.num_values - } - - /// Returns the number of null values of this vector. - #[inline] - pub fn num_nulls(&self) -> usize { - self.num_nulls - } - - /// Sets the dictionary of this to be `dict`. - pub fn set_dictionary(&mut self, dict: ParquetMutableVector) { - self.dictionary = Some(Box::new(dict)) - } - - /// Clones this into an Arrow [`ArrayData`](arrow::array::ArrayData). Note that the caller of - /// this method MUST make sure the returned `ArrayData` won't live longer than this vector - /// itself. Otherwise, dangling pointer may happen. - /// - /// # Safety - /// - /// This method is highly unsafe since it calls `CometBuffer::to_arrow` which leaks raw - /// pointer to the memory region that are tracked by `CometBuffer`. Please see comments on - /// `to_arrow` buffer to understand the motivation. - pub fn get_array_data(&mut self) -> Result { - unsafe { - let data_type = if let Some(d) = &self.dictionary { - ArrowDataType::Dictionary( - Box::new(ArrowDataType::Int32), - Box::new(d.arrow_type.clone()), - ) - } else { - self.arrow_type.clone() - }; - let mut builder = ArrayData::builder(data_type) - .len(self.num_values) - .add_buffer(self.value_buffer.to_arrow()?) - .null_bit_buffer(Some(self.validity_buffer.to_arrow()?)) - .null_count(self.num_nulls); - - if Self::is_binary_type(&self.arrow_type) && self.dictionary.is_none() { - let child = &mut self.children[0]; - builder = builder.add_buffer(child.value_buffer.to_arrow()?); - } - - if let Some(d) = &mut self.dictionary { - builder = builder.add_child_data(d.get_array_data()?); - } - Ok(builder.build_unchecked()) - } - } - - /// Returns the number of bits it takes to store one element of `arrow_type` in the value buffer - /// of this vector. - pub fn bit_width(arrow_type: &ArrowDataType) -> usize { - match arrow_type { - ArrowDataType::Boolean => 1, - ArrowDataType::Int8 => 8, - ArrowDataType::Int16 => 16, - ArrowDataType::Int32 | ArrowDataType::Float32 | ArrowDataType::Date32 => 32, - ArrowDataType::Int64 | ArrowDataType::Float64 | ArrowDataType::Timestamp(_, _) => 64, - ArrowDataType::FixedSizeBinary(type_length) => *type_length as usize * 8, - ArrowDataType::Decimal128(..) => 128, // Arrow stores decimal with 16 bytes - ArrowDataType::Binary | ArrowDataType::Utf8 => 32, // Only count offset size - dt => panic!("Unsupported Arrow data type: {dt:?}"), - } - } - - #[inline] - fn is_binary_type(dt: &ArrowDataType) -> bool { - matches!(dt, ArrowDataType::Binary | ArrowDataType::Utf8) - } - - #[inline] - fn should_reset_value_buffer(dt: &ArrowDataType) -> bool { - // - Boolean type expects have a zeroed value buffer - // - Decimal may pad buffer with 0xff so we need to clear them before a new batch - matches!(dt, ArrowDataType::Boolean | ArrowDataType::Decimal128(_, _)) - } -} diff --git a/native/core/src/parquet/parquet_exec.rs b/native/core/src/parquet/parquet_exec.rs index 38a0755658..5379c02058 100644 --- a/native/core/src/parquet/parquet_exec.rs +++ b/native/core/src/parquet/parquet_exec.rs @@ -70,6 +70,7 @@ pub(crate) fn init_datasource_exec( session_timezone: &str, case_sensitive: bool, return_null_struct_if_all_fields_missing: bool, + allow_type_promotion: bool, session_ctx: &Arc, encryption_enabled: bool, use_field_id: bool, @@ -79,6 +80,7 @@ pub(crate) fn init_datasource_exec( session_timezone, case_sensitive, return_null_struct_if_all_fields_missing, + allow_type_promotion, &object_store_url, encryption_enabled, ); @@ -197,6 +199,7 @@ fn get_options( session_timezone: &str, case_sensitive: bool, return_null_struct_if_all_fields_missing: bool, + allow_type_promotion: bool, object_store_url: &ObjectStoreUrl, encryption_enabled: bool, ) -> (TableParquetOptions, SparkParquetOptions) { @@ -210,6 +213,7 @@ fn get_options( spark_parquet_options.case_sensitive = case_sensitive; spark_parquet_options.return_null_struct_if_all_fields_missing = return_null_struct_if_all_fields_missing; + spark_parquet_options.allow_type_promotion = allow_type_promotion; if encryption_enabled { table_parquet_options.crypto.configure_factory( diff --git a/native/core/src/parquet/parquet_support.rs b/native/core/src/parquet/parquet_support.rs index 4a48aaca28..0e0e7c2a6e 100644 --- a/native/core/src/parquet/parquet_support.rs +++ b/native/core/src/parquet/parquet_support.rs @@ -93,6 +93,9 @@ pub struct SparkParquetOptions { /// requested schema does carry ids raises a runtime error rather than silently /// producing nulls (mirrors `spark.sql.parquet.fieldId.read.ignoreMissing`). pub ignore_missing_field_id: bool, + /// Whether type promotion (schema evolution) is allowed, e.g. INT32 -> INT64, + /// FLOAT -> DOUBLE. Mirrors spark.comet.schemaEvolution.enabled. + pub allow_type_promotion: bool, } impl SparkParquetOptions { @@ -108,6 +111,7 @@ impl SparkParquetOptions { return_null_struct_if_all_fields_missing: true, use_field_id: false, ignore_missing_field_id: false, + allow_type_promotion: false, } } @@ -123,6 +127,7 @@ impl SparkParquetOptions { return_null_struct_if_all_fields_missing: true, use_field_id: false, ignore_missing_field_id: false, + allow_type_promotion: false, } } } diff --git a/native/core/src/parquet/read/column.rs b/native/core/src/parquet/read/column.rs deleted file mode 100644 index cdbb76bbf0..0000000000 --- a/native/core/src/parquet/read/column.rs +++ /dev/null @@ -1,839 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{marker::PhantomData, sync::Arc}; - -use arrow::{ - array::ArrayData, - buffer::Buffer, - datatypes::{DataType as ArrowDataType, TimeUnit}, -}; - -use parquet::{ - basic::{Encoding, LogicalType, TimeUnit as ParquetTimeUnit, Type as PhysicalType}, - schema::types::{ColumnDescPtr, ColumnDescriptor}, -}; - -use crate::parquet::{data_type::*, util::jni::TypePromotionInfo, ParquetMutableVector}; - -use super::{ - levels::LevelDecoder, - values::{get_decoder, Decoder}, - ReadOptions, -}; - -use crate::common::bit::log2; -use crate::execution::operators::ExecutionError; - -/// Maximum number of decimal digits an i32 can represent -const DECIMAL_MAX_INT_DIGITS: i32 = 9; - -/// Maximum number of decimal digits an i64 can represent -const DECIMAL_MAX_LONG_DIGITS: i32 = 18; - -pub enum ColumnReader { - BoolColumnReader(TypedColumnReader), - Int8ColumnReader(TypedColumnReader), - UInt8ColumnReader(TypedColumnReader), - Int16ColumnReader(TypedColumnReader), - Int16ToDoubleColumnReader(TypedColumnReader), - UInt16ColumnReader(TypedColumnReader), - Int32ColumnReader(TypedColumnReader), - Int32To64ColumnReader(TypedColumnReader), - Int32ToDecimal64ColumnReader(TypedColumnReader), - Int32ToDoubleColumnReader(TypedColumnReader), - UInt32ColumnReader(TypedColumnReader), - Int32DecimalColumnReader(TypedColumnReader), - Int32DateColumnReader(TypedColumnReader), - Int32TimestampMicrosColumnReader(TypedColumnReader), - Int64ColumnReader(TypedColumnReader), - Int64ToDecimal64ColumnReader(TypedColumnReader), - UInt64DecimalColumnReader(TypedColumnReader), - Int64DecimalColumnReader(TypedColumnReader), - Int64TimestampMillisColumnReader(TypedColumnReader), - Int64TimestampMicrosColumnReader(TypedColumnReader), - Int64TimestampNanosColumnReader(TypedColumnReader), - Int96ColumnReader(TypedColumnReader), - FloatColumnReader(TypedColumnReader), - FloatToDoubleColumnReader(TypedColumnReader), - DoubleColumnReader(TypedColumnReader), - ByteArrayColumnReader(TypedColumnReader), - StringColumnReader(TypedColumnReader), - FLBADecimalColumnReader(TypedColumnReader), - FLBADecimal32ColumnReader(TypedColumnReader), - FLBADecimal64ColumnReader(TypedColumnReader), - FLBAColumnReader(TypedColumnReader), -} - -impl ColumnReader { - /// Creates a new column reader according to the input `desc`. - /// - /// - `desc`: The actual descriptor for the underlying Parquet files - /// - `promotion_info`: Extra information about type promotion. This is passed in to support - /// schema evolution, e.g., int -> long, where Parquet type is int but Spark type is long. - /// - `use_decimal_128`: Whether to read small precision decimals as `i128` instead of as `i32` - /// or `i64` as Spark does - /// - `use_legacy_date_timestamp_or_ntz`: Whether to read dates/timestamps that were written - /// using the legacy Julian/Gregorian hybrid calendar as it is. If false, exceptions will be - /// thrown. If the spark type is TimestampNTZ, this should be true. - pub fn get( - desc: ColumnDescriptor, - promotion_info: TypePromotionInfo, - capacity: usize, - use_decimal_128: bool, - use_legacy_date_timestamp_or_ntz: bool, - ) -> Self { - let read_options = ReadOptions { - use_legacy_date_timestamp_or_ntz, - }; - macro_rules! typed_reader { - ($reader_ty:ident, $arrow_ty:ident) => { - Self::$reader_ty(TypedColumnReader::new( - desc, - capacity, - ArrowDataType::$arrow_ty, - read_options, - )) - }; - ($reader_ty:ident, $arrow_ty:expr) => { - Self::$reader_ty(TypedColumnReader::new( - desc, - capacity, - $arrow_ty, - read_options, - )) - }; - } - - match desc.physical_type() { - PhysicalType::BOOLEAN => typed_reader!(BoolColumnReader, Boolean), - PhysicalType::INT32 => { - if let Some(ref logical_type) = desc.logical_type_ref() { - match logical_type { - lt @ LogicalType::Integer { - bit_width, - is_signed, - } => match (bit_width, is_signed) { - (8, true) => match promotion_info.physical_type { - PhysicalType::FIXED_LEN_BYTE_ARRAY => { - if promotion_info.precision <= DECIMAL_MAX_INT_DIGITS - && promotion_info.scale < 1 - { - typed_reader!(Int32ColumnReader, Int32) - } else if promotion_info.precision <= DECIMAL_MAX_LONG_DIGITS { - typed_reader!( - Int32ToDecimal64ColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } else { - typed_reader!( - Int32DecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - // promote byte to short - PhysicalType::INT32 if promotion_info.bit_width == 16 => { - typed_reader!(Int16ColumnReader, Int16) - } - // promote byte to int - PhysicalType::INT32 if promotion_info.bit_width == 32 => { - typed_reader!(Int32ColumnReader, Int32) - } - // promote byte to long - PhysicalType::INT64 => typed_reader!(Int32To64ColumnReader, Int64), - _ => typed_reader!(Int8ColumnReader, Int8), - }, - (8, false) => typed_reader!(UInt8ColumnReader, Int16), - (16, true) => match promotion_info.physical_type { - PhysicalType::DOUBLE => { - typed_reader!(Int16ToDoubleColumnReader, Float64) - } - // promote short to long - PhysicalType::INT64 => { - typed_reader!(Int32To64ColumnReader, Int64) - } - PhysicalType::INT32 if promotion_info.bit_width == 32 => { - typed_reader!(Int32ColumnReader, Int32) - } - PhysicalType::FIXED_LEN_BYTE_ARRAY => { - if promotion_info.precision <= DECIMAL_MAX_INT_DIGITS - && promotion_info.scale < 1 - { - typed_reader!(Int32ColumnReader, Int32) - } else if promotion_info.precision <= DECIMAL_MAX_LONG_DIGITS { - typed_reader!( - Int32ToDecimal64ColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } else { - typed_reader!( - Int32DecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - _ => typed_reader!(Int16ColumnReader, Int16), - }, - (16, false) => typed_reader!(UInt16ColumnReader, Int32), - (32, true) => match promotion_info.physical_type { - PhysicalType::INT32 if promotion_info.bit_width == 16 => { - typed_reader!(Int16ColumnReader, Int16) - } - _ => typed_reader!(Int32ColumnReader, Int32), - }, - (32, false) => typed_reader!(UInt32ColumnReader, Int64), - _ => unimplemented!("Unsupported INT32 annotation: {:?}", lt), - }, - LogicalType::Decimal { - scale: _, - precision: _, - } => { - if use_decimal_128 || promotion_info.precision > DECIMAL_MAX_LONG_DIGITS - { - typed_reader!( - Int32DecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } else { - typed_reader!( - Int32ToDecimal64ColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - LogicalType::Date => match promotion_info.physical_type { - PhysicalType::INT64 => typed_reader!( - Int32TimestampMicrosColumnReader, - ArrowDataType::Timestamp(TimeUnit::Microsecond, None) - ), - _ => typed_reader!(Int32DateColumnReader, Date32), - }, - lt => unimplemented!("Unsupported logical type for INT32: {:?}", lt), - } - } else { - // We support type promotion from int to long - match promotion_info.physical_type { - PhysicalType::INT32 if promotion_info.bit_width == 16 => { - typed_reader!(Int16ColumnReader, Int16) - } - PhysicalType::INT32 => typed_reader!(Int32ColumnReader, Int32), - PhysicalType::INT64 => typed_reader!(Int32To64ColumnReader, Int64), - PhysicalType::DOUBLE => typed_reader!(Int32ToDoubleColumnReader, Float64), - PhysicalType::FIXED_LEN_BYTE_ARRAY => { - if promotion_info.precision <= DECIMAL_MAX_INT_DIGITS - && promotion_info.scale < 1 - { - typed_reader!(Int32ColumnReader, Int32) - } else if promotion_info.precision <= DECIMAL_MAX_LONG_DIGITS { - typed_reader!( - Int32ToDecimal64ColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } else { - typed_reader!( - Int32DecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - t => unimplemented!("Unsupported read physical type for INT32: {}", t), - } - } - } - PhysicalType::INT64 => { - if let Some(ref logical_type) = desc.logical_type_ref() { - match logical_type { - lt @ LogicalType::Integer { - bit_width, - is_signed, - } => match (bit_width, is_signed) { - (64, true) => typed_reader!(Int64ColumnReader, Int64), - (64, false) => typed_reader!( - UInt64DecimalColumnReader, - ArrowDataType::Decimal128(20u8, 0i8) - ), - _ => panic!("Unsupported INT64 annotation: {lt:?}"), - }, - LogicalType::Decimal { - scale: _, - precision: _, - } => { - if use_decimal_128 || promotion_info.precision > DECIMAL_MAX_LONG_DIGITS - { - typed_reader!( - Int64DecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } else { - typed_reader!( - Int64ToDecimal64ColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - LogicalType::Timestamp { - is_adjusted_to_u_t_c, - unit, - } => { - // To be consistent with Spark, we always store as micro-second and - // convert milli-second to it. - let time_unit = TimeUnit::Microsecond; - let time_zone = if *is_adjusted_to_u_t_c { - Some("UTC".to_string().into()) - } else { - None - }; - match unit { - ParquetTimeUnit::MILLIS => { - typed_reader!( - Int64TimestampMillisColumnReader, - ArrowDataType::Timestamp(time_unit, time_zone) - ) - } - ParquetTimeUnit::MICROS => { - typed_reader!( - Int64TimestampMicrosColumnReader, - ArrowDataType::Timestamp(time_unit, time_zone) - ) - } - ParquetTimeUnit::NANOS => { - typed_reader!( - Int64TimestampNanosColumnReader, - ArrowDataType::Int64 - ) - } - } - } - lt => panic!("Unsupported logical type for INT64: {lt:?}"), - } - } else { - match promotion_info.physical_type { - PhysicalType::FIXED_LEN_BYTE_ARRAY => { - if promotion_info.precision <= DECIMAL_MAX_LONG_DIGITS - && promotion_info.scale < 1 - { - typed_reader!(Int64ColumnReader, Int64) - } else { - typed_reader!( - Int64DecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - // By default it is INT(64, true) - _ => typed_reader!(Int64ColumnReader, Int64), - } - } - } - PhysicalType::INT96 => { - typed_reader!( - Int96ColumnReader, - ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".to_string().into())) - ) - } - PhysicalType::FLOAT => match promotion_info.physical_type { - // We support type promotion from float to double - PhysicalType::FLOAT => typed_reader!(FloatColumnReader, Float32), - PhysicalType::DOUBLE => typed_reader!(FloatToDoubleColumnReader, Float64), - t => panic!("Unsupported read physical type: {t} for FLOAT"), - }, - - PhysicalType::DOUBLE => typed_reader!(DoubleColumnReader, Float64), - PhysicalType::BYTE_ARRAY => { - if let Some(logical_type) = desc.logical_type_ref() { - match logical_type { - LogicalType::String => typed_reader!(StringColumnReader, Utf8), - // https://github.com/apache/parquet-format/blob/master/LogicalTypes.md - // "enum type should interpret ENUM annotated field as a UTF-8" - LogicalType::Enum => typed_reader!(StringColumnReader, Utf8), - lt => panic!("Unsupported logical type for BYTE_ARRAY: {lt:?}"), - } - } else { - typed_reader!(ByteArrayColumnReader, Binary) - } - } - PhysicalType::FIXED_LEN_BYTE_ARRAY => { - if let Some(logical_type) = desc.logical_type_ref() { - match logical_type { - LogicalType::Decimal { - precision, - scale: _, - } => { - if !use_decimal_128 && precision <= &DECIMAL_MAX_INT_DIGITS { - typed_reader!(FLBADecimal32ColumnReader, Int32) - } else if !use_decimal_128 - && promotion_info.precision <= DECIMAL_MAX_LONG_DIGITS - { - typed_reader!( - FLBADecimal64ColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } else { - typed_reader!( - FLBADecimalColumnReader, - ArrowDataType::Decimal128( - promotion_info.precision as u8, - promotion_info.scale as i8 - ) - ) - } - } - LogicalType::Uuid => { - let type_length = desc.type_length(); - typed_reader!( - FLBAColumnReader, - ArrowDataType::FixedSizeBinary(type_length) - ) - } - t => panic!("Unsupported logical type for FIXED_LEN_BYTE_ARRAY: {t:?}"), - } - } else { - let type_length = desc.type_length(); - typed_reader!( - FLBAColumnReader, - ArrowDataType::FixedSizeBinary(type_length) - ) - } - } - } - } -} - -macro_rules! make_func { - ($self:ident, $func:ident $(,$args:ident)*) => ({ - match *$self { - Self::BoolColumnReader(ref typed) => typed.$func($($args),*), - Self::Int8ColumnReader(ref typed) => typed.$func($($args),*), - Self::UInt8ColumnReader(ref typed) => typed.$func($($args),*), - Self::Int16ColumnReader(ref typed) => typed.$func($($args),*), - Self::Int16ToDoubleColumnReader(ref typed) => typed.$func($($args), *), - Self::UInt16ColumnReader(ref typed) => typed.$func($($args),*), - Self::Int32ColumnReader(ref typed) => typed.$func($($args),*), - Self::Int32To64ColumnReader(ref typed) => typed.$func($($args), *), - Self::Int32ToDecimal64ColumnReader(ref typed) => typed.$func($($args), *), - Self::Int32ToDoubleColumnReader(ref typed) => typed.$func($($args), *), - Self::UInt32ColumnReader(ref typed) => typed.$func($($args),*), - Self::Int32DateColumnReader(ref typed) => typed.$func($($args),*), - Self::Int32DecimalColumnReader(ref typed) => typed.$func($($args),*), - Self::Int32TimestampMicrosColumnReader(ref typed) => typed.$func($($args),*), - Self::Int64ColumnReader(ref typed) => typed.$func($($args),*), - Self::Int64ToDecimal64ColumnReader(ref typed) => typed.$func($($args), *), - Self::UInt64DecimalColumnReader(ref typed) => typed.$func($($args),*), - Self::Int64DecimalColumnReader(ref typed) => typed.$func($($args),*), - Self::Int64TimestampMillisColumnReader(ref typed) => typed.$func($($args),*), - Self::Int64TimestampMicrosColumnReader(ref typed) => typed.$func($($args),*), - Self::Int64TimestampNanosColumnReader(ref typed) => typed.$func($($args),*), - Self::FloatColumnReader(ref typed) => typed.$func($($args),*), - Self::DoubleColumnReader(ref typed) => typed.$func($($args),*), - Self::FloatToDoubleColumnReader(ref typed) => typed.$func($($args),*), - Self::ByteArrayColumnReader(ref typed) => typed.$func($($args),*), - Self::StringColumnReader(ref typed) => typed.$func($($args),*), - Self::FLBADecimalColumnReader(ref typed) => typed.$func($($args),*), - Self::FLBADecimal32ColumnReader(ref typed) => typed.$func($($args),*), - Self::FLBADecimal64ColumnReader(ref typed) => typed.$func($($args),*), - Self::FLBAColumnReader(ref typed) => typed.$func($($args),*), - Self::Int96ColumnReader(ref typed) => typed.$func($($args),*), - } - }); -} - -macro_rules! make_func_mut { - ($self:ident, $func:ident $(,$args:ident)*) => ({ - match *$self { - Self::BoolColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int8ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::UInt8ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int16ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int16ToDoubleColumnReader(ref mut typed) => typed.$func($($args), *), - Self::UInt16ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int32ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int32To64ColumnReader(ref mut typed) => typed.$func($($args), *), - Self::Int32ToDecimal64ColumnReader(ref mut typed) => typed.$func($($args), *), - Self::Int32ToDoubleColumnReader(ref mut typed) => typed.$func($($args), *), - Self::UInt32ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int32DateColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int32DecimalColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int32TimestampMicrosColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int64ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int64ToDecimal64ColumnReader(ref mut typed) => typed.$func($($args), *), - Self::UInt64DecimalColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int64DecimalColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int64TimestampMillisColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int64TimestampMicrosColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int64TimestampNanosColumnReader(ref mut typed) => typed.$func($($args),*), - Self::FloatColumnReader(ref mut typed) => typed.$func($($args),*), - Self::DoubleColumnReader(ref mut typed) => typed.$func($($args),*), - Self::FloatToDoubleColumnReader(ref mut typed) => typed.$func($($args),*), - Self::ByteArrayColumnReader(ref mut typed) => typed.$func($($args),*), - Self::StringColumnReader(ref mut typed) => typed.$func($($args),*), - Self::FLBADecimalColumnReader(ref mut typed) => typed.$func($($args),*), - Self::FLBADecimal32ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::FLBADecimal64ColumnReader(ref mut typed) => typed.$func($($args),*), - Self::FLBAColumnReader(ref mut typed) => typed.$func($($args),*), - Self::Int96ColumnReader(ref mut typed) => typed.$func($($args),*), - } - }); -} - -impl ColumnReader { - #[inline] - pub fn get_descriptor(&self) -> &ColumnDescriptor { - make_func!(self, get_descriptor) - } - - #[inline] - pub fn set_dictionary_page( - &mut self, - page_value_count: usize, - page_data: Buffer, - encoding: Encoding, - ) { - make_func_mut!( - self, - set_dictionary_page, - page_value_count, - page_data, - encoding - ) - } - - #[inline] - pub fn set_page_v1(&mut self, page_value_count: usize, page_data: Buffer, encoding: Encoding) { - make_func_mut!(self, set_page_v1, page_value_count, page_data, encoding) - } - - #[inline] - pub fn set_page_v2( - &mut self, - page_value_count: usize, - def_level_data: Buffer, - rep_level_data: Buffer, - value_data: Buffer, - encoding: Encoding, - ) { - make_func_mut!( - self, - set_page_v2, - page_value_count, - def_level_data, - rep_level_data, - value_data, - encoding - ) - } - - #[inline] - pub fn reset_batch(&mut self) { - make_func_mut!(self, reset_batch) - } - - #[inline] - pub fn current_batch(&mut self) -> Result { - make_func_mut!(self, current_batch) - } - - #[inline] - pub fn read_batch(&mut self, total: usize, null_pad_size: usize) -> (usize, usize) { - make_func_mut!(self, read_batch, total, null_pad_size) - } - - #[inline] - pub fn skip_batch(&mut self, total: usize, put_nulls: bool) -> usize { - make_func_mut!(self, skip_batch, total, put_nulls) - } -} - -/// A batched reader for a primitive Parquet column. -pub struct TypedColumnReader { - desc: ColumnDescPtr, - arrow_type: ArrowDataType, - rep_level_decoder: Option, - def_level_decoder: Option, - value_decoder: Option>, - - /// The remaining number of values to read in the current page - num_values_in_page: usize, - /// The value vector for this column reader; reused across batches. - vector: ParquetMutableVector, - /// The batch size for this column vector. - capacity: usize, - /// Number of bits used to represent one value in Parquet. - bit_width: usize, - // Options for reading Parquet - read_options: ReadOptions, - - /// Marker to allow `T` in the generic parameter of the struct. - _phantom: PhantomData, -} - -impl TypedColumnReader { - pub fn new( - desc: ColumnDescriptor, - capacity: usize, - arrow_type: ArrowDataType, - read_options: ReadOptions, - ) -> Self { - let vector = ParquetMutableVector::new(capacity, &arrow_type); - let bit_width = ParquetMutableVector::bit_width(&arrow_type); - Self { - desc: Arc::new(desc), - arrow_type, - rep_level_decoder: None, - def_level_decoder: None, - value_decoder: None, - num_values_in_page: 0, - vector, - capacity, - bit_width, - read_options, - _phantom: PhantomData, - } - } - - #[inline] - pub fn get_descriptor(&self) -> &ColumnDescriptor { - &self.desc - } - - /// Reset the current batch. This will clear all the content of the current columnar vector as - /// well as reset all of its internal states. - #[inline] - pub fn reset_batch(&mut self) { - self.vector.reset() - } - - /// Returns the current batch that's been constructed. - /// - /// Note: the caller must make sure the returned Arrow vector is fully consumed before calling - /// `read_batch` again. - #[inline] - pub fn current_batch(&mut self) -> Result { - self.vector.get_array_data() - } - - /// Reads a batch of at most `total` values from the current page this reader has. Returns a - /// tuple where the first element is the actual number of values read (including both nulls and - /// non-nulls), and the second element is the actual number of nulls read. - /// - /// Pad nulls for the amount of `null_pad_size` before reading. - /// - /// If the return number of values is < `total`, it means the current page is drained and the - /// caller should call `set_page_v1` or `set_page_v2` before calling next `read_batch`. - pub fn read_batch(&mut self, total: usize, null_pad_size: usize) -> (usize, usize) { - debug_assert!( - self.value_decoder.is_some() && self.def_level_decoder.is_some(), - "set_page_v1/v2 should have been called" - ); - - let n = ::std::cmp::min(self.num_values_in_page, total); - self.num_values_in_page -= n; - let value_decoder = self.value_decoder.as_mut().unwrap(); - let dl_decoder = self.def_level_decoder.as_mut().unwrap(); - - let previous_num_nulls = self.vector.num_nulls; - self.vector.put_nulls(null_pad_size); - dl_decoder.read_batch(n, &mut self.vector, value_decoder.as_mut()); - - (n, self.vector.num_nulls - previous_num_nulls) - } - - /// Skips a batch of at most `total` values from the current page this reader has, and returns - /// the actual number of values skipped. - /// - /// If the return value is < `total`, it means the current page is drained and the caller should - /// call `set_page_v1` or `set_page_v2` before calling next `skip_batch`. - pub fn skip_batch(&mut self, total: usize, put_nulls: bool) -> usize { - debug_assert!( - self.value_decoder.is_some() && self.def_level_decoder.is_some(), - "set_page_v1/v2 should have been called" - ); - - let n = ::std::cmp::min(self.num_values_in_page, total); - self.num_values_in_page -= n; - let value_decoder = self.value_decoder.as_mut().unwrap(); - let dl_decoder = self.def_level_decoder.as_mut().unwrap(); - - dl_decoder.skip_batch(n, &mut self.vector, value_decoder.as_mut(), put_nulls); - - n - } - - /// Sets the dictionary page for this column reader and eagerly reads it. - /// - /// # Panics - /// - /// - If being called more than once during the lifetime of this column reader. A Parquet column - /// chunk should only contain a single dictionary page. - /// - If the input `encoding` is neither `PLAIN`, `PLAIN_DICTIONARY` nor `RLE_DICTIONARY`. - pub fn set_dictionary_page( - &mut self, - page_value_count: usize, - page_data: Buffer, - mut encoding: Encoding, - ) { - // In Parquet format v1, both dictionary page and data page use the same encoding - // `PLAIN_DICTIONARY`, while in v2, dictioanry page uses `PLAIN` and data page uses - // `RLE_DICTIONARY`. - // - // Here, we convert `PLAIN` from v2 dictionary page to `PLAIN_DICTIONARY`, so that v1 and v2 - // shares the same encoding. Later on, `get_decoder` will use the `PlainDecoder` for this - // case. - if encoding == Encoding::PLAIN { - encoding = Encoding::PLAIN_DICTIONARY; - } - - if encoding != Encoding::PLAIN_DICTIONARY { - panic!("Invalid encoding type for Parquet dictionary: {encoding}"); - } - - if self.vector.dictionary.is_some() { - panic!("Parquet column cannot have more than one dictionary"); - } - - // Create a new vector for dictionary values - let mut value_vector = ParquetMutableVector::new(page_value_count, &self.arrow_type); - - let mut dictionary = self.get_decoder(page_data, encoding); - dictionary.read_batch(&mut value_vector, page_value_count); - value_vector.num_values = page_value_count; - - // Re-create the parent vector since it is initialized with the dictionary value type, not - // the key type (which is always integer). - self.vector = ParquetMutableVector::new(self.capacity, &ArrowDataType::Int32); - self.vector.set_dictionary(value_vector); - } - - /// Resets the Parquet data page for this column reader. - pub fn set_page_v1( - &mut self, - page_value_count: usize, - page_data: Buffer, - mut encoding: Encoding, - ) { - // In v1, when data is encoded with dictionary, data page uses `PLAIN_DICTIONARY`, while v2 - // uses `RLE_DICTIONARY`. To consolidate the two, here we convert `PLAIN_DICTIONARY` to - // `RLE_DICTIONARY` following v2. Later on, `get_decoder` will use `DictDecoder` for this - // case. - if encoding == Encoding::PLAIN_DICTIONARY { - encoding = Encoding::RLE_DICTIONARY; - } - - self.num_values_in_page = page_value_count; - self.check_dictionary(&encoding); - - let mut page_buffer = page_data; - - let bit_width = log2(self.desc.max_rep_level() as u64 + 1) as u8; - let mut rl_decoder = LevelDecoder::new(Arc::clone(&self.desc), bit_width, true); - let offset = rl_decoder.set_data(page_value_count, &page_buffer); - self.rep_level_decoder = Some(rl_decoder); - page_buffer = page_buffer.slice(offset); - - let bit_width = log2(self.desc.max_def_level() as u64 + 1) as u8; - let mut dl_decoder = LevelDecoder::new(Arc::clone(&self.desc), bit_width, true); - let offset = dl_decoder.set_data(page_value_count, &page_buffer); - self.def_level_decoder = Some(dl_decoder); - page_buffer = page_buffer.slice(offset); - - let value_decoder = self.get_decoder(page_buffer, encoding); - self.value_decoder = Some(value_decoder); - } - - /// Resets the Parquet data page for this column reader. - pub fn set_page_v2( - &mut self, - page_value_count: usize, - def_level_data: Buffer, - rep_level_data: Buffer, - value_data: Buffer, - encoding: Encoding, - ) { - self.num_values_in_page = page_value_count; - self.check_dictionary(&encoding); - - let bit_width = log2(self.desc.max_rep_level() as u64 + 1) as u8; - let mut rl_decoder = LevelDecoder::new(Arc::clone(&self.desc), bit_width, false); - rl_decoder.set_data(page_value_count, &rep_level_data); - self.rep_level_decoder = Some(rl_decoder); - - let bit_width = log2(self.desc.max_def_level() as u64 + 1) as u8; - let mut dl_decoder = LevelDecoder::new(Arc::clone(&self.desc), bit_width, false); - dl_decoder.set_data(page_value_count, &def_level_data); - self.def_level_decoder = Some(dl_decoder); - - let value_decoder = self.get_decoder(value_data, encoding); - self.value_decoder = Some(value_decoder); - } - - fn check_dictionary(&mut self, encoding: &Encoding) { - // The column has a dictionary while the new page is of PLAIN encoding. In this case, we - // should eagerly decode all the dictionary indices and convert the underlying vector to a - // plain encoded vector. - if self.vector.dictionary.is_some() && *encoding == Encoding::PLAIN { - let new_vector = ParquetMutableVector::new(self.capacity, &self.arrow_type); - let old_vector = std::mem::replace(&mut self.vector, new_vector); - T::decode_dict(old_vector, &mut self.vector, self.bit_width); - debug_assert!(self.vector.dictionary.is_none()); - } - } - - fn get_decoder(&self, value_data: Buffer, encoding: Encoding) -> Box { - get_decoder::( - value_data, - encoding, - Arc::clone(&self.desc), - self.read_options, - ) - } -} diff --git a/native/core/src/parquet/read/levels.rs b/native/core/src/parquet/read/levels.rs deleted file mode 100644 index 27dd3568af..0000000000 --- a/native/core/src/parquet/read/levels.rs +++ /dev/null @@ -1,229 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::mem; - -use super::values::Decoder; -use crate::{ - common::bit::{self, read_u32, BitReader}, - parquet::ParquetMutableVector, -}; -use arrow::buffer::Buffer; -use datafusion_comet_spark_expr::utils::unlikely; -use parquet::schema::types::ColumnDescPtr; - -const INITIAL_BUF_LEN: usize = 16; - -enum Mode { - RLE, - BitPacked, -} - -/// A decoder for Parquet definition & repetition levels. -pub struct LevelDecoder { - /// The descriptor of the column that this level decoder is associated to. - desc: ColumnDescPtr, - /// Number of bits used to represent the levels. - bit_width: u8, - /// Mode for the current run. - mode: Mode, - /// Total number of values (including both null and non-null) to be decoded. - num_values: usize, - /// The current value in a RLE run. Unused if BitPacked. - current_value: i32, - /// The number of total values in the current RLE run. Unused if BitPacked. - current_count: usize, - /// The current buffer used in a BitPacked run. Unused if RLE. - /// This will be resized if the total number of values in the BitPacked run is larger than its - /// capacity. - current_buffer: Vec, // TODO: double check this - /// The index into `current_buffer` in a BitPacked run. Unused if RLE. - current_buffer_idx: usize, - /// A bit reader wrapping the input buffer for levels. - bit_reader: Option, - /// Whether we need to read the length of data. This is typically true for Parquet page V1, but - /// not for V2, since it uses separate buffer for definition & repetition levels. - need_length: bool, -} - -impl LevelDecoder { - pub fn new(desc: ColumnDescPtr, bit_width: u8, need_length: bool) -> Self { - Self { - desc, - bit_width, - mode: Mode::RLE, - num_values: 0, - current_value: 0, - current_count: 0, - current_buffer: vec![0; INITIAL_BUF_LEN], - current_buffer_idx: 0, - bit_reader: None, - need_length, - } - } - - /// Sets data for this level decoder, and returns total number of bytes consumed. This is used - /// for reading DataPage v1 levels. - pub fn set_data(&mut self, page_value_count: usize, page_data: &Buffer) -> usize { - self.num_values = page_value_count; - if self.bit_width == 0 { - // Special case where the page doesn't have encoded rl/dl data. Here we'll treat it as - // an RLE run of `page_value_count` number of 0s. - self.mode = Mode::RLE; - self.current_count = page_value_count; - 0 - } else if self.need_length { - let u32_size = mem::size_of::(); - let data_size = read_u32(page_data.as_slice()) as usize; - self.bit_reader = Some(BitReader::new(page_data.slice(u32_size), data_size)); - u32_size + data_size - } else { - // No need to read length, just read the whole buffer - self.bit_reader = Some(BitReader::new_all(page_data.to_owned())); - 0 - } - } - - /// Reads a batch of `total` values into `vector`. The value decoding is done by - /// `value_decoder`. - pub fn read_batch( - &mut self, - total: usize, - vector: &mut ParquetMutableVector, - value_decoder: &mut dyn Decoder, - ) { - let mut left = total; - while left > 0 { - if unlikely(self.current_count == 0) { - self.read_next_group(); - } - - debug_assert!(self.current_count > 0); - - let n = ::std::cmp::min(left, self.current_count); - let max_def_level = self.desc.max_def_level(); - - match self.mode { - Mode::RLE => { - if self.current_value as i16 == max_def_level { - bit::set_bits(vector.validity_buffer.as_slice_mut(), vector.num_values, n); - value_decoder.read_batch(vector, n); - vector.num_values += n; - } else { - vector.put_nulls(n); - } - } - Mode::BitPacked => { - for i in 0..n { - if self.current_buffer[self.current_buffer_idx + i] == max_def_level as i32 - { - bit::set_bit(vector.validity_buffer.as_slice_mut(), vector.num_values); - value_decoder.read(vector); - vector.num_values += 1; - } else { - vector.put_null(); - } - } - self.current_buffer_idx += n; - } - } - - left -= n; - self.current_count -= n; - } - } - - /// Skips a batch of `total` values. The value decoding is done by `value_decoder`. - pub fn skip_batch( - &mut self, - total: usize, - vector: &mut ParquetMutableVector, - value_decoder: &mut dyn Decoder, - put_nulls: bool, - ) { - let mut skip = total; - while skip > 0 { - if unlikely(self.current_count == 0) { - self.read_next_group(); - } - - debug_assert!(self.current_count > 0); - - let n = ::std::cmp::min(skip, self.current_count); - let max_def_level = self.desc.max_def_level(); - - match self.mode { - Mode::RLE => { - if self.current_value as i16 == max_def_level { - value_decoder.skip_batch(n); - } - } - Mode::BitPacked => { - let mut num_skips = 0; - for i in 0..n { - if self.current_buffer[self.current_buffer_idx + i] == max_def_level as i32 - { - num_skips += 1; - } - } - value_decoder.skip_batch(num_skips); - self.current_buffer_idx += n; - } - } - if put_nulls { - vector.put_nulls(n); - } - - skip -= n; - self.current_count -= n; - } - } - - /// Loads the next group from this RLE/BitPacked hybrid reader. - fn read_next_group(&mut self) { - let bit_reader = self.bit_reader.as_mut().expect("bit_reader should be set"); - if let Some(indicator_value) = bit_reader.get_vlq_int() { - self.mode = if indicator_value & 1 == 1 { - Mode::BitPacked - } else { - Mode::RLE - }; - - match self.mode { - Mode::BitPacked => { - self.current_count = ((indicator_value >> 1) * 8) as usize; - if self.current_buffer.len() < self.current_count { - self.current_buffer.resize(self.current_count, 0); - } - self.current_buffer_idx = 0; - bit_reader.get_batch( - &mut self.current_buffer[..self.current_count], - self.bit_width as usize, - ); - } - Mode::RLE => { - // RLE - self.current_count = (indicator_value >> 1) as usize; - let value_width = (self.bit_width as usize).div_ceil(8); - self.current_value = bit_reader - .get_aligned::(value_width) - .expect("current value should be set"); - } - } - } - } -} diff --git a/native/core/src/parquet/read/mod.rs b/native/core/src/parquet/read/mod.rs deleted file mode 100644 index f66f9da3da..0000000000 --- a/native/core/src/parquet/read/mod.rs +++ /dev/null @@ -1,108 +0,0 @@ -// 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. - -pub mod column; -pub mod levels; -pub mod values; - -pub use column::ColumnReader; -use parquet::schema::types::ColumnDescPtr; - -use super::ParquetMutableVector; -use crate::common::bit::{self, BitReader}; -use arrow::buffer::Buffer; -use bytes::Buf; - -#[derive(Clone, Copy)] -pub struct ReadOptions { - // Whether to read legacy dates/timestamps as it is. If false, throw exceptions. - pub(crate) use_legacy_date_timestamp_or_ntz: bool, -} - -/// Internal states for PLAIN decoder. Used in combination of `PlainDecoding`. -pub struct PlainDecoderInner { - /// The input buffer containing values to be decoded - data: Buffer, - - /// The current offset in `data`, in bytes. - offset: usize, - - /// Reads `data` bit by bit, used if `T` is [`BoolType`]. - bit_reader: BitReader, - - /// Options for reading Parquet - read_options: ReadOptions, - - /// The Parquet column descriptor - desc: ColumnDescPtr, -} - -/// A trait for [`super::DataType`] to implement how PLAIN encoded data is to be decoded into Arrow -/// format given an input and output buffer. -/// -/// The actual implementations of this trait is in `read/values.rs`. -pub trait PlainDecoding { - /// Decodes `num` of items from `src`, and store the result into `dst`, in Arrow format. - /// - /// Note: this assumes the `src` has data for at least `num` elements, and won't do any - /// bound checking. The condition MUST be guaranteed from the caller side. - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize); - - /// Skip `num` of items from `src` - /// - /// Note: this assumes the `src` has data for at least `num` elements, and won't do any - /// bound checking. The condition MUST be guaranteed from the caller side. - fn skip(src: &mut PlainDecoderInner, num: usize); -} - -pub trait PlainDictDecoding { - /// Eagerly decode vector `src` which must have dictionary associated. The decoded values are - /// appended into `dst`. - fn decode_dict(src: ParquetMutableVector, dst: &mut ParquetMutableVector, bit_width: usize) { - assert!(dst.dictionary.is_none()); - assert!(src.dictionary.is_some()); - - let mut value_buf = src.value_buffer.as_slice(); - let validity_buf = src.validity_buffer.as_slice(); - let dictionary = src.dictionary.as_ref().unwrap(); - - for i in 0..src.num_values { - if bit::get_bit(validity_buf, i) { - // non-null value: lookup the value position and copy its value into `dst` - let val_idx = value_buf.get_u32_le(); - Self::decode_dict_one(i, val_idx as usize, dictionary, dst, bit_width); - dst.num_values += 1; - } else { - value_buf.advance(4); - dst.put_null(); - } - } - - dst.validity_buffer = src.validity_buffer; - } - - /// Decode a single value from `src`, whose position in the dictionary indices (i.e., keys) - /// is `idx` and the positions in the dictionary values is `val_idx`. The decoded value is - /// appended to `dst`. - fn decode_dict_one( - idx: usize, - val_idx: usize, - src: &ParquetMutableVector, - dst: &mut ParquetMutableVector, - bit_width: usize, - ); -} diff --git a/native/core/src/parquet/read/values.rs b/native/core/src/parquet/read/values.rs deleted file mode 100644 index 58f083eca9..0000000000 --- a/native/core/src/parquet/read/values.rs +++ /dev/null @@ -1,1131 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{marker::PhantomData, mem}; - -use arrow::buffer::Buffer; -use bytes::Buf; -use log::debug; -use parquet::{basic::Encoding, schema::types::ColumnDescPtr}; - -use super::{PlainDecoderInner, PlainDecoding, PlainDictDecoding, ReadOptions}; -use crate::write_null; -use crate::write_val_or_null; -use crate::{ - common::bit::{self, BitReader}, - parquet::{data_type::*, ParquetMutableVector}, -}; -use arrow::datatypes::DataType as ArrowDataType; -use datafusion_comet_spark_expr::utils::unlikely; - -pub fn get_decoder( - value_data: Buffer, - encoding: Encoding, - desc: ColumnDescPtr, - read_options: ReadOptions, -) -> Box { - let decoder: Box = match encoding { - Encoding::PLAIN | Encoding::PLAIN_DICTIONARY => { - Box::new(PlainDecoder::::new(value_data, desc, read_options)) - } - // This is for dictionary indices - Encoding::RLE_DICTIONARY => Box::new(DictDecoder::new(value_data)), - _ => panic!("Unsupported encoding: {encoding}"), - }; - decoder -} - -/// A Parquet decoder for values within a Parquet data page. -pub trait Decoder { - /// Consumes a single value from the decoder and stores it into `dst`. - /// - /// # Preconditions - /// - /// * `dst` have enough length to hold at least one value. - /// * `data` of this decoder should have enough bytes left to be decoded. - fn read(&mut self, dst: &mut ParquetMutableVector); - - /// Consumes a batch of `num` values from the data and stores them to `dst`. - /// - /// # Preconditions - /// - /// * `dst` should have length >= `num * T::type_size()` . - /// * `data` of this decoder should have >= `num * T::type_size()` bytes left to be decoded. - fn read_batch(&mut self, dst: &mut ParquetMutableVector, num: usize); - - /// Skips a batch of `num` values from the data. - /// - /// # Preconditions - /// - /// * `data` of this decoder should have >= `num * T::type_size()` bytes left to be decoded. - fn skip_batch(&mut self, num: usize); - - /// Returns the encoding for this decoder. - fn encoding(&self) -> Encoding; -} - -/// The switch off date between Julian and Gregorian calendar. See -/// https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html -const JULIAN_GREGORIAN_SWITCH_OFF_DAY: i32 = -141427; - -/// The switch off timestamp (in micros) between Julian and Gregorian calendar. See -/// https://docs.oracle.com/javase/7/docs/api/java/util/GregorianCalendar.html -const JULIAN_GREGORIAN_SWITCH_OFF_TS: i64 = -2208988800000000; - -/// See http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian -/// Also see Spark's `DateTimeUtils.JULIAN_DAY_OF_EPOCH` -const JULIAN_DAY_OF_EPOCH: i32 = 2440588; - -/// Number of micro seconds per milli second. -const MICROS_PER_MILLIS: i64 = 1000; - -const MICROS_PER_DAY: i64 = 24_i64 * 60 * 60 * 1000 * 1000; - -pub struct PlainDecoder { - /// Internal states for this decoder. - inner: PlainDecoderInner, - - /// Marker to allow `T` in the generic parameter of the struct. - _phantom: PhantomData, -} - -impl PlainDecoder { - pub fn new(value_data: Buffer, desc: ColumnDescPtr, read_options: ReadOptions) -> Self { - let len = value_data.len(); - let inner = PlainDecoderInner { - data: value_data.clone(), - offset: 0, - bit_reader: BitReader::new(value_data, len), - read_options, - desc, - }; - Self { - inner, - _phantom: PhantomData, - } - } -} - -macro_rules! make_plain_default_impl { - ($($ty: ident), *) => { - $( - impl PlainDecoding for $ty { - /// Default implementation for PLAIN encoding. Uses `mempcy` when the physical - /// layout is the same between Parquet and Arrow. - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - let byte_width = src.desc.type_length() as usize; - let num_bytes = byte_width * num; - let dst_offset = byte_width * dst.num_values; - - bit::memcpy( - &src_data[src.offset..src.offset + num_bytes], - &mut dst.value_buffer[dst_offset..]); - src.offset += num_bytes; - } - - fn skip(src: &mut PlainDecoderInner, num: usize) { - let num_bytes = src.desc.type_length() as usize * num; - src.offset += num_bytes; - } - } - )* - }; -} - -make_plain_default_impl! { Int32Type, Int64Type, FloatType, DoubleType, FLBAType } - -macro_rules! make_plain_dict_impl { - ($($ty: ident), *) => { - $( - impl PlainDictDecoding for $ty { - fn decode_dict_one( - idx: usize, - val_idx: usize, - src: &ParquetMutableVector, - dst: &mut ParquetMutableVector, - bit_width: usize, - ) { - let byte_width = bit_width / 8; - bit::memcpy( - &src.value_buffer[val_idx * byte_width..(val_idx+1) * byte_width], - &mut dst.value_buffer[idx * byte_width..], - ); - } - } - )* - }; -} - -make_plain_dict_impl! { Int8Type, UInt8Type, Int16Type, UInt16Type, Int32Type, UInt32Type } -make_plain_dict_impl! { Int32DateType, Int64Type, FloatType, Int64ToDecimal64Type, FLBAType } -make_plain_dict_impl! { DoubleType, Int64TimestampMillisType, Int64TimestampMicrosType } - -macro_rules! make_int_variant_dict_impl { - ($ty:ty, $src_ty:ty, $dst_ty:ty) => { - impl PlainDictDecoding for $ty { - fn decode_dict_one( - idx: usize, - val_idx: usize, - src: &ParquetMutableVector, - dst: &mut ParquetMutableVector, - _: usize, - ) { - let src_ptr = src.value_buffer.as_ptr() as *const $src_ty; - let dst_ptr = dst.value_buffer.as_mut_ptr() as *mut $dst_ty; - unsafe { - // SAFETY the caller must ensure `idx`th pointer is in bounds - dst_ptr - .add(idx) - .write_unaligned(src_ptr.add(val_idx).read_unaligned() as $dst_ty); - } - } - } - }; -} - -make_int_variant_dict_impl!(Int16ToDoubleType, i16, f64); -make_int_variant_dict_impl!(Int32To64Type, i32, i64); -make_int_variant_dict_impl!(Int32ToDecimal64Type, i32, i64); -make_int_variant_dict_impl!(Int32ToDoubleType, i32, f64); -make_int_variant_dict_impl!(Int32TimestampMicrosType, i32, i64); -make_int_variant_dict_impl!(FloatToDoubleType, f32, f64); -make_int_variant_dict_impl!(Int32DecimalType, i128, i128); -make_int_variant_dict_impl!(Int64DecimalType, i128, i128); -make_int_variant_dict_impl!(UInt64Type, u128, u128); - -impl PlainDecoding for Int32DateType { - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - let byte_width = src.desc.type_length() as usize; - let num_bytes = byte_width * num; - let dst_offset = byte_width * dst.num_values; - - if !src.read_options.use_legacy_date_timestamp_or_ntz { - // By default we panic if the date value is before the switch date between Julian - // calendar and Gregorian calendar, which is 1582-10-15, and -141727 days - // before the unix epoch date 1970-01-01. - let mut offset = src.offset; - for _ in 0..num { - let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 - as *const i32; - - // TODO: optimize this further as checking value one by one is not very efficient - unsafe { - if unlikely(v.read_unaligned() < JULIAN_GREGORIAN_SWITCH_OFF_DAY) { - panic!( - "Encountered date value {}, which is before 1582-10-15 (counting backwards \ - from Unix epoch date 1970-01-01), and could be ambigous depending on \ - whether a legacy Julian/Gregorian hybrid calendar is used, or a Proleptic \ - Gregorian calendar is used.", - *v - ); - } - } - - offset += byte_width; - } - } - - bit::memcpy( - &src_data[src.offset..src.offset + num_bytes], - &mut dst.value_buffer[dst_offset..], - ); - - src.offset += num_bytes; - } - - fn skip(src: &mut PlainDecoderInner, num: usize) { - let num_bytes = src.desc.type_length() as usize * num; - src.offset += num_bytes; - } -} - -impl PlainDecoding for Int32TimestampMicrosType { - #[inline] - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - let byte_width = src.desc.type_length() as usize; - let num_bytes = byte_width * num; - - { - let mut offset = src.offset; - for _ in 0..num { - let v = src_data[offset..offset + byte_width].as_ptr() as *const i32; - - // TODO: optimize this further as checking value one by one is not very efficient - unsafe { - if unlikely(v.read_unaligned() < JULIAN_GREGORIAN_SWITCH_OFF_DAY) { - panic!( - "Encountered timestamp value {}, which is before 1582-10-15 (counting \ - backwards from Unix epoch date 1970-01-01), and could be ambigous \ - depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ - or a Proleptic Gregorian calendar is used.", - *v - ); - } - } - - offset += byte_width; - } - } - - let mut offset = src.offset; - let dst_byte_width = byte_width * 2; - let mut dst_offset = dst_byte_width * dst.num_values; - for _ in 0..num { - let v = src_data[offset..offset + byte_width].as_ptr() as *const i32; - let v = unsafe { v.read_unaligned() }; - let v = (v as i64).wrapping_mul(MICROS_PER_DAY); - bit::memcpy_value(&v, dst_byte_width, &mut dst.value_buffer[dst_offset..]); - offset += byte_width; - dst_offset += dst_byte_width; - } - - src.offset += num_bytes; - } - - #[inline] - fn skip(src: &mut PlainDecoderInner, num: usize) { - let num_bytes = src.desc.type_length() as usize * num; - src.offset += num_bytes; - } -} - -impl PlainDecoding for Int64TimestampMillisType { - #[inline] - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - let byte_width = src.desc.type_length() as usize; - let num_bytes = byte_width * num; - - if !src.read_options.use_legacy_date_timestamp_or_ntz { - let mut offset = src.offset; - for _ in 0..num { - unsafe { - let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 - as *const i64; - let v = v.read_unaligned() * MICROS_PER_MILLIS; - - // TODO: optimize this further as checking value one by one is not very - // efficient - if unlikely(v < JULIAN_GREGORIAN_SWITCH_OFF_TS) { - panic!( - "Encountered timestamp value {v}, which is before 1582-10-15 (counting \ - backwards from Unix epoch date 1970-01-01), and could be ambigous \ - depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ - or a Proleptic Gregorian calendar is used." - ); - } - - offset += byte_width; - } - } - } - - unsafe { - let mut offset = src.offset; - let mut dst_offset = byte_width * dst.num_values; - for _ in 0..num { - let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 - as *const i64; - let v = v.read_unaligned() * MICROS_PER_MILLIS; - bit::memcpy_value(&v, byte_width, &mut dst.value_buffer[dst_offset..]); - offset += byte_width; - dst_offset += byte_width; - } - } - - src.offset += num_bytes; - } - - #[inline] - fn skip(src: &mut PlainDecoderInner, num: usize) { - let num_bytes = src.desc.type_length() as usize * num; - src.offset += num_bytes; - } -} - -impl PlainDecoding for Int64TimestampMicrosType { - #[inline] - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - let byte_width = src.desc.type_length() as usize; - let num_bytes = byte_width * num; - let dst_offset = byte_width * dst.num_values; - - if !src.read_options.use_legacy_date_timestamp_or_ntz { - let mut offset = src.offset; - for _ in 0..num { - unsafe { - let v = &src_data[offset..offset + byte_width] as *const [u8] as *const u8 - as *const i64; - - // TODO: optimize this further as checking value one by one is not very - // efficient - if unlikely(v.read_unaligned() < JULIAN_GREGORIAN_SWITCH_OFF_TS) { - panic!( - "Encountered timestamp value {}, which is before 1582-10-15 (counting \ - backwards from Unix epoch date 1970-01-01), and could be ambigous \ - depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ - or a Proleptic Gregorian calendar is used.", - *v - ); - } - - offset += byte_width; - } - } - } - - bit::memcpy( - &src_data[src.offset..src.offset + num_bytes], - &mut dst.value_buffer[dst_offset..], - ); - - src.offset += num_bytes; - } - - #[inline] - fn skip(src: &mut PlainDecoderInner, num: usize) { - let num_bytes = src.desc.type_length() as usize * num; - src.offset += num_bytes; - } -} - -impl PlainDecoding for BoolType { - /// Specific implementation for PLAIN encoding of boolean type. Even though both Parquet and - /// Arrow share the same physical layout for the type (which is 1 bit for each value), we'll - /// need to treat the number of bytes specifically. - #[inline] - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - src.bit_reader - .get_bits(&mut dst.value_buffer, dst.num_values, num); - } - - #[inline] - fn skip(src: &mut PlainDecoderInner, num: usize) { - src.bit_reader.skip_bits(num); - } -} - -// Does it make sense to encode booleans with dictionary? -impl PlainDictDecoding for BoolType { - #[inline] - fn decode_dict_one( - idx: usize, - val_idx: usize, - src: &ParquetMutableVector, - dst: &mut ParquetMutableVector, - _: usize, - ) { - let v = bit::get_bit(src.value_buffer.as_slice(), val_idx); - if v { - bit::set_bit(dst.value_buffer.as_slice_mut(), idx); - } // `dst` should be zero initialized so no need to call `unset_bit`. - } -} - -macro_rules! make_int_variant_impl { - ($dst_type:ty, $copy_fn:ident, $type_width:expr) => { - impl PlainDecoding for $dst_type { - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let dst_slice = dst.value_buffer.as_slice_mut(); - let dst_offset = dst.num_values * $type_width; - $copy_fn(&src.data[src.offset..], &mut dst_slice[dst_offset..], num); - src.offset += 4 * num; // Parquet stores Int8/Int16 using 4 bytes - } - - fn skip(src: &mut PlainDecoderInner, num: usize) { - src.offset += 4 * num; // Parquet stores Int8/Int16 using 4 bytes - } - } - }; -} - -make_int_variant_impl!(Int8Type, copy_i32_to_i8, 1); -make_int_variant_impl!(Int16Type, copy_i32_to_i16, 2); -make_int_variant_impl!(Int16ToDoubleType, copy_i32_to_f64, 8); // Parquet uses Int16 using 4 bytes -make_int_variant_impl!(Int32To64Type, copy_i32_to_i64, 8); -make_int_variant_impl!(Int32ToDoubleType, copy_i32_to_f64, 8); -make_int_variant_impl!(FloatToDoubleType, copy_f32_to_f64, 8); - -// unsigned type require double the width and zeroes are written for the second half -// because they are implemented as the next size up signed type -make_int_variant_impl!(UInt8Type, copy_i32_to_u8, 2); -make_int_variant_impl!(UInt16Type, copy_i32_to_u16, 4); -make_int_variant_impl!(UInt32Type, copy_i32_to_u32, 8); - -macro_rules! make_int_decimal_variant_impl { - ($ty:ty, $copy_fn:ident, $type_width:expr, $dst_type:ty) => { - impl PlainDecoding for $ty { - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let dst_slice = dst.value_buffer.as_slice_mut(); - let dst_offset = dst.num_values * std::mem::size_of::<$dst_type>(); - $copy_fn(&src.data[src.offset..], &mut dst_slice[dst_offset..], num); - - let src_precision = src.desc.type_precision() as u32; - let src_scale = ::std::cmp::max(src.desc.type_scale(), 0) as u32; - let (dst_precision, dst_scale) = match dst.arrow_type { - ArrowDataType::Decimal128(p, s) if s >= 0 => (p as u32, s as u32), - _ => unreachable!(), - }; - let upper = (10 as $dst_type).pow(dst_precision); - let v = dst_slice[dst_offset..].as_mut_ptr() as *mut $dst_type; - if dst_scale > src_scale { - let mul = (10 as $dst_type).pow(dst_scale - src_scale); - for i in 0..num { - unsafe { - // SAFETY the caller must ensure `i`th pointer is in bounds - let v = v.add(i); - write_val_or_null!(v, v.read_unaligned() * mul, upper, dst, i); - } - } - } else if dst_scale < src_scale { - let div = (10 as $dst_type).pow(src_scale - dst_scale); - for i in 0..num { - unsafe { - // SAFETY the caller must ensure `i`th pointer is in bounds - let v = v.add(i); - write_val_or_null!(v, v.read_unaligned() / div, upper, dst, i); - } - } - } else if src_precision > dst_precision { - for i in 0..num { - unsafe { - // SAFETY the caller must ensure `i`th pointer is in bounds - let v = v.add(i); - write_null!(v.read_unaligned(), upper, dst, i); - } - } - } - - src.offset += $type_width * num; - } - - fn skip(src: &mut PlainDecoderInner, num: usize) { - src.offset += $type_width * num; - } - } - }; -} -make_int_decimal_variant_impl!(Int32ToDecimal64Type, copy_i32_to_i64, 4, i64); -make_int_decimal_variant_impl!(Int32DecimalType, copy_i32_to_i128, 4, i128); -make_int_decimal_variant_impl!(Int64ToDecimal64Type, copy_i64_to_i64, 8, i64); -make_int_decimal_variant_impl!(Int64DecimalType, copy_i64_to_i128, 8, i128); -make_int_decimal_variant_impl!(UInt64Type, copy_u64_to_u128, 8, u128); - -#[macro_export] -macro_rules! write_val_or_null { - ($v: expr, $adjusted: expr, $upper: expr, $dst: expr, $i: expr) => { - let adjusted = $adjusted; - $v.write_unaligned(adjusted); - write_null!(adjusted, $upper, $dst, $i); - }; -} - -#[macro_export] -macro_rules! write_null { - ($val: expr, $upper: expr, $dst: expr, $i: expr) => { - if $upper <= $val { - bit::unset_bit($dst.validity_buffer.as_slice_mut(), $dst.num_values + $i); - $dst.num_nulls += 1; - } - }; -} - -macro_rules! generate_cast_to_unsigned { - ($name: ident, $src_type:ty, $dst_type:ty, $zero_value:expr) => { - pub fn $name(src: &[u8], dst: &mut [u8], num: usize) { - debug_assert!( - src.len() >= num * std::mem::size_of::<$src_type>(), - "Source slice is too small" - ); - debug_assert!( - dst.len() >= num * std::mem::size_of::<$dst_type>() * 2, - "Destination slice is too small" - ); - - let src_ptr = src.as_ptr() as *const $src_type; - let dst_ptr = dst.as_mut_ptr() as *mut $dst_type; - unsafe { - for i in 0..num { - dst_ptr - .add(2 * i) - .write_unaligned(src_ptr.add(i).read_unaligned() as $dst_type); - // write zeroes - dst_ptr.add(2 * i + 1).write_unaligned($zero_value); - } - } - } - }; -} - -generate_cast_to_unsigned!(copy_i32_to_u32, i32, u32, 0_u32); - -macro_rules! generate_cast_to_signed { - ($name: ident, $src_type:ty, $dst_type:ty) => { - pub fn $name(src: &[u8], dst: &mut [u8], num: usize) { - debug_assert!( - src.len() >= num * std::mem::size_of::<$src_type>(), - "Source slice is too small" - ); - debug_assert!( - dst.len() >= num * std::mem::size_of::<$dst_type>(), - "Destination slice is too small" - ); - - let src_ptr = src.as_ptr() as *const $src_type; - let dst_ptr = dst.as_mut_ptr() as *mut $dst_type; - unsafe { - for i in 0..num { - dst_ptr - .add(i) - .write_unaligned(src_ptr.add(i).read_unaligned() as $dst_type); - } - } - } - }; -} - -generate_cast_to_signed!(copy_i32_to_i8, i32, i8); -generate_cast_to_signed!(copy_i32_to_i16, i32, i16); -generate_cast_to_signed!(copy_i32_to_i64, i32, i64); -generate_cast_to_signed!(copy_i32_to_i128, i32, i128); -generate_cast_to_signed!(copy_i32_to_f64, i32, f64); -generate_cast_to_signed!(copy_i64_to_i64, i64, i64); -generate_cast_to_signed!(copy_i64_to_i128, i64, i128); -generate_cast_to_signed!(copy_u64_to_u128, u64, u128); -generate_cast_to_signed!(copy_f32_to_f64, f32, f64); -// even for u8/u16, need to copy full i16/i32 width for Spark compatibility -generate_cast_to_signed!(copy_i32_to_u8, i32, i16); -generate_cast_to_signed!(copy_i32_to_u16, i32, i32); - -// Shared implementation for variants of Binary type -macro_rules! make_plain_binary_impl { - ($($ty: ident), *) => { - $( - impl PlainDecoding for $ty { - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - let mut src_offset = src.offset; - - let mut offset_offset = dst.num_values * 4; - let offset_buf = &mut dst.value_buffer.as_slice_mut(); - let mut offset_value = read_num_bytes!(i32, 4, &offset_buf[offset_offset..]); - offset_offset += 4; - - // The actual content of a byte array is stored contiguously in the child vector - let child = &mut dst.children[0]; - let mut value_offset = child.num_values; // num_values == num of bytes - - (0..num).for_each(|_| { - let len = read_num_bytes!(i32, 4, &src_data[src_offset..]) as usize; - offset_value += len as i32; - - // Copy offset for the current string value into the offset buffer - bit::memcpy_value(&offset_value, 4, &mut offset_buf[offset_offset..]); - - // Reserve additional space in child value buffer if not enough - let value_buf_len = child.value_buffer.len(); - - if unlikely(value_buf_len < value_offset + len) { - let new_capacity = ::std::cmp::max(value_offset + len, value_buf_len * 2); - debug!("Reserving additional space ({} -> {} bytes) for value buffer", - value_buf_len, new_capacity); - child.value_buffer.resize(new_capacity); - } - - // Copy the actual string content into the value buffer - src_offset += mem::size_of::(); - bit::memcpy( - &src_data[src_offset..src_offset + len], - &mut child.value_buffer.as_slice_mut()[value_offset..], - ); - - value_offset += len; - src_offset += len; - offset_offset += 4; - }); - - src.offset = src_offset; - child.num_values = value_offset; - } - - fn skip(src: &mut PlainDecoderInner, num: usize) { - let src_data = &src.data; - let mut src_offset = src.offset; - - (0..num).for_each(|_| { - let len = read_num_bytes!(i32, 4, &src_data[src_offset..]) as usize; - src_offset += mem::size_of::(); - src_offset += len; - }); - - src.offset = src_offset; - } - } - )* - }; -} - -make_plain_binary_impl! { ByteArrayType, StringType } - -macro_rules! make_plain_dict_binary_impl { - ($($ty: ident), *) => { - $( - impl PlainDictDecoding for $ty { - #[inline] - fn decode_dict_one( - idx: usize, - val_idx: usize, - src: &ParquetMutableVector, - dst: &mut ParquetMutableVector, - _: usize, - ) { - debug_assert!(src.children.len() == 1); - debug_assert!(dst.children.len() == 1); - - let src_child = &src.children[0]; - let dst_child = &mut dst.children[0]; - - // get the offset & data for the binary value at index `val_idx` - let mut start_slice = &src.value_buffer[val_idx * 4..]; - let start = start_slice.get_u32_le() as usize; - let mut end_slice = &src.value_buffer[(val_idx + 1) * 4..]; - let end = end_slice.get_u32_le() as usize; - - debug_assert!(end >= start); - - let len = end - start; - let curr_offset = read_num_bytes!(u32, 4, &dst.value_buffer[idx * 4..]) as usize; - - // Reserve additional space in child value buffer if not enough - let value_buf_len = dst_child.value_buffer.len(); - - if unlikely(value_buf_len < curr_offset + len) { - let new_capacity = ::std::cmp::max(curr_offset + len, value_buf_len * 2); - debug!("Reserving additional space ({} -> {} bytes) for value buffer \ - during dictionary fallback", value_buf_len, - new_capacity); - dst_child.value_buffer.resize(new_capacity); - } - - bit::memcpy( - &src_child.value_buffer[start..end], - &mut dst_child.value_buffer[curr_offset..], - ); - - bit::memcpy_value( - &((curr_offset + len) as u32), - 4, - &mut dst.value_buffer[(idx + 1) * 4..], - ); - - dst_child.num_values += len; - } - } - )* - }; -} - -make_plain_dict_binary_impl! { ByteArrayType, StringType } - -macro_rules! make_plain_decimal_int_impl { - ($($ty: ident; $dst_type:ty), *) => { - $( - impl PlainDecoding for $ty { - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let num_bytes = std::mem::size_of::<$dst_type>(); - let byte_width = src.desc.type_length() as usize; - let num_bits = num_bytes.saturating_sub(byte_width) * 8; - - let src_data = &src.data[src.offset..]; - let dst_data = &mut dst.value_buffer[dst.num_values * num_bytes..]; - - let mut src_offset = 0; - - let src_precision = src.desc.type_precision() as u32; - let src_scale = ::std::cmp::max(src.desc.type_scale(), 0) as u32; - let (dst_precision, dst_scale) = match dst.arrow_type { - ArrowDataType::Decimal128(p, s) if s >= 0 => (p as u32, s as u32), - _ => (src_precision, src_scale), - }; - let upper = (10 as $dst_type).pow(dst_precision); - let mul_div = (10 as $dst_type).pow(dst_scale.abs_diff(src_scale)); - - for i in 0..num { - let mut unscaled: $dst_type = 0; - for _ in 0..byte_width { - unscaled = unscaled << 8 | src_data[src_offset] as $dst_type; - src_offset += 1; - } - unscaled = (unscaled << num_bits) >> num_bits; - if dst_scale > src_scale { - unscaled *= mul_div; - } else if dst_scale < src_scale { - unscaled /= mul_div; - } - bit::memcpy_value(&unscaled, num_bytes, &mut dst_data[i * num_bytes..]); - if src_precision > dst_precision { - write_null!(unscaled, upper, dst, i); - } - } - - src.offset += num * byte_width; - } - - fn skip(src: &mut PlainDecoderInner, num: usize) { - let num_bytes_to_skip = num * src.desc.type_length() as usize; - src.offset += num_bytes_to_skip; - } - } - - impl PlainDictDecoding for $ty { - #[inline] - fn decode_dict_one(_: usize, val_idx: usize, src: &ParquetMutableVector, dst: &mut ParquetMutableVector, _: usize) { - let num_bytes = std::mem::size_of::<$dst_type>(); - bit::memcpy( - &src.value_buffer[val_idx * num_bytes..(val_idx + 1) * num_bytes], - &mut dst.value_buffer[dst.num_values * num_bytes..], - ); - } - } - )* - }; -} - -make_plain_decimal_int_impl!(FLBADecimal32Type; i32, FLBADecimal64Type; i64, FLBADecimalType; i128); - -// Int96 contains 12 bytes -const INT96_SRC_BYTE_WIDTH: usize = 12; -// We convert INT96 to micros and store using i64 -const INT96_DST_BYTE_WIDTH: usize = 8; - -fn int96_to_microsecond(v: &[u8]) -> i64 { - let nanos = &v[..INT96_DST_BYTE_WIDTH] as *const [u8] as *const u8 as *const i64; - let day = &v[INT96_DST_BYTE_WIDTH..] as *const [u8] as *const u8 as *const i32; - - unsafe { - ((day.read_unaligned() - JULIAN_DAY_OF_EPOCH) as i64) - .wrapping_mul(MICROS_PER_DAY) - .wrapping_add(nanos.read_unaligned() / 1000) - } -} - -/// Decode timestamps represented as INT96 into i64 with micros precision -impl PlainDecoding for Int96TimestampMicrosType { - #[inline] - fn decode(src: &mut PlainDecoderInner, dst: &mut ParquetMutableVector, num: usize) { - let src_data = &src.data; - - if !src.read_options.use_legacy_date_timestamp_or_ntz { - let mut offset = src.offset; - for _ in 0..num { - // TODO: optimize this further as checking value one by one is not very efficient - let micros = int96_to_microsecond(&src_data[offset..offset + INT96_SRC_BYTE_WIDTH]); - - if unlikely(micros < JULIAN_GREGORIAN_SWITCH_OFF_TS) { - panic!( - "Encountered timestamp value {micros}, which is before 1582-10-15 (counting \ - backwards from Unix epoch date 1970-01-01), and could be ambigous \ - depending on whether a legacy Julian/Gregorian hybrid calendar is used, \ - or a Proleptic Gregorian calendar is used." - ); - } - - offset += INT96_SRC_BYTE_WIDTH; - } - } - - let mut offset = src.offset; - let mut dst_offset = INT96_DST_BYTE_WIDTH * dst.num_values; - for _ in 0..num { - let micros = int96_to_microsecond(&src_data[offset..offset + INT96_SRC_BYTE_WIDTH]); - - bit::memcpy_value( - µs, - INT96_DST_BYTE_WIDTH, - &mut dst.value_buffer[dst_offset..], - ); - - offset += INT96_SRC_BYTE_WIDTH; - dst_offset += INT96_DST_BYTE_WIDTH; - } - - src.offset = offset; - } - - #[inline] - fn skip(src: &mut PlainDecoderInner, num: usize) { - src.offset += INT96_SRC_BYTE_WIDTH * num; - } -} - -impl PlainDictDecoding for Int96TimestampMicrosType { - fn decode_dict_one( - _: usize, - val_idx: usize, - src: &ParquetMutableVector, - dst: &mut ParquetMutableVector, - _: usize, - ) { - let src_offset = val_idx * INT96_DST_BYTE_WIDTH; - let dst_offset = dst.num_values * INT96_DST_BYTE_WIDTH; - - bit::memcpy( - &src.value_buffer[src_offset..src_offset + INT96_DST_BYTE_WIDTH], - &mut dst.value_buffer[dst_offset..dst_offset + INT96_DST_BYTE_WIDTH], - ); - } -} - -impl Decoder for PlainDecoder { - #[inline] - fn read(&mut self, dst: &mut ParquetMutableVector) { - self.read_batch(dst, 1) - } - - /// Default implementation for PLAIN encoding, which uses a `memcpy` to copy from Parquet to the - /// Arrow vector. NOTE: this only works if the Parquet physical type has the same type width as - /// the Arrow's physical type (e.g., Parquet INT32 vs Arrow INT32). For other cases, we should - /// have special implementations. - #[inline] - fn read_batch(&mut self, dst: &mut ParquetMutableVector, num: usize) { - T::decode(&mut self.inner, dst, num); - } - - #[inline] - fn skip_batch(&mut self, num: usize) { - T::skip(&mut self.inner, num); - } - - #[inline] - fn encoding(&self) -> Encoding { - Encoding::PLAIN - } -} - -/// A decoder for Parquet dictionary indices, which is always of integer type, and encoded with -/// RLE/BitPacked encoding. -pub struct DictDecoder { - /// Number of bits used to represent dictionary indices. Must be between `[0, 64]`. - bit_width: usize, - - /// Bit reader - bit_reader: BitReader, - - /// Number of values left in the current RLE run - rle_left: usize, - - /// Number of values left in the current BIT_PACKED run - bit_packed_left: usize, - - /// Current value in the RLE run. Unused if BIT_PACKED - current_value: u32, -} - -impl DictDecoder { - pub fn new(buf: Buffer) -> Self { - let bit_width = buf.as_bytes()[0] as usize; - - Self { - bit_width, - bit_reader: BitReader::new_all(buf.slice(1)), - rle_left: 0, - bit_packed_left: 0, - current_value: 0, - } - } -} - -impl DictDecoder { - /// Reads the header of the next RLE/BitPacked run, and update the internal state such as # of - /// values for the next run, as well as the current value in case it's RLE. - fn reload(&mut self) { - if let Some(indicator_value) = self.bit_reader.get_vlq_int() { - if indicator_value & 1 == 1 { - self.bit_packed_left = ((indicator_value >> 1) * 8) as usize; - } else { - self.rle_left = (indicator_value >> 1) as usize; - let value_width = self.bit_width.div_ceil(8); - self.current_value = self.bit_reader.get_aligned::(value_width).unwrap(); - } - } else { - panic!("Can't read VLQ int from BitReader"); - } - } -} - -impl Decoder for DictDecoder { - fn read(&mut self, dst: &mut ParquetMutableVector) { - let dst_slice = dst.value_buffer.as_slice_mut(); - let dst_offset = dst.num_values * 4; - - // We've finished the current run. Now load the next. - if self.rle_left == 0 && self.bit_packed_left == 0 { - self.reload(); - } - - let value = if self.rle_left > 0 { - self.rle_left -= 1; - self.current_value - } else { - self.bit_packed_left -= 1; - self.bit_reader.get_u32_value(self.bit_width) - }; - - // Directly copy the value into the destination buffer - unsafe { - let dst = &mut dst_slice[dst_offset..] as *mut [u8] as *mut u8 as *mut u32; - *dst = value; - } - } - - fn read_batch(&mut self, dst: &mut ParquetMutableVector, num: usize) { - let mut values_read = 0; - let dst_slice = dst.value_buffer.as_slice_mut(); - let mut dst_offset = dst.num_values * 4; - - while values_read < num { - let num_to_read = num - values_read; - let mut dst = &mut dst_slice[dst_offset..] as *mut [u8] as *mut u8 as *mut u32; - - if self.rle_left > 0 { - let n = std::cmp::min(num_to_read, self.rle_left); - unsafe { - // Copy the current RLE value into the destination buffer. - for _ in 0..n { - *dst = self.current_value; - dst = dst.offset(1); - } - dst_offset += 4 * n; - } - self.rle_left -= n; - values_read += n; - } else if self.bit_packed_left > 0 { - let n = std::cmp::min(num_to_read, self.bit_packed_left); - unsafe { - // Decode the next `n` BitPacked values into u32 and put the result directly to - // `dst`. - self.bit_reader.get_u32_batch(dst, n, self.bit_width); - } - dst_offset += 4 * n; - self.bit_packed_left -= n; - values_read += n; - } else { - self.reload(); - } - } - } - - fn skip_batch(&mut self, num: usize) { - let mut values_skipped = 0; - - while values_skipped < num { - let num_to_skip = num - values_skipped; - - if self.rle_left > 0 { - let n = std::cmp::min(num_to_skip, self.rle_left); - self.rle_left -= n; - values_skipped += n; - } else if self.bit_packed_left > 0 { - let n = std::cmp::min(num_to_skip, self.bit_packed_left); - self.bit_reader.skip_bits(n * self.bit_width); - self.bit_packed_left -= n; - values_skipped += n; - } else { - self.reload(); - } - } - } - - fn encoding(&self) -> Encoding { - Encoding::RLE_DICTIONARY - } -} - -#[cfg(test)] -mod test { - use super::*; - use parquet::data_type::AsBytes; - - #[test] - fn test_i32_to_i8() { - let source = - hex::decode("8a000000dbffffff1800000034ffffff300000001d000000abffffff37fffffff1000000") - .unwrap(); - let expected = hex::decode("8adb1834301dab37f1").unwrap(); - let num = source.len() / 4; - let mut dest: Vec = vec![b' '; num]; - copy_i32_to_i8(source.as_bytes(), dest.as_mut_slice(), num); - assert_eq!(expected.as_bytes(), dest.as_bytes()); - } - - #[test] - fn test_i32_to_u8() { - let source = - hex::decode("8a000000dbffffff1800000034ffffff300000001d000000abffffff37fffffff1000000") - .unwrap(); - let expected = hex::decode("8a00dbff180034ff30001d00abff37fff100").unwrap(); - let num = source.len() / 4; - let mut dest: Vec = vec![b' '; num * 2]; - copy_i32_to_u8(source.as_bytes(), dest.as_mut_slice(), num); - assert_eq!(expected.as_bytes(), dest.as_bytes()); - } - - #[test] - fn test_i32_to_i16() { - let source = - hex::decode("8a0e0000db93ffff1826000034f4ffff300200001d2b0000abe3ffff378dfffff1470000") - .unwrap(); - let expected = hex::decode("8a0edb93182634f430021d2babe3378df147").unwrap(); - let num = source.len() / 4; - let mut dest: Vec = vec![b' '; num * 2]; - copy_i32_to_i16(source.as_bytes(), dest.as_mut_slice(), num); - assert_eq!(expected.as_bytes(), dest.as_bytes()); - } - - #[test] - fn test_i32_to_u16() { - let source = hex::decode( - "ff7f0000008000000180000002800000038000000480000005800000068000000780000008800000", - ) - .unwrap(); - let expected = hex::decode( - "ff7f0000008000000180000002800000038000000480000005800000068000000780000008800000", - ) - .unwrap(); - let num = source.len() / 4; - let mut dest: Vec = vec![b' '; num * 4]; - copy_i32_to_u16(source.as_bytes(), dest.as_mut_slice(), num); - assert_eq!(expected.as_bytes(), dest.as_bytes()); - } - - #[test] - fn test_i32_to_u32() { - let source = hex::decode( - "ffffff7f000000800100008002000080030000800400008005000080060000800700008008000080", - ) - .unwrap(); - let expected = hex::decode("ffffff7f00000000000000800000000001000080000000000200008000000000030000800000000004000080000000000500008000000000060000800000000007000080000000000800008000000000").unwrap(); - let num = source.len() / 4; - let mut dest: Vec = vec![b' '; num * 8]; - copy_i32_to_u32(source.as_bytes(), dest.as_mut_slice(), num); - assert_eq!(expected.as_bytes(), dest.as_bytes()); - } -} diff --git a/native/core/src/parquet/schema_adapter.rs b/native/core/src/parquet/schema_adapter.rs index 40cea642ac..09d5a1dd62 100644 --- a/native/core/src/parquet/schema_adapter.rs +++ b/native/core/src/parquet/schema_adapter.rs @@ -17,7 +17,9 @@ use crate::parquet::cast_column::CometCastColumnExpr; use crate::parquet::parquet_support::{spark_parquet_convert, SparkParquetOptions}; +use arrow::array::new_empty_array; use arrow::datatypes::{DataType, Field, FieldRef, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; use datafusion::common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion::common::{DataFusionError, Result as DataFusionResult}; use datafusion::physical_expr::expressions::Column; @@ -32,6 +34,8 @@ use datafusion_physical_expr_adapter::{ }; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use std::collections::HashMap; +use std::fmt::{self, Display}; +use std::hash::{Hash, Hasher}; use std::sync::Arc; /// Factory for creating Spark-compatible physical expression adapters. @@ -232,6 +236,102 @@ fn remap_physical_schema( Ok((Arc::new(Schema::new(remapped_fields)), name_map)) } +/// Format an Arrow `DataType` as Spark's catalog string (e.g. `Int64` -> `bigint`), +/// so SchemaColumnConvertNotSupportedException messages match Spark's vectorized reader. +fn spark_catalog_name(dt: &DataType) -> String { + match dt { + DataType::Boolean => "boolean".to_string(), + DataType::Int8 => "tinyint".to_string(), + DataType::Int16 => "smallint".to_string(), + DataType::Int32 => "int".to_string(), + DataType::Int64 => "bigint".to_string(), + DataType::Float32 => "float".to_string(), + DataType::Float64 => "double".to_string(), + DataType::Utf8 | DataType::LargeUtf8 => "string".to_string(), + DataType::Binary | DataType::LargeBinary => "binary".to_string(), + DataType::Date32 => "date".to_string(), + DataType::Timestamp(_, Some(_)) => "timestamp".to_string(), + DataType::Timestamp(_, None) => "timestamp_ntz".to_string(), + DataType::Decimal128(p, s) | DataType::Decimal256(p, s) => { + format!("decimal({p},{s})") + } + _ => "unknown".to_string(), + } +} + +/// Format an Arrow `DataType` as the Parquet primitive type name +/// (e.g. `Int64` -> `INT64`, matching `PrimitiveTypeName.toString()` in parquet-mr). +fn parquet_primitive_name(dt: &DataType) -> &'static str { + match dt { + DataType::Boolean => "BOOLEAN", + DataType::Int8 | DataType::Int16 | DataType::Int32 => "INT32", + DataType::Int64 => "INT64", + DataType::Float32 => "FLOAT", + DataType::Float64 => "DOUBLE", + DataType::Utf8 | DataType::LargeUtf8 | DataType::Binary | DataType::LargeBinary => "BINARY", + // Spark stores DATE as INT32 with a DATE logical-type annotation. + DataType::Date32 => "INT32", + // Spark stores TIMESTAMP as INT64 with a timestamp annotation, or as + // INT96 (legacy nanos). arrow-rs surfaces both as `Timestamp`; without + // the original physical name we report INT64, which matches the + // common case. + DataType::Timestamp(_, _) => "INT64", + // Mirror Spark's `SparkToParquetSchemaConverter` decimal mapping: + // precision 1-9 -> INT32, 10-18 -> INT64, 19+ -> FIXED_LEN_BYTE_ARRAY. + DataType::Decimal128(p, _) | DataType::Decimal256(p, _) => { + if *p <= 9 { + "INT32" + } else if *p <= 18 { + "INT64" + } else { + "FIXED_LEN_BYTE_ARRAY" + } + } + _ => "UNKNOWN", + } +} + +fn is_string_or_binary(dt: &DataType) -> bool { + matches!( + dt, + DataType::Utf8 | DataType::LargeUtf8 | DataType::Binary | DataType::LargeBinary + ) +} + +/// Build a Spark-shaped `SchemaColumnConvertNotSupportedException` carrier for a +/// rejected Parquet -> Spark conversion. The bracketed column wrapping mirrors +/// `Arrays.toString(descriptor.getPath())` in Spark's vectorized reader. +fn parquet_schema_convert_err( + field_name: &str, + physical_type: &DataType, + target_type: &DataType, +) -> DataFusionError { + DataFusionError::External(Box::new(SparkError::ParquetSchemaConvert { + file_path: String::new(), + column: format!("[{}]", field_name), + physical_type: parquet_primitive_name(physical_type).to_string(), + spark_type: spark_catalog_name(target_type), + })) +} + +/// Build a `RejectOnNonEmpty` expr wrapping `child`. The rejection fires only +/// when the input batch is non-empty (mirrors Spark's per-row-group check). +fn reject_on_non_empty_expr( + child: Arc, + target_field: &FieldRef, + field_name: &str, + physical_type: &DataType, + target_type: &DataType, +) -> Arc { + Arc::new(RejectOnNonEmpty { + child, + target_field: Arc::clone(target_field), + column: format!("[{}]", field_name), + physical_type: parquet_primitive_name(physical_type).to_string(), + spark_type: spark_catalog_name(target_type), + }) +} + /// Check if a specific column name has duplicate matches in the physical schema /// (case-insensitive). Returns the error info if so. fn check_column_duplicate(col_name: &str, physical_schema: &SchemaRef) -> Option<(String, String)> { @@ -353,7 +453,7 @@ impl PhysicalExprAdapter for SparkPhysicalExprAdapter { // Walk the expression tree to find Column references let mut duplicate_err: Option = None; let _ = Arc::::clone(&expr).transform(|e| { - if let Some(col) = e.as_any().downcast_ref::() { + if let Some(col) = e.downcast_ref::() { if let Some((req, matched)) = check_column_duplicate(col.name(), orig_physical) { duplicate_err = Some(DataFusionError::External(Box::new( @@ -403,7 +503,7 @@ impl PhysicalExprAdapter for SparkPhysicalExprAdapter { // the actual parquet stream schema, which uses the original physical names. let expr = if let Some(name_map) = &self.logical_to_physical_names { expr.transform(|e| { - if let Some(col) = e.as_any().downcast_ref::() { + if let Some(col) = e.downcast_ref::() { if let Some(physical_name) = name_map.get(col.name()) { return Ok(Transformed::yes(Arc::new(Column::new( physical_name, @@ -432,7 +532,7 @@ impl SparkPhysicalExprAdapter { expr: Arc, ) -> DataFusionResult> { expr.transform(|e| { - if let Some(column) = e.as_any().downcast_ref::() { + if let Some(column) = e.downcast_ref::() { let col_name = column.name(); // Resolve fields by name because this is the fallback path @@ -487,6 +587,21 @@ impl SparkPhysicalExprAdapter { }; if logical_field.data_type() != physical_field.data_type() { + // Mirror the same string/binary -> non-string/binary rejection in + // `replace_with_spark_cast`; this branch is reached when the default + // adapter rejected the cast and we'd otherwise build a CometCastColumnExpr + // that can't actually convert (e.g. BINARY -> DECIMAL with no + // `DecimalLogicalTypeAnnotation`). See #4088 and #4351. + let physical_type = physical_field.data_type(); + let target_type = logical_field.data_type(); + if is_string_or_binary(physical_type) && !is_string_or_binary(target_type) { + return Err(parquet_schema_convert_err( + physical_field.name(), + physical_type, + target_type, + )); + } + let cast_expr: Arc = Arc::new( CometCastColumnExpr::new( remapped, @@ -512,111 +627,188 @@ impl SparkPhysicalExprAdapter { &self, expr: Arc, ) -> DataFusionResult>> { - // Check for CastColumnExpr and replace with spark_expr::Cast - // CastColumnExpr is in datafusion_physical_expr::expressions - if let Some(cast) = expr - .as_any() - .downcast_ref::() + // Check for CastExpr and replace with spark_expr::Cast + if let Some(cast) = expr.downcast_ref::() { let child = Arc::clone(cast.expr()); - let physical_type = cast.input_field().data_type(); + let input_field = child.return_field(&self.physical_file_schema)?; + let physical_type = input_field.data_type(); let target_type = cast.target_field().data_type(); - // Reject reading a string/binary Parquet column as anything other - // than string, binary, or a binary-encoded decimal. This mirrors - // Spark's TypeUtil.checkParquetType for the BINARY case (lines - // 208-221): a BINARY (or UTF8-annotated BINARY) physical column is - // only readable as StringType, BinaryType, or a binary-encoded - // decimal; every other target type (numeric, boolean, date, - // timestamp, ...) raises SchemaColumnConvertNotSupportedException. - // - // Without this guard, Spark's Cast below (in is_adapting_schema - // mode) falls through to DataFusion's cast, which silently parses - // the bytes (returning nulls for non-numeric strings, parsing - // date/timestamp/boolean strings, or in some paths reinterpreting - // raw bytes). See issue #4088. - if matches!( + // Reject reading a string/binary Parquet column as anything else. Spark's + // `ParquetVectorUpdaterFactory.getUpdater` BINARY case allows StringType / + // BinaryType, or DecimalType only when the column carries a + // `DecimalLogicalTypeAnnotation` (which arrow-rs surfaces as `Decimal128`, + // not `Binary`). Without this guard, runtime cast paths silently return + // nulls, parse strings, or surface as a generic Arrow type-mismatch error. + // See #4088 and #4351. + if is_string_or_binary(physical_type) && !is_string_or_binary(target_type) { + return Err(parquet_schema_convert_err( + input_field.name(), + physical_type, + target_type, + )); + } + + // Reject reading a primitive numeric Parquet column as StringType / + // BinaryType. Spark has no `int -> string` etc. updater. Defer to + // runtime via `RejectOnNonEmpty` so empty Parquet files (SPARK-26709) + // pass and the JVM shim translates to + // `SchemaColumnConvertNotSupportedException`. + let physical_is_primitive_numeric = matches!( physical_type, - DataType::Utf8 | DataType::LargeUtf8 | DataType::Binary | DataType::LargeBinary - ) && !matches!( - target_type, - DataType::Utf8 - | DataType::LargeUtf8 - | DataType::Binary - | DataType::LargeBinary - | DataType::Decimal128(_, _) - | DataType::Decimal256(_, _) - ) { - return Err(DataFusionError::External(Box::new( - SparkError::ParquetSchemaConvert { - file_path: String::new(), - column: cast.input_field().name().to_string(), - physical_type: physical_type.to_string(), - spark_type: target_type.to_string(), - }, - ))); + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + ); + if physical_is_primitive_numeric && is_string_or_binary(target_type) { + let rejection = reject_on_non_empty_expr( + child, + cast.target_field(), + input_field.name(), + physical_type, + target_type, + ); + return Ok(Transformed::yes(rejection)); } - // Decimal-to-decimal scale-narrowing check. - // Reject reads where the read schema has a smaller scale than the - // file's, because Spark's Cast below would silently truncate - // fractional digits, producing wrong values. This matches the - // unconditionally-lossy case in issue #4089 (e.g. Decimal(10,2) read - // as Decimal(5,0)). - // - // Other decimal mismatches are intentionally NOT rejected here, - // even though Spark's vectorized reader would reject them via - // `ParquetVectorUpdaterFactory#isDecimalTypeMatched` (which requires - // exact precision and scale): - // - // - Precision-only changes with the same scale (e.g. Decimal(5,2) - // read as Decimal(3,2)): Spark 4.0's parquet-mr fallback path - // (PARQUET_VECTORIZED_READER_ENABLED=false) and the vectorized - // type-widening path produce null on per-value overflow, which - // DataFusion's cast already does in the adapting-schema path. - // - // - Scale widening (e.g. Decimal(10,2) read as Decimal(10,4)): the - // cast is lossless (no truncation, no overflow), so allowing it - // here is strictly more permissive than Spark's vectorized reader - // without risking wrong values. - if let (DataType::Decimal128(_src_p, src_s), DataType::Decimal128(_dst_p, dst_s)) = + // Decimal-to-decimal narrowing. Spark's `isDecimalTypeMatched` (the + // `DecimalLogicalTypeAnnotation` branch) allows the read only when + // `dst_scale >= src_scale` AND + // `dst_precision - dst_scale >= src_precision - src_scale`. + // Either failure means silently dropping fractional digits or losing + // integer-side magnitude. See #4089 and #4343. + if let (DataType::Decimal128(src_p, src_s), DataType::Decimal128(dst_p, dst_s)) = (physical_type, target_type) { - if dst_s < src_s { - return Err(DataFusionError::External(Box::new( - SparkError::ParquetSchemaConvert { - file_path: String::new(), - column: cast.input_field().name().to_string(), - physical_type: physical_type.to_string(), - spark_type: target_type.to_string(), - }, - ))); + let src_int_precision = i32::from(*src_p) - i32::from(*src_s); + let dst_int_precision = i32::from(*dst_p) - i32::from(*dst_s); + if dst_s < src_s || dst_int_precision < src_int_precision { + return Err(parquet_schema_convert_err( + input_field.name(), + physical_type, + target_type, + )); } } - // For complex nested types (Struct, List, Map), Timestamp timezone - // mismatches, and Timestamp→Int64 (nanosAsLong), use CometCastColumnExpr - // with spark_parquet_convert which handles field-name-based selection, - // reordering, nested type casting, metadata-only timestamp timezone - // relabeling, and raw value reinterpretation correctly. + // Integer-to-decimal narrowing. Spark's `canReadAsDecimal` requires + // `precision - scale >= 10` for an INT32 source and `>= 20` for INT64. + // Unconditional in all Spark versions, so reject at plan time. See #4344. + let int_decimal_min_int_precision = match physical_type { + DataType::Int8 | DataType::Int16 | DataType::Int32 => Some(10i32), + DataType::Int64 => Some(20i32), + _ => None, + }; + if let Some(min_int_precision) = int_decimal_min_int_precision { + let dst_precision_scale = match target_type { + DataType::Decimal128(p, s) | DataType::Decimal256(p, s) => Some((*p, *s)), + _ => None, + }; + if let Some((dst_p, dst_s)) = dst_precision_scale { + let dst_int_precision = i32::from(dst_p) - i32::from(dst_s); + if dst_int_precision < min_int_precision { + return Err(parquet_schema_convert_err( + input_field.name(), + physical_type, + target_type, + )); + } + } + } + + // Type promotion (widening). When `allow_type_promotion` is false, + // reject the three widenings (INT32→INT64, FLOAT→DOUBLE, INT32→DOUBLE) + // that Spark 3.x's vectorized reader rejects. The flag tracks Comet's + // per-Spark-version constant in ShimCometConf. Deferred to runtime so + // empty files (SPARK-26709) pass. + if !self.parquet_options.allow_type_promotion { + let is_disallowed_promotion = matches!( + (physical_type, target_type), + (DataType::Int32, DataType::Int64) + | (DataType::Float32, DataType::Float64) + | (DataType::Int32, DataType::Float64) + ); + if is_disallowed_promotion { + let rejection = reject_on_non_empty_expr( + Arc::clone(&child), + cast.target_field(), + input_field.name(), + physical_type, + target_type, + ); + return Ok(Transformed::yes(rejection)); + } + } + + // Reject primitive Parquet conversions Spark's vectorized reader rejects + // on every supported version (no matching branch in + // `ParquetVectorUpdaterFactory.getUpdater`): // - // Timestamp mismatches (e.g., Timestamp(us, None) -> Timestamp(us, Some("UTC"))) - // occur when INT96 Parquet timestamps are coerced to Timestamp(us, None) by - // DataFusion but the logical schema expects Timestamp(us, Some("UTC")). - // Using Spark's Cast here would incorrectly treat the None-timezone values as - // local time (TimestampNTZ) and apply a timezone conversion, but the values are - // already in UTC. spark_parquet_convert handles this as a metadata-only change. + // - `INT64 -> Int*` truncates lower bits. + // - `INT64 -> Float*` and `INT32 -> Float32` lose precision. + // - `Float* -> Int*` and `Float64 -> Float32` truncate / overflow. + // - `INT32 -> Timestamp` / `INT64 -> Date32` / `INT64 -> Timestamp`: + // date/timestamp-annotated columns surface as Date32 / Timestamp, + // so reaching this branch means the column was un-annotated. + // - `Date32 -> Timestamp(LTZ)`: Spark only allows Date -> TimestampNTZ. + // - `Timestamp -> Date32`: no Timestamp updater branches into Date. // - // Timestamp→Int64 occurs when Spark's `nanosAsLong` config converts - // TIMESTAMP(NANOS) to LongType. Spark's Cast would divide by MICROS_PER_SECOND - // (assuming microseconds), but the values are nanoseconds. Arrow cast correctly - // reinterprets the raw i64 value without conversion. - // Reject scalar/complex mismatches at planning time. Spark's - // vectorized reader rejects e.g. reading a TIMESTAMP column as - // ARRAY with SchemaColumnConvertNotSupportedException - // (see SPARK-45604). Without this guard the runtime cast would - // raise a less-specific Arrow CastError. Same-complex-type pairs - // and timestamp→timestamp / timestamp→int64 are handled below. + // Deferred to runtime (SPARK-26709). See #4297. + let is_spark_rejected_conversion = matches!( + (physical_type, target_type), + // Long -> narrower int. + ( + DataType::Int64, + DataType::Int8 | DataType::Int16 | DataType::Int32, + ) + // Long -> floating point. + | (DataType::Int64, DataType::Float32 | DataType::Float64) + // Long -> date / timestamp (raw INT64; annotated columns surface as Date32/Timestamp). + | (DataType::Int64, DataType::Date32) + | (DataType::Int64, DataType::Timestamp(_, _)) + // Int -> float (DoubleType is allowed via IntegerToDoubleUpdater; FloatType is not). + | ( + DataType::Int8 | DataType::Int16 | DataType::Int32, + DataType::Float32, + ) + // Int -> timestamp (raw INT32; DATE-annotated columns surface as Date32). + | ( + DataType::Int8 | DataType::Int16 | DataType::Int32, + DataType::Timestamp(_, _), + ) + // Float -> int / Double -> int (no integer branches under FLOAT/DOUBLE). + | ( + DataType::Float32 | DataType::Float64, + DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, + ) + // Double -> float (narrowing). + | (DataType::Float64, DataType::Float32) + // Date -> Timestamp(LTZ). Spark allows Date -> TimestampNTZ only. + | (DataType::Date32, DataType::Timestamp(_, Some(_))) + // Timestamp -> Date. + | (DataType::Timestamp(_, _), DataType::Date32) + ); + if is_spark_rejected_conversion { + let rejection = reject_on_non_empty_expr( + child, + cast.target_field(), + input_field.name(), + physical_type, + target_type, + ); + return Ok(Transformed::yes(rejection)); + } + + // Scalar/complex mismatch (e.g. TIMESTAMP read as ARRAY): + // Spark's vectorized reader rejects with + // SchemaColumnConvertNotSupportedException (SPARK-45604). Same-shape + // complex pairs and timestamp→timestamp / timestamp→int64 fall through + // to CometCastColumnExpr below. let is_complex = |t: &DataType| { matches!( t, @@ -624,16 +816,18 @@ impl SparkPhysicalExprAdapter { ) }; if is_complex(physical_type) != is_complex(target_type) { - return Err(DataFusionError::External(Box::new( - SparkError::ParquetSchemaConvert { - file_path: String::new(), - column: cast.input_field().name().to_string(), - physical_type: physical_type.to_string(), - spark_type: target_type.to_string(), - }, - ))); + return Err(parquet_schema_convert_err( + input_field.name(), + physical_type, + target_type, + )); } + // Same-shape complex casts, timestamp tz relabel (e.g. Timestamp(us, None) + // -> Timestamp(us, Some("UTC")) for INT96 reads), and Timestamp -> Int64 + // (Spark's `nanosAsLong`) need spark_parquet_convert: it handles nested + // field selection, metadata-only tz changes, and raw-value reinterpretation + // that Spark's Cast would otherwise convert incorrectly. if matches!( (physical_type, target_type), (DataType::Struct(_), DataType::Struct(_)) @@ -645,7 +839,7 @@ impl SparkPhysicalExprAdapter { let comet_cast: Arc = Arc::new( CometCastColumnExpr::new( child, - Arc::clone(cast.input_field()), + Arc::clone(&input_field), Arc::clone(cast.target_field()), None, ) @@ -750,12 +944,115 @@ impl SparkPhysicalExprAdapter { } } +/// Defers a Parquet type-promotion rejection to runtime: returns an empty array +/// when the input batch has no rows, and raises `ParquetSchemaConvert` otherwise. +/// +/// Mirrors Spark's vectorized reader, which only invokes +/// `ParquetVectorUpdaterFactory.getUpdater` while decoding a row group. A +/// Parquet file with no row groups (e.g. one written from an empty DataFrame) +/// never triggers the per-row-group check, so a partition mixing such a file +/// with another whose schema would otherwise fail the type-promotion check +/// (SPARK-26709) is still readable. +#[derive(Debug, Eq)] +struct RejectOnNonEmpty { + child: Arc, + target_field: FieldRef, + column: String, + physical_type: String, + spark_type: String, +} + +impl PartialEq for RejectOnNonEmpty { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.target_field.eq(&other.target_field) + && self.column == other.column + && self.physical_type == other.physical_type + && self.spark_type == other.spark_type + } +} + +impl Hash for RejectOnNonEmpty { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.target_field.hash(state); + self.column.hash(state); + self.physical_type.hash(state); + self.spark_type.hash(state); + } +} + +impl Display for RejectOnNonEmpty { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "REJECT_PARQUET_TYPE_PROMOTION({} AS {})", + self.column, self.spark_type + ) + } +} + +impl PhysicalExpr for RejectOnNonEmpty { + fn data_type(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(self.target_field.data_type().clone()) + } + + fn nullable(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(self.target_field.is_nullable()) + } + + fn evaluate(&self, batch: &RecordBatch) -> DataFusionResult { + if batch.num_rows() == 0 { + return Ok(ColumnarValue::Array(new_empty_array( + self.target_field.data_type(), + ))); + } + Err(DataFusionError::External(Box::new( + SparkError::ParquetSchemaConvert { + file_path: String::new(), + column: self.column.clone(), + physical_type: self.physical_type.clone(), + spark_type: self.spark_type.clone(), + }, + ))) + } + + fn return_field(&self, _input_schema: &Schema) -> DataFusionResult { + Ok(Arc::clone(&self.target_field)) + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.child] + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> DataFusionResult> { + assert_eq!(children.len(), 1); + Ok(Arc::new(RejectOnNonEmpty { + child: children.pop().expect("child"), + target_field: Arc::clone(&self.target_field), + column: self.column.clone(), + physical_type: self.physical_type.clone(), + spark_type: self.spark_type.clone(), + })) + } + + fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + Display::fmt(self, f) + } +} + #[cfg(test)] mod test { use crate::parquet::parquet_support::SparkParquetOptions; use crate::parquet::schema_adapter::SparkPhysicalExprAdapterFactory; use arrow::array::UInt32Array; - use arrow::array::{Int32Array, StringArray}; + use arrow::array::{ + BinaryArray, Date32Array, Decimal128Array, Float32Array, Float64Array, Int32Array, + Int64Array, StringArray, TimestampMicrosecondArray, + }; use arrow::datatypes::SchemaRef; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -774,25 +1071,542 @@ mod test { use std::fs::File; use std::sync::Arc; + /// Reading a non-BINARY Parquet column as `StringType` must raise the same + /// `_LEGACY_ERROR_TEMP_2063`-shaped error as Spark's vectorized reader + /// (`ParquetVectorUpdaterFactory.getUpdater` has no INT32 -> string updater). #[tokio::test] - async fn parquet_roundtrip_int_as_string() -> Result<(), DataFusionError> { - let file_schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, false), - ])); + async fn parquet_int_read_as_string_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int32, false), + values, + DataType::Utf8, + ) + .await?; + assert!( + msg.contains("Column: [[a]]") + && msg.contains("Expected: string") + && msg.contains("Found: INT32"), + "unexpected error: {msg}" + ); + Ok(()) + } - let ids = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; - let names = Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])) - as Arc; - let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![ids, names])?; + /// Companion: BINARY (string physical) read as IntegerType must raise the + /// same Spark-compatible error. + #[tokio::test] + async fn parquet_string_read_as_int_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(StringArray::from(vec!["bcd", "efg"])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Utf8, false), + values, + DataType::Int32, + ) + .await?; + assert!( + msg.contains("Column: [[a]]") + && msg.contains("Expected: int") + && msg.contains("Found: BINARY"), + "unexpected error: {msg}" + ); + Ok(()) + } - let required_schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Utf8, false), - Field::new("name", DataType::Utf8, false), - ])); + /// Reading a plain BINARY Parquet column (no `DecimalLogicalTypeAnnotation`) + /// as `DecimalType` must raise a Spark-compatible `ParquetSchemaConvert` + /// error. Spark's `canReadAsDecimal` / `canReadAsBinaryDecimal` both require + /// the column to carry a `DecimalLogicalTypeAnnotation`. See #4351. + #[tokio::test] + async fn parquet_binary_read_as_decimal_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(BinaryArray::from_vec(vec![b"1.2", b"3.4"])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Binary, false), + values, + DataType::Decimal128(37, 1), + ) + .await?; + assert!( + msg.contains("Column: [[a]]") + && msg.contains("Expected: decimal(37,1)") + && msg.contains("Found: BINARY"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// INT32 -> Decimal where `precision - scale < 10` (the minimum that can + /// represent the full INT32 range). See #4344. + #[tokio::test] + async fn parquet_int32_read_as_narrow_decimal_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int32, false), + values, + DataType::Decimal128(9, 0), + ) + .await?; + assert!( + msg.contains("Column: [[a]]") + && msg.contains("Expected: decimal") + && msg.contains("Found: INT32"), + "unexpected error: {msg}" + ); + Ok(()) + } + /// INT32 -> Decimal where `precision - scale >= 10` is allowed. + #[tokio::test] + async fn parquet_int32_read_as_wide_decimal_succeeds() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let values = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![values])?; + let required_schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Decimal128(10, 0), + false, + )])); let _ = roundtrip(&batch, required_schema).await?; + Ok(()) + } + + /// INT64 -> Decimal where `precision - scale < 20`. See #4344. + #[tokio::test] + async fn parquet_int64_read_as_narrow_decimal_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int64Array::from(vec![1i64, 2, 3])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int64, false), + values, + DataType::Decimal128(19, 0), + ) + .await?; + assert!( + msg.contains("Column: [[a]]") + && msg.contains("Expected: decimal") + && msg.contains("Found: INT64"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// Non-zero scale that pushes `precision - scale` below the integer minimum + /// (INT32 -> Decimal(10, 1) leaves int-precision 9). + #[tokio::test] + async fn parquet_int32_read_as_decimal_with_scale_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int32, false), + values, + DataType::Decimal128(10, 1), + ) + .await?; + assert!( + msg.contains("Column: [[a]]") + && msg.contains("Expected: decimal") + && msg.contains("Found: INT32"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// Helper to build a tiny decimal Parquet batch for the decimal-to-decimal tests. + fn decimal_batch(precision: u8, scale: i8) -> Result { + let file_schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Decimal128(precision, scale), + false, + )])); + let values = Arc::new( + Decimal128Array::from(vec![123i128, 456]) + .with_precision_and_scale(precision, scale) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?, + ) as Arc; + Ok(RecordBatch::try_new(file_schema, vec![values])?) + } + + /// Reading Decimal(P, S) as Decimal(P', S) where P' < P (precision-only + /// narrowing, equal scale) must raise the Spark-compatible error. Spark's + /// `isDecimalTypeMatched` rejects this because `precisionIncrease < 0` + /// while `scaleIncrease == 0`. See #4343. + #[tokio::test] + async fn parquet_decimal_precision_narrowing_errors() -> Result<(), DataFusionError> { + let batch = decimal_batch(10, 2)?; + let required_schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Decimal128(5, 2), + false, + )])); + + let err = roundtrip(&batch, required_schema) + .await + .expect_err("expected ParquetSchemaConvert for Decimal(10, 2) -> Decimal(5, 2)"); + let msg = err.to_string(); + assert!( + msg.contains("Column: [[a]]") && msg.contains("Expected: decimal(5,2)"), + "unexpected error: {msg}" + ); + Ok(()) + } + /// Reading Decimal(P, S) as Decimal(P', S') where the integer-precision + /// `P - S` shrinks must raise the Spark-compatible error. Example: + /// Decimal(10, 4) (int-precision 6) -> Decimal(5, 2) (int-precision 3). + /// See #4343. + #[tokio::test] + async fn parquet_decimal_int_precision_narrowing_errors() -> Result<(), DataFusionError> { + let batch = decimal_batch(10, 4)?; + let required_schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Decimal128(5, 2), + false, + )])); + + let err = roundtrip(&batch, required_schema) + .await + .expect_err("expected ParquetSchemaConvert for Decimal(10, 4) -> Decimal(5, 2)"); + let msg = err.to_string(); + assert!(msg.contains("Column: [[a]]"), "unexpected error: {msg}"); + Ok(()) + } + + /// Reading Decimal(P, S) as Decimal(P, S') where S' > S but `P - S` did + /// not grow means the cast would shift integer digits into the fractional + /// part and lose the most-significant digit. Example: Decimal(5, 2) -> + /// Decimal(5, 3): scaleIncrease=1, precisionIncrease=0. See #4343. + #[tokio::test] + async fn parquet_decimal_scale_widening_without_precision_errors() -> Result<(), DataFusionError> + { + let batch = decimal_batch(5, 2)?; + let required_schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Decimal128(5, 3), + false, + )])); + + let err = roundtrip(&batch, required_schema) + .await + .expect_err("expected ParquetSchemaConvert for Decimal(5, 2) -> Decimal(5, 3)"); + let msg = err.to_string(); + assert!(msg.contains("Column: [[a]]"), "unexpected error: {msg}"); + Ok(()) + } + + /// Sanity check: widening both precision and scale by the same amount is + /// allowed (the cast is lossless). Decimal(5, 2) -> Decimal(7, 4) gives + /// scaleIncrease=2, precisionIncrease=2, so `precisionIncrease >= scaleIncrease`. + #[tokio::test] + async fn parquet_decimal_widening_succeeds() -> Result<(), DataFusionError> { + let batch = decimal_batch(5, 2)?; + let required_schema = Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Decimal128(7, 4), + false, + )])); + + let _ = roundtrip(&batch, required_schema).await?; + Ok(()) + } + + /// Helper for the #4297 rejection tests: write a 1-row batch and assert + /// that reading it under `read_type` raises `ParquetSchemaConvert`. + async fn assert_rejected_conversion( + file_field: Field, + values: Arc, + read_type: DataType, + ) -> Result { + let file_schema = Arc::new(Schema::new(vec![file_field])); + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![values])?; + let read_field_name = file_schema.field(0).name(); + let required_schema = Arc::new(Schema::new(vec![Field::new( + read_field_name, + read_type, + false, + )])); + let err = roundtrip(&batch, required_schema) + .await + .expect_err("expected ParquetSchemaConvert"); + Ok(err.to_string()) + } + + /// `INT64 -> INT32` truncates to the lower 32 bits in DataFusion's cast. + /// Spark's vectorized reader rejects this. See #4297. + #[tokio::test] + async fn parquet_long_read_as_int_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(Int64Array::from(vec![1i64, 1 << 33])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int64, false), + values, + DataType::Int32, + ) + .await?; + assert!( + msg.contains("Found: INT64") && msg.contains("Expected: int"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `INT64 -> Float64` loses precision for large values; Spark rejects. + #[tokio::test] + async fn parquet_long_read_as_double_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int64Array::from(vec![1i64, (1i64 << 54) + 1])) + as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int64, false), + values, + DataType::Float64, + ) + .await?; + assert!( + msg.contains("Found: INT64") && msg.contains("Expected: double"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Float64 -> Float32` overflows / loses precision; Spark rejects. + #[tokio::test] + async fn parquet_double_read_as_float_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(Float64Array::from(vec![1.5_f64, 1e40])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Float64, false), + values, + DataType::Float32, + ) + .await?; + assert!( + msg.contains("Found: DOUBLE") && msg.contains("Expected: float"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Float32 -> Int64` truncates the fractional part; Spark rejects. + #[tokio::test] + async fn parquet_float_read_as_long_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(Float32Array::from(vec![1.5_f32, 2.5])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Float32, false), + values, + DataType::Int64, + ) + .await?; + assert!( + msg.contains("Found: FLOAT") && msg.contains("Expected: bigint"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Float64 -> Int64` similarly. + #[tokio::test] + async fn parquet_double_read_as_long_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(Float64Array::from(vec![1.5_f64, 2.5])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Float64, false), + values, + DataType::Int64, + ) + .await?; + assert!( + msg.contains("Found: DOUBLE") && msg.contains("Expected: bigint"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Int32 -> Float32` loses precision for values past `2^24`. Spark + /// allows `Int32 -> Float64` but rejects `Int32 -> Float32`. + #[tokio::test] + async fn parquet_int_read_as_float_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(Int32Array::from(vec![1, (1 << 25) + 1])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int32, false), + values, + DataType::Float32, + ) + .await?; + assert!( + msg.contains("Found: INT32") && msg.contains("Expected: float"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Int32 -> Timestamp(_, None)`: raw INT32 reinterpreted as epoch seconds + /// produces dates near the Unix epoch. Only DATE-annotated INT32 columns + /// (which surface as `Date32`) are allowed to read as `TimestampNTZ`. + #[tokio::test] + async fn parquet_int_read_as_timestamp_ntz_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int32, false), + values, + DataType::Timestamp(arrow::datatypes::TimeUnit::Microsecond, None), + ) + .await?; + assert!( + msg.contains("Found: INT32") && msg.contains("Expected: timestamp"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Int64 -> Date32` similarly: raw INT64 (no DATE annotation, otherwise + /// the file would surface as `Date32`). + #[tokio::test] + async fn parquet_long_read_as_date_errors() -> Result<(), DataFusionError> { + let values = Arc::new(Int64Array::from(vec![1i64, 2])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Int64, false), + values, + DataType::Date32, + ) + .await?; + assert!( + msg.contains("Found: INT64") && msg.contains("Expected: date"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Date32 -> Timestamp(_, Some(_))` (LTZ). Spark's vectorized reader + /// allows `Date -> TimestampNTZ` but not `Date -> Timestamp(LTZ)`. + #[tokio::test] + async fn parquet_date_read_as_ltz_timestamp_errors() -> Result<(), DataFusionError> { + let values = + Arc::new(Date32Array::from(vec![18262, 18263])) as Arc; + let msg = assert_rejected_conversion( + Field::new("a", DataType::Date32, false), + values, + DataType::Timestamp(arrow::datatypes::TimeUnit::Microsecond, Some("UTC".into())), + ) + .await?; + assert!( + msg.contains("Found: INT32") && msg.contains("Expected: timestamp"), + "unexpected error: {msg}" + ); + Ok(()) + } + + /// `Timestamp(_, _) -> Date32`: no Timestamp updater branches into + /// `DateType`, so Spark rejects. + #[tokio::test] + async fn parquet_timestamp_read_as_date_errors() -> Result<(), DataFusionError> { + let values = Arc::new(TimestampMicrosecondArray::from(vec![0i64, 1_000_000])) + as Arc; + let msg = assert_rejected_conversion( + Field::new( + "a", + DataType::Timestamp(arrow::datatypes::TimeUnit::Microsecond, None), + false, + ), + values, + DataType::Date32, + ) + .await?; + assert!(msg.contains("Expected: date"), "unexpected error: {msg}"); + Ok(()) + } + + /// SPARK-26709: an empty Parquet file with a column that would otherwise fail + /// the type-promotion check (INT32 read as INT64 when allow_type_promotion is + /// false) must still be readable. Spark's vectorized reader only enforces the + /// check per row group, so a file with no row groups passes silently. The + /// adapter's plan-time rejection must not fire for the empty-file case. + #[tokio::test] + async fn parquet_empty_file_disallowed_widening() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![Field::new("col", DataType::Int32, false)])); + let filename = get_temp_filename(); + let filename = filename.as_path().as_os_str().to_str().unwrap().to_string(); + let file = File::create(&filename)?; + let writer = ArrowWriter::try_new(file, Arc::clone(&file_schema), None)?; + writer.close()?; + + let required_schema = + Arc::new(Schema::new(vec![Field::new("col", DataType::Int64, false)])); + + let mut spark_parquet_options = SparkParquetOptions::new(EvalMode::Legacy, "UTC", false); + spark_parquet_options.allow_type_promotion = false; + + let expr_adapter_factory: Arc = Arc::new( + SparkPhysicalExprAdapterFactory::new(spark_parquet_options, None), + ); + + let object_store_url = ObjectStoreUrl::local_filesystem(); + let parquet_source = ParquetSource::new(required_schema); + let files = FileGroup::new(vec![PartitionedFile::from_path(filename)?]); + let file_scan_config = + FileScanConfigBuilder::new(object_store_url, Arc::new(parquet_source)) + .with_file_groups(vec![files]) + .with_expr_adapter(Some(expr_adapter_factory)) + .build(); + + let parquet_exec = DataSourceExec::new(Arc::new(file_scan_config)); + let mut stream = parquet_exec.execute(0, Arc::new(TaskContext::default()))?; + while let Some(batch) = stream.next().await { + let batch = batch?; + assert_eq!(batch.num_rows(), 0); + } + Ok(()) + } + + /// Companion to `parquet_empty_file_disallowed_widening`: a file with rows + /// must still raise `ParquetSchemaConvert` when the same widening is + /// rejected. Verifies the runtime check fires on non-empty input, + /// matching Spark's per-row-group behavior. + #[tokio::test] + async fn parquet_non_empty_file_disallowed_widening_errors() -> Result<(), DataFusionError> { + let file_schema = Arc::new(Schema::new(vec![Field::new("col", DataType::Int32, false)])); + let values = Arc::new(Int32Array::from(vec![1, 2, 3])) as Arc; + let batch = RecordBatch::try_new(Arc::clone(&file_schema), vec![values])?; + + let filename = get_temp_filename(); + let filename = filename.as_path().as_os_str().to_str().unwrap().to_string(); + let file = File::create(&filename)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&file_schema), None)?; + writer.write(&batch)?; + writer.close()?; + + let required_schema = + Arc::new(Schema::new(vec![Field::new("col", DataType::Int64, false)])); + + let mut spark_parquet_options = SparkParquetOptions::new(EvalMode::Legacy, "UTC", false); + spark_parquet_options.allow_type_promotion = false; + + let expr_adapter_factory: Arc = Arc::new( + SparkPhysicalExprAdapterFactory::new(spark_parquet_options, None), + ); + + let object_store_url = ObjectStoreUrl::local_filesystem(); + let parquet_source = ParquetSource::new(required_schema); + let files = FileGroup::new(vec![PartitionedFile::from_path(filename)?]); + let file_scan_config = + FileScanConfigBuilder::new(object_store_url, Arc::new(parquet_source)) + .with_file_groups(vec![files]) + .with_expr_adapter(Some(expr_adapter_factory)) + .build(); + + let parquet_exec = DataSourceExec::new(Arc::new(file_scan_config)); + let mut stream = parquet_exec.execute(0, Arc::new(TaskContext::default()))?; + let first = stream.next().await.unwrap(); + let err = first.expect_err("expected ParquetSchemaConvert error on non-empty file"); + let msg = err.to_string(); + // The JVM shim sees the inner "[col]" via the JSON `column` field, matching + // Spark's `Arrays.toString(descriptor.getPath())` format. The Rust display + // wraps with another `[...]` from the error template. + assert!( + msg.contains("Column: [[col]]") + && msg.contains("Expected: bigint") + && msg.contains("Found: INT32"), + "unexpected error: {msg}" + ); Ok(()) } diff --git a/native/core/src/parquet/util/bit_packing.rs b/native/core/src/parquet/util/bit_packing.rs deleted file mode 100644 index fd39ba8fa3..0000000000 --- a/native/core/src/parquet/util/bit_packing.rs +++ /dev/null @@ -1,3658 +0,0 @@ -// 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. - -/// Unpack 32 values with bit width `num_bits` from `in_ptr`, and write to `out_ptr`. -/// Return the `in_ptr` where the starting offset points to the first byte after all the -/// bytes that were consumed. -// TODO: may be better to make these more compact using if-else conditions. -// However, this may require const generics: -// https://github.com/rust-lang/rust/issues/44580 -// to eliminate the branching cost. -// TODO: we should use SIMD instructions to further optimize this. I have explored -// https://github.com/tantivy-search/bitpacking -// but the layout it uses for SIMD is different from Parquet. -// TODO: support packing as well, which is used for encoding. -pub unsafe fn unpack32(mut in_ptr: *const u32, out_ptr: *mut u32, num_bits: usize) -> *const u32 { - in_ptr = match num_bits { - 0 => nullunpacker32(in_ptr, out_ptr), - 1 => unpack1_32(in_ptr, out_ptr), - 2 => unpack2_32(in_ptr, out_ptr), - 3 => unpack3_32(in_ptr, out_ptr), - 4 => unpack4_32(in_ptr, out_ptr), - 5 => unpack5_32(in_ptr, out_ptr), - 6 => unpack6_32(in_ptr, out_ptr), - 7 => unpack7_32(in_ptr, out_ptr), - 8 => unpack8_32(in_ptr, out_ptr), - 9 => unpack9_32(in_ptr, out_ptr), - 10 => unpack10_32(in_ptr, out_ptr), - 11 => unpack11_32(in_ptr, out_ptr), - 12 => unpack12_32(in_ptr, out_ptr), - 13 => unpack13_32(in_ptr, out_ptr), - 14 => unpack14_32(in_ptr, out_ptr), - 15 => unpack15_32(in_ptr, out_ptr), - 16 => unpack16_32(in_ptr, out_ptr), - 17 => unpack17_32(in_ptr, out_ptr), - 18 => unpack18_32(in_ptr, out_ptr), - 19 => unpack19_32(in_ptr, out_ptr), - 20 => unpack20_32(in_ptr, out_ptr), - 21 => unpack21_32(in_ptr, out_ptr), - 22 => unpack22_32(in_ptr, out_ptr), - 23 => unpack23_32(in_ptr, out_ptr), - 24 => unpack24_32(in_ptr, out_ptr), - 25 => unpack25_32(in_ptr, out_ptr), - 26 => unpack26_32(in_ptr, out_ptr), - 27 => unpack27_32(in_ptr, out_ptr), - 28 => unpack28_32(in_ptr, out_ptr), - 29 => unpack29_32(in_ptr, out_ptr), - 30 => unpack30_32(in_ptr, out_ptr), - 31 => unpack31_32(in_ptr, out_ptr), - 32 => unpack32_32(in_ptr, out_ptr), - _ => unimplemented!(), - }; - in_ptr -} - -unsafe fn nullunpacker32(in_buf: *const u32, mut out: *mut u32) -> *const u32 { - for _ in 0..32 { - *out = 0; - out = out.offset(1); - } - in_buf -} - -unsafe fn unpack1_32(in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 1) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 2) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 3) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 5) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 7) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 9) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 11) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 19) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 21) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 23) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 25) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 26) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 27) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 29) & 1; - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 30) & 1; - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - - in_buf.offset(1) -} - -unsafe fn unpack2_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 2); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - out = out.offset(1); - in_buf = in_buf.offset(1); - *out = (in_buf.read_unaligned()) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 2); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 2); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - - in_buf.offset(1) -} - -unsafe fn unpack3_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 27) % (1u32 << 3); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (3 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 25) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 3); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (3 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 23) % (1u32 << 3); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 3); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - - in_buf.offset(1) -} - -unsafe fn unpack4_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 4); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 4); - - in_buf.offset(1) -} - -unsafe fn unpack5_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 25) % (1u32 << 5); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (5 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 23) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 28) % (1u32 << 5); - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (5 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 26) % (1u32 << 5); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (5 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 5); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (5 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 5); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 5); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - - in_buf.offset(1) -} - -unsafe fn unpack6_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 6); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (6 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 6); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (6 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 6); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 6); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (6 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 6); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (6 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 6); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 6); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - - in_buf.offset(1) -} - -unsafe fn unpack7_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (7 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 24) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (7 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (7 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 23) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (7 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (7 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (7 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 7); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 7); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - - in_buf.offset(1) -} - -unsafe fn unpack8_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 8); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 8); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - - in_buf.offset(1) -} - -unsafe fn unpack9_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (9 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 22) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (9 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (9 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 21) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (9 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (9 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (9 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (9 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (9 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 9); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 9); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - - in_buf.offset(1) -} - -unsafe fn unpack10_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (10 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (10 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (10 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (10 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (10 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (10 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (10 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (10 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 10); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 10); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - - in_buf.offset(1) -} - -unsafe fn unpack11_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (11 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (11 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (11 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (11 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (11 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (11 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (11 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (11 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 19) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (11 - 9); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 11); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 20) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (11 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 11); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 21; - - in_buf.offset(1) -} - -unsafe fn unpack12_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (12 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 12); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (12 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 12); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - - in_buf.offset(1) -} - -unsafe fn unpack13_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 13); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (13 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (13 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 13); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (13 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (13 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 13); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (13 - 9); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (13 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 13); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (13 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (13 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 13); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 17) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (13 - 11); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (13 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 13); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 18) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (13 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (13 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 13); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 19; - - in_buf.offset(1) -} - -unsafe fn unpack14_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 14); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (14 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (14 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (14 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 14); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (14 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (14 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (14 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 14); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (14 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (14 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (14 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 14); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (14 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (14 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (14 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 14); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 18; - - in_buf.offset(1) -} - -unsafe fn unpack15_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 15); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 15) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (15 - 13); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (15 - 11); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (15 - 9); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (15 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (15 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (15 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (15 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 15); - out = out.offset(1); - *out = ((in_buf.read_unaligned()) >> 16) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (15 - 14); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (15 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (15 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (15 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (15 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (15 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (15 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 15); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 17; - - in_buf.offset(1) -} - -unsafe fn unpack16_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - out = out.offset(1); - in_buf = in_buf.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 16); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 16; - - in_buf.offset(1) -} - -unsafe fn unpack17_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (17 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (17 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (17 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (17 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (17 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (17 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (17 - 14); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 14) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (17 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (17 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (17 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (17 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (17 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (17 - 9); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (17 - 11); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (17 - 13); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 13) % (1u32 << 17); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (17 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - - in_buf.offset(1) -} - -unsafe fn unpack18_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (18 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (18 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (18 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (18 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (18 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (18 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (18 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (18 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (18 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (18 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (18 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (18 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (18 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (18 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (18 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 18); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (18 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - - in_buf.offset(1) -} - -unsafe fn unpack19_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (19 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (19 - 12); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 12) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (19 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (19 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (19 - 11); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 11) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (19 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (19 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (19 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (19 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (19 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (19 - 9); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (19 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (19 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (19 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (19 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (19 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (19 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 19); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (19 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - - in_buf.offset(1) -} - -unsafe fn unpack20_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (20 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (20 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (20 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 20); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (20 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - - in_buf.offset(1) -} - -unsafe fn unpack21_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (21 - 10); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 10) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (21 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (21 - 9); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 9) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (21 - 19); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (21 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (21 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (21 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (21 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (21 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (21 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (21 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (21 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (21 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (21 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (21 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (21 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (21 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (21 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (21 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 21); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (21 - 11); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 11; - - in_buf.offset(1) -} - -unsafe fn unpack22_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (22 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (22 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (22 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (22 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (22 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (22 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (22 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (22 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (22 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (22 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (22 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (22 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (22 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (22 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (22 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (22 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (22 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (22 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 22); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (22 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (22 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - - in_buf.offset(1) -} - -unsafe fn unpack23_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (23 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (23 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (23 - 19); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (23 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (23 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (23 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (23 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (23 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (23 - 11); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 11; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (23 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (23 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (23 - 7); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 7) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (23 - 21); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (23 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (23 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (23 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (23 - 8); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 8) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (23 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (23 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (23 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 23); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (23 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (23 - 9); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 9; - - in_buf.offset(1) -} - -unsafe fn unpack24_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 24); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (24 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (24 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - - in_buf.offset(1) -} - -unsafe fn unpack25_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (25 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (25 - 11); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 11; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (25 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (25 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (25 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (25 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (25 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (25 - 19); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (25 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (25 - 5); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 5) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (25 - 23); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (25 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (25 - 9); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 9; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (25 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (25 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (25 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (25 - 6); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 6) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (25 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (25 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (25 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (25 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 25); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (25 - 21); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (25 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (25 - 7); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 7; - - in_buf.offset(1) -} - -unsafe fn unpack26_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 26); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (26 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (26 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (26 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (26 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 26); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (26 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (26 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (26 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (26 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 26); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (26 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (26 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (26 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (26 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 26); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (26 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (26 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (26 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (26 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 26); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (26 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (26 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (26 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (26 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 26); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (26 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (26 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (26 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (26 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - - in_buf.offset(1) -} - -unsafe fn unpack27_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 27); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (27 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (27 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (27 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (27 - 7); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 7; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (27 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 27); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (27 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (27 - 19); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (27 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (27 - 9); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 9; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (27 - 4); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 4) % (1u32 << 27); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (27 - 26); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (27 - 21); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (27 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (27 - 11); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 11; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (27 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (27 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 27); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (27 - 23); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (27 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (27 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (27 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (27 - 3); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 3) % (1u32 << 27); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 25)) << (27 - 25); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (27 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (27 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (27 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (27 - 5); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 5; - - in_buf.offset(1) -} - -unsafe fn unpack28_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 28); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 28); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 28); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 28); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (28 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (28 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (28 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (28 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (28 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (28 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - - in_buf.offset(1) -} - -unsafe fn unpack29_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 29); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (29 - 26); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (29 - 23); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (29 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (29 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (29 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (29 - 11); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 11; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (29 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (29 - 5); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 5; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (29 - 2); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 2) % (1u32 << 29); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (29 - 28); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 25)) << (29 - 25); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (29 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (29 - 19); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (29 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (29 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (29 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (29 - 7); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 7; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (29 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (29 - 1); - out = out.offset(1); - - *out = ((in_buf.read_unaligned()) >> 1) % (1u32 << 29); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 27)) << (29 - 27); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (29 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (29 - 21); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (29 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (29 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (29 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (29 - 9); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 9; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (29 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (29 - 3); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 3; - - in_buf.offset(1) -} - -unsafe fn unpack30_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 30); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (30 - 28); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (30 - 26); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (30 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (30 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (30 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (30 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (30 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (30 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (30 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (30 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (30 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (30 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (30 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (30 - 2); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 2; - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) % (1u32 << 30); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (30 - 28); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (30 - 26); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (30 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (30 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (30 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (30 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (30 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (30 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (30 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (30 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (30 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (30 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (30 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (30 - 2); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 2; - - in_buf.offset(1) -} - -unsafe fn unpack31_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = (in_buf.read_unaligned()) % (1u32 << 31); - out = out.offset(1); - *out = (in_buf.read_unaligned()) >> 31; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 30)) << (31 - 30); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 30; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 29)) << (31 - 29); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 29; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 28)) << (31 - 28); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 28; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 27)) << (31 - 27); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 27; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 26)) << (31 - 26); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 26; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 25)) << (31 - 25); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 25; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 24)) << (31 - 24); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 24; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 23)) << (31 - 23); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 23; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 22)) << (31 - 22); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 22; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 21)) << (31 - 21); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 21; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 20)) << (31 - 20); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 20; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 19)) << (31 - 19); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 19; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 18)) << (31 - 18); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 18; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 17)) << (31 - 17); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 17; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 16)) << (31 - 16); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 16; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 15)) << (31 - 15); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 15; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 14)) << (31 - 14); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 14; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 13)) << (31 - 13); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 13; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 12)) << (31 - 12); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 12; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 11)) << (31 - 11); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 11; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 10)) << (31 - 10); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 10; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 9)) << (31 - 9); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 9; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 8)) << (31 - 8); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 8; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 7)) << (31 - 7); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 7; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 6)) << (31 - 6); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 6; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 5)) << (31 - 5); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 5; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 4)) << (31 - 4); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 4; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 3)) << (31 - 3); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 3; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 2)) << (31 - 2); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 2; - in_buf = in_buf.offset(1); - *out |= ((in_buf.read_unaligned()) % (1u32 << 1)) << (31 - 1); - out = out.offset(1); - - *out = (in_buf.read_unaligned()) >> 1; - - in_buf.offset(1) -} - -unsafe fn unpack32_32(mut in_buf: *const u32, mut out: *mut u32) -> *const u32 { - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - in_buf = in_buf.offset(1); - out = out.offset(1); - - *out = in_buf.read_unaligned(); - - in_buf.offset(1) -} diff --git a/native/core/src/parquet/util/buffer.rs b/native/core/src/parquet/util/buffer.rs deleted file mode 100644 index 72cfa3fe33..0000000000 --- a/native/core/src/parquet/util/buffer.rs +++ /dev/null @@ -1,128 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{ops::Index, slice::SliceIndex, sync::Arc}; - -/// An immutable byte buffer. -pub trait Buffer { - /// Returns the length (in bytes) of this buffer. - fn len(&self) -> usize; - - /// Returns the byte array of this buffer, in range `[0, len)`. - fn data(&self) -> &[u8]; - - /// Returns whether this buffer is empty or not. - fn is_empty(&self) -> bool { - self.len() == 0 - } -} - -impl Buffer for Vec { - fn len(&self) -> usize { - self.len() - } - - fn data(&self) -> &[u8] { - self - } -} - -pub struct BufferRef { - inner: Arc, - offset: usize, - len: usize, -} - -impl BufferRef { - pub fn new(inner: Arc) -> Self { - let len = inner.len(); - Self { - inner, - offset: 0, - len, - } - } - - /// Returns the length of this buffer. - #[inline] - pub fn len(&self) -> usize { - self.len - } - - #[inline] - pub fn is_empty(&self) -> bool { - self.len == 0 - } - - #[inline] - pub fn data(&self) -> &[u8] { - self.inner.data() - } - - /// Creates a new byte buffer containing elements in `[offset, offset+len)` - #[inline] - pub fn slice(&self, offset: usize, len: usize) -> BufferRef { - assert!( - self.offset + offset + len <= self.inner.len(), - "can't create a buffer slice with offset exceeding original \ - JNI array len {}, self.offset: {}, offset: {}, len: {}", - self.inner.len(), - self.offset, - offset, - len - ); - - Self { - inner: Arc::clone(&self.inner), - offset: self.offset + offset, - len, - } - } - - /// Creates a new byte buffer containing all elements starting from `offset` in this byte array. - #[inline] - pub fn start(&self, offset: usize) -> BufferRef { - assert!( - self.offset + offset <= self.inner.len(), - "can't create a buffer slice with offset exceeding original \ - JNI array len {}, self.offset: {}, offset: {}", - self.inner.len(), - self.offset, - offset - ); - let len = self.inner.len() - offset - self.offset; - self.slice(offset, len) - } -} - -impl AsRef<[u8]> for BufferRef { - fn as_ref(&self) -> &[u8] { - let slice = self.inner.as_ref().data(); - &slice[self.offset..self.offset + self.len] - } -} - -impl Index for BufferRef -where - Idx: SliceIndex<[u8]>, -{ - type Output = Idx::Output; - - fn index(&self, index: Idx) -> &Self::Output { - &self.as_ref()[index] - } -} diff --git a/native/core/src/parquet/util/jni.rs b/native/core/src/parquet/util/jni.rs index 1fb9519c8d..c1a7cf5017 100644 --- a/native/core/src/parquet/util/jni.rs +++ b/native/core/src/parquet/util/jni.rs @@ -15,193 +15,8 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - -use jni::{ - errors::Result as JNIResult, - objects::{JObjectArray, JString}, - sys::{jboolean, jint}, - Env, -}; - use arrow::error::ArrowError; use arrow::ipc::reader::StreamReader; -use datafusion::execution::object_store::ObjectStoreUrl; -use object_store::path::Path; -use parquet::{ - basic::{Encoding, LogicalType, TimeUnit, Type as PhysicalType}, - schema::types::{ColumnDescriptor, ColumnPath, PrimitiveTypeBuilder}, -}; -use url::{ParseError, Url}; - -/// Convert primitives from Spark side into a `ColumnDescriptor`. -#[allow(clippy::too_many_arguments)] -pub fn convert_column_descriptor( - env: &mut Env, - physical_type_id: jint, - logical_type_id: jint, - max_dl: jint, - max_rl: jint, - bit_width: jint, - is_signed: jboolean, - type_length: jint, - precision: jint, - scale: jint, - time_unit: jint, - is_adjusted_utc: jboolean, - jni_path: JObjectArray, -) -> JNIResult { - let physical_type = convert_physical_type(physical_type_id); - let type_length = fix_type_length(&physical_type, type_length); - let logical_type = if logical_type_id >= 0 { - Some(convert_logical_type( - logical_type_id, - bit_width, - is_signed, - precision, - scale, - time_unit, - is_adjusted_utc, - )) - } else { - // id < 0 means there is no logical type associated - None - }; - - // We don't care the column name here - let ty = PrimitiveTypeBuilder::new("f", physical_type) - .with_logical_type(logical_type) - .with_length(type_length) - .with_precision(precision) // Parquet crate requires to set this even with logical type - .with_scale(scale) - .build() - .unwrap(); // TODO: convert Parquet errot to JNI error - let path = convert_column_path(env, jni_path).unwrap(); - - let result = ColumnDescriptor::new(Arc::new(ty), max_dl as i16, max_rl as i16, path); - Ok(result) -} - -pub fn convert_encoding(ordinal: jint) -> Encoding { - match ordinal { - 0 => Encoding::PLAIN, - 1 => Encoding::RLE, - #[allow(deprecated)] - 3 => Encoding::BIT_PACKED, - 4 => Encoding::PLAIN_DICTIONARY, - 5 => Encoding::DELTA_BINARY_PACKED, - 6 => Encoding::DELTA_LENGTH_BYTE_ARRAY, - 7 => Encoding::DELTA_BYTE_ARRAY, - 8 => Encoding::RLE_DICTIONARY, - _ => panic!("Invalid Java Encoding ordinal: {ordinal}"), - } -} - -#[derive(Debug)] -pub struct TypePromotionInfo { - pub(crate) physical_type: PhysicalType, - pub(crate) precision: i32, - pub(crate) scale: i32, - pub(crate) bit_width: i32, -} - -impl TypePromotionInfo { - pub fn new_from_jni( - physical_type_id: jint, - precision: jint, - scale: jint, - bit_width: jint, - ) -> Self { - let physical_type = convert_physical_type(physical_type_id); - Self { - physical_type, - precision, - scale, - bit_width, - } - } - - pub fn new(physical_type: PhysicalType, precision: i32, scale: i32, bit_width: i32) -> Self { - Self { - physical_type, - precision, - scale, - bit_width, - } - } -} - -fn convert_column_path(env: &mut Env, path_array: JObjectArray) -> JNIResult { - let array_len = path_array.len(env)?; - let mut res: Vec = Vec::new(); - for i in 0..array_len { - let p = path_array.get_element(env, i)?; - let p: JString = unsafe { JString::from_raw(env, p.into_raw()) }; - res.push(p.try_to_string(env)?); - } - Ok(ColumnPath::new(res)) -} - -fn convert_physical_type(id: jint) -> PhysicalType { - match id { - 0 => PhysicalType::BOOLEAN, - 1 => PhysicalType::INT32, - 2 => PhysicalType::INT64, - 3 => PhysicalType::INT96, - 4 => PhysicalType::FLOAT, - 5 => PhysicalType::DOUBLE, - 6 => PhysicalType::BYTE_ARRAY, - 7 => PhysicalType::FIXED_LEN_BYTE_ARRAY, - _ => panic!("Invalid id for Parquet physical type: {id} "), - } -} - -fn convert_logical_type( - id: jint, - bit_width: jint, - is_signed: jboolean, - precision: jint, - scale: jint, - time_unit: jint, - is_adjusted_utc: jboolean, -) -> LogicalType { - match id { - 0 => LogicalType::Integer { - bit_width: bit_width as i8, - is_signed, - }, - 1 => LogicalType::String, - 2 => LogicalType::Decimal { scale, precision }, - 3 => LogicalType::Date, - 4 => LogicalType::Timestamp { - is_adjusted_to_u_t_c: is_adjusted_utc, - unit: convert_time_unit(time_unit), - }, - 5 => LogicalType::Enum, - 6 => LogicalType::Uuid, - _ => panic!("Invalid id for Parquet logical type: {id}"), - } -} - -fn convert_time_unit(time_unit: jint) -> TimeUnit { - match time_unit { - 0 => TimeUnit::MILLIS, - 1 => TimeUnit::MICROS, - 2 => TimeUnit::NANOS, - _ => panic!("Invalid time unit id for Parquet: {time_unit}"), - } -} - -/// Fixes the type length in case they are not set (Parquet only explicitly set it for -/// FIXED_LEN_BYTE_ARRAY type). -fn fix_type_length(t: &PhysicalType, type_length: i32) -> i32 { - match t { - PhysicalType::INT32 | PhysicalType::FLOAT => 4, - PhysicalType::INT64 | PhysicalType::DOUBLE => 8, - PhysicalType::INT96 => 12, - _ => type_length, - } -} pub fn deserialize_schema(ipc_bytes: &[u8]) -> Result { let reader = unsafe { @@ -210,46 +25,3 @@ pub fn deserialize_schema(ipc_bytes: &[u8]) -> Result Result<(ObjectStoreUrl, Path), ParseError> { - // we define origin of a url as scheme + "://" + authority + ["/" + bucket] - let url = Url::parse(url_.as_ref()).unwrap(); - let mut object_store_origin = url.scheme().to_owned(); - let mut object_store_path = Path::from_url_path(url.path()).unwrap(); - if object_store_origin == "s3a" { - object_store_origin = "s3".to_string(); - object_store_origin.push_str("://"); - object_store_origin.push_str(url.authority()); - object_store_origin.push('/'); - let path_splits = url.path_segments().map(|c| c.collect::>()).unwrap(); - object_store_origin.push_str(path_splits.first().unwrap()); - let new_path = path_splits[1..path_splits.len() - 1].join("/"); - //TODO: (ARROW NATIVE) check the use of unwrap here - object_store_path = Path::from_url_path(new_path.clone().as_str()).unwrap(); - } else { - object_store_origin.push_str("://"); - object_store_origin.push_str(url.authority()); - object_store_origin.push('/'); - } - Ok(( - ObjectStoreUrl::parse(object_store_origin).unwrap(), - object_store_path, - )) -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_get_file_path() { - let inp = "file:///comet/spark-warehouse/t1/part1=2019-01-01%2011%253A11%253A11/part-00000-84d7ed74-8f28-456c-9270-f45376eea144.c000.snappy.parquet"; - let expected = "comet/spark-warehouse/t1/part1=2019-01-01 11%3A11%3A11/part-00000-84d7ed74-8f28-456c-9270-f45376eea144.c000.snappy.parquet"; - - if let Ok((_obj_store_url, path)) = get_file_path(inp.to_string()) { - let actual = path.to_string(); - assert_eq!(actual, expected); - } - } -} diff --git a/native/core/src/parquet/util/memory.rs b/native/core/src/parquet/util/memory.rs deleted file mode 100644 index ed2ab98077..0000000000 --- a/native/core/src/parquet/util/memory.rs +++ /dev/null @@ -1,557 +0,0 @@ -// 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. - -//! Utility methods and structs for working with memory. - -use std::{ - fmt::{Debug, Display, Formatter, Result as FmtResult}, - io::{Result as IoResult, Write}, - mem, - ops::{Index, IndexMut}, - sync::{ - atomic::{AtomicI64, Ordering}, - Arc, Weak, - }, -}; - -// ---------------------------------------------------------------------- -// Memory Tracker classes - -/// Reference counted pointer for [`MemTracker`]. -pub type MemTrackerPtr = Arc; -/// Non-owning reference for [`MemTracker`]. -pub type WeakMemTrackerPtr = Weak; - -/// Struct to track memory usage information. -#[derive(Debug)] -pub struct MemTracker { - // In the tuple, the first element is the current memory allocated (in bytes), - // and the second element is the maximum memory allocated so far (in bytes). - current_memory_usage: AtomicI64, - max_memory_usage: AtomicI64, -} - -impl MemTracker { - /// Creates new memory tracker. - #[inline] - pub fn new() -> MemTracker { - MemTracker { - current_memory_usage: Default::default(), - max_memory_usage: Default::default(), - } - } - - /// Returns the current memory consumption, in bytes. - pub fn memory_usage(&self) -> i64 { - self.current_memory_usage.load(Ordering::Acquire) - } - - /// Returns the maximum memory consumption so far, in bytes. - pub fn max_memory_usage(&self) -> i64 { - self.max_memory_usage.load(Ordering::Acquire) - } - - /// Adds `num_bytes` to the memory consumption tracked by this memory tracker. - #[inline] - pub fn alloc(&self, num_bytes: i64) { - let new_current = self - .current_memory_usage - .fetch_add(num_bytes, Ordering::Acquire) - + num_bytes; - self.max_memory_usage - .fetch_max(new_current, Ordering::Acquire); - } -} - -impl Default for MemTracker { - fn default() -> Self { - Self::new() - } -} - -// ---------------------------------------------------------------------- -// Buffer classes - -/// Type alias for [`Buffer`]. -pub type ByteBuffer = Buffer; -/// Type alias for [`BufferPtr`]. -pub type ByteBufferPtr = BufferPtr; - -/// A resize-able buffer class with generic member, with optional memory tracker. -/// -/// Note that a buffer has two attributes: -/// `capacity` and `size`: the former is the total number of space reserved for -/// the buffer, while the latter is the actual number of elements. -/// Invariant: `capacity` >= `size`. -/// The total allocated bytes for a buffer equals to `capacity * sizeof()`. -pub struct Buffer { - data: Vec, - mem_tracker: Option, - type_length: usize, -} - -impl Buffer { - /// Creates new empty buffer. - pub fn new() -> Self { - Buffer { - data: vec![], - mem_tracker: None, - type_length: std::mem::size_of::(), - } - } - - /// Adds [`MemTracker`] for this buffer. - #[inline] - pub fn with_mem_tracker(mut self, mc: MemTrackerPtr) -> Self { - mc.alloc((self.data.capacity() * self.type_length) as i64); - self.mem_tracker = Some(mc); - self - } - - /// Returns slice of data in this buffer. - #[inline] - pub fn data(&self) -> &[T] { - self.data.as_slice() - } - - /// Sets data for this buffer. - #[inline] - pub fn set_data(&mut self, new_data: Vec) { - if let Some(ref mc) = self.mem_tracker { - let capacity_diff = new_data.capacity() as i64 - self.data.capacity() as i64; - mc.alloc(capacity_diff * self.type_length as i64); - } - self.data = new_data; - } - - /// Resizes underlying data in place to a new length `new_size`. - /// - /// If `new_size` is less than current length, data is truncated, otherwise, it is - /// extended to `new_size` with provided default value `init_value`. - /// - /// Memory tracker is also updated, if available. - #[inline] - pub fn resize(&mut self, new_size: usize, init_value: T) { - let old_capacity = self.data.capacity(); - self.data.resize(new_size, init_value); - if let Some(ref mc) = self.mem_tracker { - let capacity_diff = self.data.capacity() as i64 - old_capacity as i64; - mc.alloc(capacity_diff * self.type_length as i64); - } - } - - /// Clears underlying data. - #[inline] - pub fn clear(&mut self) { - self.data.clear() - } - - /// Reserves capacity `additional_capacity` for underlying data vector. - /// - /// Memory tracker is also updated, if available. - #[inline] - pub fn reserve(&mut self, additional_capacity: usize) { - let old_capacity = self.data.capacity(); - self.data.reserve(additional_capacity); - if self.data.capacity() > old_capacity { - if let Some(ref mc) = self.mem_tracker { - let capacity_diff = self.data.capacity() as i64 - old_capacity as i64; - mc.alloc(capacity_diff * self.type_length as i64); - } - } - } - - /// Returns [`BufferPtr`] with buffer data. - /// Buffer data is reset. - #[inline] - pub fn consume(&mut self) -> BufferPtr { - let old_data = mem::take(&mut self.data); - let mut result = BufferPtr::new(old_data); - if let Some(ref mc) = self.mem_tracker { - result = result.with_mem_tracker(Arc::clone(mc)); - } - result - } - - /// Adds `value` to the buffer. - #[inline] - pub fn push(&mut self, value: T) { - self.data.push(value) - } - - /// Returns current capacity for the buffer. - #[inline] - pub fn capacity(&self) -> usize { - self.data.capacity() - } - - /// Returns current size for the buffer. - #[inline] - pub fn size(&self) -> usize { - self.data.len() - } - - /// Returns `true` if memory tracker is added to buffer, `false` otherwise. - #[inline] - pub fn is_mem_tracked(&self) -> bool { - self.mem_tracker.is_some() - } - - /// Returns memory tracker associated with this buffer. - /// This may panic, if memory tracker is not set, use method above to check if - /// memory tracker is available. - #[inline] - pub fn mem_tracker(&self) -> &MemTrackerPtr { - self.mem_tracker.as_ref().unwrap() - } -} - -impl Default for Buffer { - fn default() -> Self { - Self::new() - } -} - -impl Index for Buffer { - type Output = T; - - fn index(&self, index: usize) -> &T { - &self.data[index] - } -} - -impl IndexMut for Buffer { - fn index_mut(&mut self, index: usize) -> &mut T { - &mut self.data[index] - } -} - -// TODO: implement this for other types -impl Write for Buffer { - #[inline] - fn write(&mut self, buf: &[u8]) -> IoResult { - let old_capacity = self.data.capacity(); - let bytes_written = self.data.write(buf)?; - if let Some(ref mc) = self.mem_tracker { - if self.data.capacity() - old_capacity > 0 { - mc.alloc((self.data.capacity() - old_capacity) as i64) - } - } - Ok(bytes_written) - } - - fn flush(&mut self) -> IoResult<()> { - // No-op - self.data.flush() - } -} - -impl AsRef<[u8]> for Buffer { - fn as_ref(&self) -> &[u8] { - self.data.as_slice() - } -} - -impl Drop for Buffer { - #[inline] - fn drop(&mut self) { - if let Some(ref mc) = self.mem_tracker { - mc.alloc(-((self.data.capacity() * self.type_length) as i64)); - } - } -} - -// ---------------------------------------------------------------------- -// Immutable Buffer (BufferPtr) classes - -/// An representation of a slice on a reference-counting and read-only byte array. -/// Sub-slices can be further created from this. The byte array will be released -/// when all slices are dropped. -#[allow(clippy::rc_buffer)] -#[derive(Clone, Debug)] -pub struct BufferPtr { - data: Arc>, - start: usize, - len: usize, - // TODO: will this create too many references? rethink about this. - mem_tracker: Option, -} - -impl BufferPtr { - /// Creates new buffer from a vector. - pub fn new(v: Vec) -> Self { - let len = v.len(); - Self { - data: Arc::new(v), - start: 0, - len, - mem_tracker: None, - } - } - - /// Returns slice of data in this buffer. - #[inline] - pub fn data(&self) -> &[T] { - &self.data[self.start..self.start + self.len] - } - - /// Updates this buffer with new `start` position and length `len`. - /// - /// Range should be within current start position and length. - #[inline] - pub fn with_range(mut self, start: usize, len: usize) -> Self { - self.set_range(start, len); - self - } - - /// Updates this buffer with new `start` position and length `len`. - /// - /// Range should be within current start position and length. - #[inline] - pub fn set_range(&mut self, start: usize, len: usize) { - assert!(self.start <= start && start + len <= self.start + self.len); - self.start = start; - self.len = len; - } - - /// Adds memory tracker to this buffer. - pub fn with_mem_tracker(mut self, mc: MemTrackerPtr) -> Self { - self.mem_tracker = Some(mc); - self - } - - /// Returns start position of this buffer. - #[inline] - pub fn start(&self) -> usize { - self.start - } - - /// Returns length of this buffer - #[inline] - pub fn len(&self) -> usize { - self.len - } - - /// Returns whether this buffer is empty - #[inline] - pub fn is_empty(&self) -> bool { - self.len == 0 - } - - /// Returns `true` if this buffer has memory tracker, `false` otherwise. - pub fn is_mem_tracked(&self) -> bool { - self.mem_tracker.is_some() - } - - /// Returns a shallow copy of the buffer. - /// Reference counted pointer to the data is copied. - pub fn all(&self) -> BufferPtr { - BufferPtr { - data: Arc::clone(&self.data), - start: self.start, - len: self.len, - mem_tracker: self.mem_tracker.as_ref().cloned(), - } - } - - /// Returns a shallow copy of the buffer that starts with `start` position. - pub fn start_from(&self, start: usize) -> BufferPtr { - assert!(start <= self.len); - BufferPtr { - data: Arc::clone(&self.data), - start: self.start + start, - len: self.len - start, - mem_tracker: self.mem_tracker.as_ref().cloned(), - } - } - - /// Returns a shallow copy that is a range slice within this buffer. - pub fn range(&self, start: usize, len: usize) -> BufferPtr { - assert!(start + len <= self.len); - BufferPtr { - data: Arc::clone(&self.data), - start: self.start + start, - len, - mem_tracker: self.mem_tracker.as_ref().cloned(), - } - } -} - -impl Index for BufferPtr { - type Output = T; - - fn index(&self, index: usize) -> &T { - assert!(index < self.len); - &self.data[self.start + index] - } -} - -impl Display for BufferPtr { - fn fmt(&self, f: &mut Formatter) -> FmtResult { - write!(f, "{:?}", self.data) - } -} - -impl Drop for BufferPtr { - fn drop(&mut self) { - if let Some(ref mc) = self.mem_tracker { - if Arc::strong_count(&self.data) == 1 && Arc::weak_count(&self.data) == 0 { - mc.alloc(-(self.data.capacity() as i64)); - } - } - } -} - -impl AsRef<[u8]> for BufferPtr { - #[inline] - fn as_ref(&self) -> &[u8] { - &self.data[self.start..self.start + self.len] - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_byte_buffer_mem_tracker() { - let mem_tracker = Arc::new(MemTracker::new()); - - let mut buffer = ByteBuffer::new().with_mem_tracker(Arc::clone(&mem_tracker)); - buffer.set_data(vec![0; 10]); - assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); - buffer.set_data(vec![0; 20]); - let capacity = buffer.capacity() as i64; - assert_eq!(mem_tracker.memory_usage(), capacity); - - let max_capacity = { - let mut buffer2 = ByteBuffer::new().with_mem_tracker(Arc::clone(&mem_tracker)); - buffer2.reserve(30); - assert_eq!( - mem_tracker.memory_usage(), - buffer2.capacity() as i64 + capacity - ); - buffer2.set_data(vec![0; 100]); - assert_eq!( - mem_tracker.memory_usage(), - buffer2.capacity() as i64 + capacity - ); - buffer2.capacity() as i64 + capacity - }; - - assert_eq!(mem_tracker.memory_usage(), capacity); - assert_eq!(mem_tracker.max_memory_usage(), max_capacity); - - buffer.reserve(40); - assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); - - buffer.consume(); - assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); - } - - #[test] - fn test_byte_ptr_mem_tracker() { - let mem_tracker = Arc::new(MemTracker::new()); - - let mut buffer = ByteBuffer::new().with_mem_tracker(Arc::clone(&mem_tracker)); - buffer.set_data(vec![0; 60]); - - { - let buffer_capacity = buffer.capacity() as i64; - let buf_ptr = buffer.consume(); - assert_eq!(mem_tracker.memory_usage(), buffer_capacity); - { - let buf_ptr1 = buf_ptr.all(); - { - let _ = buf_ptr.start_from(20); - assert_eq!(mem_tracker.memory_usage(), buffer_capacity); - } - assert_eq!(mem_tracker.memory_usage(), buffer_capacity); - let _ = buf_ptr1.range(30, 20); - assert_eq!(mem_tracker.memory_usage(), buffer_capacity); - } - assert_eq!(mem_tracker.memory_usage(), buffer_capacity); - } - assert_eq!(mem_tracker.memory_usage(), buffer.capacity() as i64); - } - - #[test] - fn test_byte_buffer() { - let mut buffer = ByteBuffer::new(); - assert_eq!(buffer.size(), 0); - assert_eq!(buffer.capacity(), 0); - - let mut buffer2 = ByteBuffer::new(); - buffer2.reserve(40); - assert_eq!(buffer2.size(), 0); - assert_eq!(buffer2.capacity(), 40); - - buffer.set_data((0..5).collect()); - assert_eq!(buffer.size(), 5); - assert_eq!(buffer[4], 4); - - buffer.set_data((0..20).collect()); - assert_eq!(buffer.size(), 20); - assert_eq!(buffer[10], 10); - - let expected: Vec = (0..20).collect(); - { - let data = buffer.data(); - assert_eq!(data, expected.as_slice()); - } - - buffer.reserve(40); - assert!(buffer.capacity() >= 40); - - let byte_ptr = buffer.consume(); - assert_eq!(buffer.size(), 0); - assert_eq!(byte_ptr.as_ref(), expected.as_slice()); - - let values: Vec = (0..30).collect(); - let _ = buffer.write(values.as_slice()); - let _ = buffer.flush(); - - assert_eq!(buffer.data(), values.as_slice()); - } - - #[test] - fn test_byte_ptr() { - let values = (0..50).collect(); - let ptr = ByteBufferPtr::new(values); - assert_eq!(ptr.len(), 50); - assert_eq!(ptr.start(), 0); - assert_eq!(ptr[40], 40); - - let ptr2 = ptr.all(); - assert_eq!(ptr2.len(), 50); - assert_eq!(ptr2.start(), 0); - assert_eq!(ptr2[40], 40); - - let ptr3 = ptr.start_from(20); - assert_eq!(ptr3.len(), 30); - assert_eq!(ptr3.start(), 20); - assert_eq!(ptr3[0], 20); - - let ptr4 = ptr3.range(10, 10); - assert_eq!(ptr4.len(), 10); - assert_eq!(ptr4.start(), 30); - assert_eq!(ptr4[0], 30); - - let expected: Vec = (30..40).collect(); - assert_eq!(ptr4.as_ref(), expected.as_slice()); - } -} diff --git a/native/core/src/parquet/util/mod.rs b/native/core/src/parquet/util/mod.rs index e71a6e2c78..2fb402b7e6 100644 --- a/native/core/src/parquet/util/mod.rs +++ b/native/core/src/parquet/util/mod.rs @@ -15,11 +15,4 @@ // specific language governing permissions and limitations // under the License. -pub(crate) mod bit_packing; pub mod jni; -pub mod memory; - -mod buffer; -pub use buffer::*; - -pub mod test_common; diff --git a/native/core/src/parquet/util/test_common/mod.rs b/native/core/src/parquet/util/test_common/mod.rs deleted file mode 100644 index d92544608e..0000000000 --- a/native/core/src/parquet/util/test_common/mod.rs +++ /dev/null @@ -1,23 +0,0 @@ -// 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. - -pub mod page_util; -pub mod rand_gen; - -pub use self::rand_gen::{random_bools, random_bytes, random_numbers, random_numbers_range}; - -pub use datafusion_comet_spark_expr::test_common::file_util::{get_temp_file, get_temp_filename}; diff --git a/native/core/src/parquet/util/test_common/page_util.rs b/native/core/src/parquet/util/test_common/page_util.rs deleted file mode 100644 index 6bda2db31a..0000000000 --- a/native/core/src/parquet/util/test_common/page_util.rs +++ /dev/null @@ -1,310 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::{collections::VecDeque, mem, sync::Arc}; - -use rand::distr::uniform::SampleUniform; - -use parquet::{ - basic::Encoding, - column::page::{Page, PageIterator, PageMetadata, PageReader}, - data_type::DataType, - encodings::{ - encoding::{get_encoder, DictEncoder, Encoder}, - levels::{max_buffer_size, LevelEncoder}, - }, - errors::Result, - schema::types::ColumnDescPtr, -}; - -use super::random_numbers_range; -use bytes::Bytes; - -pub trait DataPageBuilder { - fn add_rep_levels(&mut self, max_level: i16, rep_levels: &[i16]); - fn add_def_levels(&mut self, max_level: i16, def_levels: &[i16]); - fn add_values(&mut self, encoding: Encoding, values: &[T::T]); - fn add_indices(&mut self, indices: Bytes); - fn consume(self) -> Page; -} - -/// A utility struct for building data pages (v1 or v2). Callers must call: -/// - add_rep_levels() -/// - add_def_levels() -/// - add_values() for normal data page / add_indices() for dictionary data page -/// - consume() -/// in order to populate and obtain a data page. -pub struct DataPageBuilderImpl { - desc: ColumnDescPtr, - encoding: Option, - num_values: u32, - buffer: Vec, - rep_levels_byte_len: u32, - def_levels_byte_len: u32, - datapage_v2: bool, -} - -impl DataPageBuilderImpl { - // `num_values` is the number of non-null values to put in the data page. - // `datapage_v2` flag is used to indicate if the generated data page should use V2 - // format or not. - pub fn new(desc: ColumnDescPtr, num_values: u32, datapage_v2: bool) -> Self { - DataPageBuilderImpl { - desc, - encoding: None, - num_values, - buffer: vec![], - rep_levels_byte_len: 0, - def_levels_byte_len: 0, - datapage_v2, - } - } - - // Adds levels to the buffer and return number of encoded bytes - fn add_levels(&mut self, max_level: i16, levels: &[i16]) -> u32 { - if max_level <= 0 { - return 0; - } - let size = max_buffer_size(Encoding::RLE, max_level, levels.len()); - let mut level_encoder = LevelEncoder::v1(Encoding::RLE, max_level, size); - level_encoder.put(levels); - let encoded_levels = level_encoder.consume(); - // Actual encoded bytes (without length offset) - let encoded_bytes = &encoded_levels[mem::size_of::()..]; - if self.datapage_v2 { - // Level encoder always initializes with offset of i32, where it stores - // length of encoded data; for data page v2 we explicitly - // store length, therefore we should skip i32 bytes. - self.buffer.extend_from_slice(encoded_bytes); - } else { - self.buffer.extend_from_slice(encoded_levels.as_slice()); - } - encoded_bytes.len() as u32 - } -} - -impl DataPageBuilder for DataPageBuilderImpl { - fn add_rep_levels(&mut self, max_levels: i16, rep_levels: &[i16]) { - self.num_values = rep_levels.len() as u32; - self.rep_levels_byte_len = self.add_levels(max_levels, rep_levels); - } - - fn add_def_levels(&mut self, max_levels: i16, def_levels: &[i16]) { - assert!( - self.num_values == def_levels.len() as u32, - "Must call `add_rep_levels() first!`" - ); - - self.def_levels_byte_len = self.add_levels(max_levels, def_levels); - } - - fn add_values(&mut self, encoding: Encoding, values: &[T::T]) { - assert!( - self.num_values >= values.len() as u32, - "num_values: {}, values.len(): {}", - self.num_values, - values.len() - ); - self.encoding = Some(encoding); - let mut encoder: Box> = - get_encoder::(encoding, &self.desc).expect("get_encoder() should be OK"); - encoder.put(values).expect("put() should be OK"); - let encoded_values = encoder - .flush_buffer() - .expect("consume_buffer() should be OK"); - self.buffer.extend_from_slice(&encoded_values); - } - - fn add_indices(&mut self, indices: Bytes) { - self.encoding = Some(Encoding::RLE_DICTIONARY); - self.buffer.extend_from_slice(indices.as_ref()); - } - - fn consume(self) -> Page { - if self.datapage_v2 { - Page::DataPageV2 { - buf: Bytes::copy_from_slice(&self.buffer), - num_values: self.num_values, - encoding: self.encoding.unwrap(), - num_nulls: 0, /* set to dummy value - don't need this when reading - * data page */ - num_rows: self.num_values, /* also don't need this when reading - * data page */ - def_levels_byte_len: self.def_levels_byte_len, - rep_levels_byte_len: self.rep_levels_byte_len, - is_compressed: false, - statistics: None, // set to None, we do not need statistics for tests - } - } else { - Page::DataPage { - buf: Bytes::copy_from_slice(&self.buffer), - num_values: self.num_values, - encoding: self.encoding.unwrap(), - def_level_encoding: Encoding::RLE, - rep_level_encoding: Encoding::RLE, - statistics: None, // set to None, we do not need statistics for tests - } - } - } -} - -/// A utility page reader which stores pages in memory. -pub struct InMemoryPageReader> { - page_iter: P, -} - -impl> InMemoryPageReader

{ - pub fn new(pages: impl IntoIterator) -> Self { - Self { - page_iter: pages.into_iter(), - } - } -} - -impl + Send> PageReader for InMemoryPageReader

{ - fn get_next_page(&mut self) -> Result> { - Ok(self.page_iter.next()) - } - - fn peek_next_page(&mut self) -> Result> { - unimplemented!() - } - - fn skip_next_page(&mut self) -> Result<()> { - unimplemented!() - } -} - -impl + Send> Iterator for InMemoryPageReader

{ - type Item = Result; - - fn next(&mut self) -> Option { - self.get_next_page().transpose() - } -} - -/// A utility page iterator which stores page readers in memory, used for tests. -#[derive(Clone)] -pub struct InMemoryPageIterator>> { - page_reader_iter: I, -} - -impl>> InMemoryPageIterator { - pub fn new(pages: impl IntoIterator, IntoIter = I>) -> Self { - Self { - page_reader_iter: pages.into_iter(), - } - } -} - -impl>> Iterator for InMemoryPageIterator { - type Item = Result>; - - fn next(&mut self) -> Option { - self.page_reader_iter - .next() - .map(|x| Ok(Box::new(InMemoryPageReader::new(x)) as Box)) - } -} - -impl> + Send> PageIterator for InMemoryPageIterator {} - -#[allow(clippy::too_many_arguments)] -pub fn make_pages( - desc: ColumnDescPtr, - encoding: Encoding, - num_pages: usize, - levels_per_page: usize, - min: T::T, - max: T::T, - def_levels: &mut Vec, - rep_levels: &mut Vec, - values: &mut Vec, - pages: &mut VecDeque, - use_v2: bool, -) where - T::T: PartialOrd + SampleUniform + Copy, -{ - let mut num_values = 0; - let max_def_level = desc.max_def_level(); - let max_rep_level = desc.max_rep_level(); - - let mut dict_encoder = DictEncoder::::new(Arc::clone(&desc)); - - for i in 0..num_pages { - let mut num_values_cur_page = 0; - let level_range = i * levels_per_page..(i + 1) * levels_per_page; - - if max_def_level > 0 { - random_numbers_range(levels_per_page, 0, max_def_level + 1, def_levels); - for dl in &def_levels[level_range.clone()] { - if *dl == max_def_level { - num_values_cur_page += 1; - } - } - } else { - num_values_cur_page = levels_per_page; - } - if max_rep_level > 0 { - random_numbers_range(levels_per_page, 0, max_rep_level + 1, rep_levels); - } - random_numbers_range(num_values_cur_page, min, max, values); - - // Generate the current page - - let mut pb = - DataPageBuilderImpl::new(Arc::clone(&desc), num_values_cur_page as u32, use_v2); - if max_rep_level > 0 { - pb.add_rep_levels(max_rep_level, &rep_levels[level_range.clone()]); - } - if max_def_level > 0 { - pb.add_def_levels(max_def_level, &def_levels[level_range]); - } - - let value_range = num_values..num_values + num_values_cur_page; - match encoding { - Encoding::PLAIN_DICTIONARY | Encoding::RLE_DICTIONARY => { - let _ = dict_encoder.put(&values[value_range.clone()]); - let indices = dict_encoder - .write_indices() - .expect("write_indices() should be OK"); - pb.add_indices(indices); - } - Encoding::PLAIN => { - pb.add_values::(encoding, &values[value_range]); - } - enc => panic!("Unexpected encoding {enc}"), - } - - let data_page = pb.consume(); - pages.push_back(data_page); - num_values += num_values_cur_page; - } - - if encoding == Encoding::PLAIN_DICTIONARY || encoding == Encoding::RLE_DICTIONARY { - let dict = dict_encoder - .write_dict() - .expect("write_dict() should be OK"); - let dict_page = Page::DictionaryPage { - buf: dict, - num_values: dict_encoder.num_entries() as u32, - encoding: Encoding::RLE_DICTIONARY, - is_sorted: false, - }; - pages.push_front(dict_page); - } -} diff --git a/native/core/src/parquet/util/test_common/rand_gen.rs b/native/core/src/parquet/util/test_common/rand_gen.rs deleted file mode 100644 index 0116b2c8c8..0000000000 --- a/native/core/src/parquet/util/test_common/rand_gen.rs +++ /dev/null @@ -1,57 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use rand::{ - distr::{uniform::SampleUniform, Distribution, StandardUniform}, - rng, RngExt, -}; - -pub fn random_bytes(n: usize) -> Vec { - let mut result = vec![]; - let mut rng = rng(); - for _ in 0..n { - result.push(rng.random_range(0..255)); - } - result -} - -pub fn random_bools(n: usize) -> Vec { - let mut result = vec![]; - let mut rng = rng(); - for _ in 0..n { - result.push(rng.random::()); - } - result -} - -pub fn random_numbers(n: usize) -> Vec -where - StandardUniform: Distribution, -{ - let mut rng = rng(); - StandardUniform.sample_iter(&mut rng).take(n).collect() -} - -pub fn random_numbers_range(n: usize, low: T, high: T, result: &mut Vec) -where - T: PartialOrd + SampleUniform + Copy, -{ - let mut rng = rng(); - for _ in 0..n { - result.push(rng.random_range(low..high)); - } -} diff --git a/native/jni-bridge/src/errors.rs b/native/jni-bridge/src/errors.rs index 4fbcb28e2a..7bf4073c8d 100644 --- a/native/jni-bridge/src/errors.rs +++ b/native/jni-bridge/src/errors.rs @@ -718,7 +718,8 @@ mod tests { init(); INIT.call_once(|| { - // Add common classes to the classpath in so that we can find CometException + // Add comet-common classes to the classpath so we can find the Comet exception + // classes (CometNativeException, CometQueryExecutionException, etc.). let mut common_classes = PathBuf::from(env!("CARGO_MANIFEST_DIR")); common_classes.push("../../common/target/classes"); let mut class_path = common_classes diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 7cefe06da7..ed1684b240 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -121,6 +121,12 @@ message NativeScanCommon { bool use_field_id = 15; // True when spark.sql.parquet.fieldId.read.ignoreMissing is set. bool ignore_missing_field_id = 16; + // Whether widening type promotion is allowed (e.g. INT32 -> INT64, + // FLOAT -> DOUBLE). Set from Comet's per-Spark-version constant in + // ShimCometConf (false on 3.x, true on 4.x). When false, reading a column + // with a disallowed promoted type throws an error matching Spark's + // SchemaColumnConvertNotSupportedException behavior. + bool allow_type_promotion = 17; } message NativeScan { @@ -355,6 +361,8 @@ message Explode { bool outer = 2; // Expressions for other columns to project alongside the exploded values repeated spark.spark_expression.Expr project_list = 3; + // Whether to emit a position column alongside the exploded values (posexplode) + bool position = 4; } message HashJoin { diff --git a/native/proto/src/proto/types.proto b/native/proto/src/proto/types.proto index fec972a8f0..df0c0c5553 100644 --- a/native/proto/src/proto/types.proto +++ b/native/proto/src/proto/types.proto @@ -59,6 +59,7 @@ message DataType { LIST = 14; MAP = 15; STRUCT = 16; + TIME = 17; } DataTypeId type_id = 1; diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 505879e319..d1cc204717 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -43,7 +43,7 @@ futures = { workspace = true } itertools = "0.14.0" jni = "0.21" log = "0.4" -lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] } +lz4_flex = { version = "0.13.1", default-features = false, features = ["frame"] } # parquet is only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } simd-adler32 = "0.3.9" diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 8502c79624..a5de73ae00 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -25,7 +25,8 @@ use crate::partitioners::{ use crate::{CometPartitioning, CompressionCodec}; use async_trait::async_trait; use datafusion::common::exec_datafusion_err; -use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; +use datafusion::common::tree_node::TreeNodeRecursion; +use datafusion::physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::EmptyRecordBatchStream; use datafusion::{ @@ -40,7 +41,6 @@ use datafusion::{ }; use futures::{StreamExt, TryFutureExt, TryStreamExt}; use std::{ - any::Any, fmt, fmt::{Debug, Formatter}, sync::Arc, @@ -119,11 +119,6 @@ impl DisplayAs for ShuffleWriterExec { #[async_trait] impl ExecutionPlan for ShuffleWriterExec { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "ShuffleWriterExec" } @@ -145,6 +140,27 @@ impl ExecutionPlan for ShuffleWriterExec { vec![&self.input] } + fn apply_expressions( + &self, + f: &mut dyn FnMut(&dyn PhysicalExpr) -> Result, + ) -> Result { + let mut tnr = TreeNodeRecursion::Continue; + match &self.partitioning { + CometPartitioning::Hash(exprs, _) => { + for expr in exprs { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } + } + CometPartitioning::RangePartitioning(ordering, _, _, _) => { + for sort_expr in ordering.iter() { + tnr = tnr.visit_sibling(|| f(sort_expr.expr.as_ref()))?; + } + } + CometPartitioning::SinglePartition | CometPartitioning::RoundRobin(_, _) => {} + } + Ok(tnr) + } + fn with_new_children( self: Arc, children: Vec>, diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 33ffc1c886..1b0359059c 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -118,3 +118,7 @@ harness = false [[bench]] name = "map_sort" harness = false + +[[bench]] +name = "to_time" +harness = false diff --git a/native/spark-expr/benches/to_time.rs b/native/spark-expr/benches/to_time.rs new file mode 100644 index 0000000000..81815fd980 --- /dev/null +++ b/native/spark-expr/benches/to_time.rs @@ -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. + +use arrow::array::StringArray; +use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion::physical_plan::ColumnarValue; +use datafusion_comet_spark_expr::spark_to_time; +use std::sync::Arc; + +fn criterion_benchmark(c: &mut Criterion) { + let mut group = c.benchmark_group("to_time"); + + let hh_mm = create_string_array(10000, |i| format!("{}:{:02}", i % 24, i % 60)); + group.bench_function("hh_mm", |b| { + b.iter(|| spark_to_time(std::slice::from_ref(&hh_mm), true).unwrap()); + }); + + let hh_mm_ss = + create_string_array(10000, |i| format!("{}:{:02}:{:02}", i % 24, i % 60, i % 60)); + group.bench_function("hh_mm_ss", |b| { + b.iter(|| spark_to_time(std::slice::from_ref(&hh_mm_ss), true).unwrap()); + }); + + let fractional = create_string_array(10000, |i| { + format!( + "{}:{:02}:{:02}.{:06}", + i % 24, + i % 60, + i % 60, + i * 7 % 1000000 + ) + }); + group.bench_function("fractional", |b| { + b.iter(|| spark_to_time(std::slice::from_ref(&fractional), true).unwrap()); + }); + + let am_pm = create_string_array(10000, |i| { + let hour = (i % 12) + 1; + let suffix = if i % 2 == 0 { "AM" } else { "PM" }; + format!("{}:{:02}:{:02} {}", hour, i % 60, i % 60, suffix) + }); + group.bench_function("am_pm", |b| { + b.iter(|| spark_to_time(std::slice::from_ref(&am_pm), true).unwrap()); + }); + + group.finish(); +} + +fn create_string_array(size: usize, f: impl Fn(usize) -> String) -> ColumnarValue { + let values: Vec = (0..size).map(&f).collect(); + let array = StringArray::from(values); + ColumnarValue::Array(Arc::new(array)) +} + +fn config() -> Criterion { + Criterion::default() +} + +criterion_group! { + name = benches; + config = config(); + targets = criterion_benchmark +} +criterion_main!(benches); diff --git a/native/spark-expr/src/agg_funcs/avg.rs b/native/spark-expr/src/agg_funcs/avg.rs index 3760b42504..28c02b6306 100644 --- a/native/spark-expr/src/agg_funcs/avg.rs +++ b/native/spark-expr/src/agg_funcs/avg.rs @@ -28,7 +28,7 @@ use datafusion::logical_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; use datafusion::physical_expr::expressions::format_state_name; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; use arrow::array::ArrowNativeTypeOp; use datafusion::logical_expr::function::{AccumulatorArgs, StateFieldsArgs}; @@ -67,11 +67,6 @@ impl Avg { } impl AggregateUDFImpl for Avg { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { // All numeric types use Float64 accumulation after casting match (&self.input_data_type, &self.result_data_type) { @@ -238,8 +233,8 @@ where impl GroupsAccumulator for AvgGroupsAccumulator where - T: ArrowNumericType + Send, - F: Fn(T::Native, i64) -> Result + Send, + T: ArrowNumericType + Send + 'static, + F: Fn(T::Native, i64) -> Result + Send + 'static, { fn update_batch( &mut self, diff --git a/native/spark-expr/src/agg_funcs/avg_decimal.rs b/native/spark-expr/src/agg_funcs/avg_decimal.rs index 05d74e1f99..a71bbe44ea 100644 --- a/native/spark-expr/src/agg_funcs/avg_decimal.rs +++ b/native/spark-expr/src/agg_funcs/avg_decimal.rs @@ -28,7 +28,7 @@ use datafusion::logical_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; use datafusion::physical_expr::expressions::format_state_name; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; use crate::utils::{build_bool_state, is_valid_decimal_precision, unlikely}; use crate::{decimal_sum_overflow_error, EvalMode, SparkErrorWithContext}; @@ -108,11 +108,6 @@ impl AvgDecimal { } impl AggregateUDFImpl for AvgDecimal { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { match (&self.sum_data_type, &self.result_data_type) { (Decimal128(sum_precision, sum_scale), Decimal128(target_precision, target_scale)) => { diff --git a/native/spark-expr/src/agg_funcs/correlation.rs b/native/spark-expr/src/agg_funcs/correlation.rs index 499ef2aa0c..b47ce0ba60 100644 --- a/native/spark-expr/src/agg_funcs/correlation.rs +++ b/native/spark-expr/src/agg_funcs/correlation.rs @@ -15,21 +15,20 @@ // specific language governing permissions and limitations // under the License. -use arrow::compute::{and, filter, is_not_null}; +use arrow::array::{Array, ArrayRef, BooleanArray, Float64Array}; +use arrow::compute::{and, is_not_null}; +use arrow::datatypes::{DataType, Field, FieldRef}; +use std::sync::Arc; -use std::{any::Any, sync::Arc}; - -use crate::agg_funcs::covariance::CovarianceAccumulator; +use crate::agg_funcs::covariance::{CovarianceAccumulator, CovarianceGroupsAccumulator}; use crate::agg_funcs::stddev::StddevAccumulator; -use arrow::datatypes::FieldRef; -use arrow::{ - array::ArrayRef, - datatypes::{DataType, Field}, -}; +use crate::agg_funcs::variance::VarianceGroupsAccumulator; +use arrow::compute::filter; use datafusion::common::{Result, ScalarValue}; use datafusion::logical_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion::logical_expr::type_coercion::aggregates::NUMERICS; -use datafusion::logical_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; +use datafusion::logical_expr::{ + Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, Signature, Volatility, +}; use datafusion::physical_expr::expressions::format_state_name; use datafusion::physical_expr::expressions::StatsType; @@ -51,18 +50,13 @@ impl Correlation { assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), - signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), + signature: Signature::numeric(2, Volatility::Immutable), null_on_divide_by_zero, } } } impl AggregateUDFImpl for Correlation { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { &self.name } @@ -118,6 +112,19 @@ impl AggregateUDFImpl for Correlation { )), ]) } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(CorrelationGroupsAccumulator::new( + self.null_on_divide_by_zero, + ))) + } } /// An accumulator to compute correlation @@ -248,3 +255,274 @@ impl Accumulator for CorrelationAccumulator { + self.stddev2.size() } } + +/// Grouped correlation accumulator. Mirrors the per-row `CorrelationAccumulator` +/// composition: one population covariance + two population variance sub- +/// accumulators for the per-column m2 values. Combined filter handles +/// "skip rows where either input is null" without filtering `group_indices`. +#[derive(Debug)] +struct CorrelationGroupsAccumulator { + covar: CovarianceGroupsAccumulator, + var1: VarianceGroupsAccumulator, + var2: VarianceGroupsAccumulator, + null_on_divide_by_zero: bool, +} + +impl CorrelationGroupsAccumulator { + fn new(null_on_divide_by_zero: bool) -> Self { + // Children run with StatsType::Population, which never hits the + // count == 1 sample-divide-by-zero branch, so the children's + // null_on_divide_by_zero is dead code. The top-level evaluate() + // applies the count <= 1 rule. Pass `false` to children to keep + // that intent explicit. + Self { + covar: CovarianceGroupsAccumulator::new(StatsType::Population, false), + var1: VarianceGroupsAccumulator::new(StatsType::Population, false), + var2: VarianceGroupsAccumulator::new(StatsType::Population, false), + null_on_divide_by_zero, + } + } +} + +impl GroupsAccumulator for CorrelationGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to update_batch"); + // Fast path: when both inputs are fully non-null, skip the + // is_not_null/and combination and forward the caller's filter as-is. + // Mirrors the per-row CorrelationAccumulator short-circuit. + let dense = values[0].null_count() == 0 && values[1].null_count() == 0; + let combined: Option = if dense { + None + } else { + let null_mask = and(&is_not_null(&values[0])?, &is_not_null(&values[1])?)?; + Some(match opt_filter { + Some(f) => and(f, &null_mask)?, + None => null_mask, + }) + }; + let filter_for_children: Option<&BooleanArray> = match (&combined, opt_filter) { + (Some(c), _) => Some(c), + (None, f) => f, + }; + + self.covar + .update_batch(values, group_indices, filter_for_children, total_num_groups)?; + self.var1.update_batch( + &values[0..1], + group_indices, + filter_for_children, + total_num_groups, + )?; + self.var2.update_batch( + &values[1..2], + group_indices, + filter_for_children, + total_num_groups, + )?; + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 6, "six state columns to merge_batch"); + // state column order: count, mean1, mean2, algo_const, m2_1, m2_2 + let covar_state = [ + Arc::clone(&values[0]), + Arc::clone(&values[1]), + Arc::clone(&values[2]), + Arc::clone(&values[3]), + ]; + let var1_state = [ + Arc::clone(&values[0]), + Arc::clone(&values[1]), + Arc::clone(&values[4]), + ]; + let var2_state = [ + Arc::clone(&values[0]), + Arc::clone(&values[2]), + Arc::clone(&values[5]), + ]; + + self.covar + .merge_batch(&covar_state, group_indices, opt_filter, total_num_groups)?; + self.var1 + .merge_batch(&var1_state, group_indices, opt_filter, total_num_groups)?; + self.var2 + .merge_batch(&var2_state, group_indices, opt_filter, total_num_groups)?; + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + // All three children see the same rows (combined null-mask in + // update_batch / shared count column on merge), so counts and means + // are duplicated. We only need: covar.counts, covar.algo_consts, + // var1.m2s, var2.m2s. Drain everything once and compute correlation + // inline rather than calling each child's evaluate() (which would + // allocate three Float64Arrays we'd then unpack and discard). + let counts = emit_to.take_needed(&mut self.covar.counts); + let _ = emit_to.take_needed(&mut self.covar.mean1s); + let _ = emit_to.take_needed(&mut self.covar.mean2s); + let algo_consts = emit_to.take_needed(&mut self.covar.algo_consts); + let _ = emit_to.take_needed(&mut self.var1.counts); + let _ = emit_to.take_needed(&mut self.var1.means); + let m2_1s = emit_to.take_needed(&mut self.var1.m2s); + let _ = emit_to.take_needed(&mut self.var2.counts); + let _ = emit_to.take_needed(&mut self.var2.means); + let m2_2s = emit_to.take_needed(&mut self.var2.m2s); + + let n = counts.len(); + let mut values = Vec::with_capacity(n); + let mut validity = Vec::with_capacity(n); + for i in 0..n { + let count = counts[i]; + if count == 0.0 { + values.push(0.0); + validity.push(false); + continue; + } + if count == 1.0 { + if self.null_on_divide_by_zero { + values.push(0.0); + validity.push(false); + } else { + values.push(f64::NAN); + validity.push(true); + } + continue; + } + // Population stats: divide m2 / count, c / count. The 1/count + // factors cancel in c / (s1 * s2), so we work with raw moments. + let s1_sq = m2_1s[i]; + let s2_sq = m2_2s[i]; + if s1_sq == 0.0 || s2_sq == 0.0 { + values.push(0.0); + validity.push(false); + continue; + } + values.push(algo_consts[i] / (s1_sq * s2_sq).sqrt()); + validity.push(true); + } + + Ok(Arc::new(Float64Array::new( + values.into(), + Some(arrow::buffer::NullBuffer::from(validity)), + ))) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + // covar.state -> [count, mean1, mean2, algo_const] + // var1.state -> [count, mean1, m2_1] + // var2.state -> [count, mean2, m2_2] + // Combined state (matches Correlation::state_fields): + // [count, mean1, mean2, algo_const, m2_1, m2_2] + let covar_state = self.covar.state(emit_to)?; + let var1_state = self.var1.state(emit_to)?; + let var2_state = self.var2.state(emit_to)?; + Ok(vec![ + Arc::clone(&covar_state[0]), + Arc::clone(&covar_state[1]), + Arc::clone(&covar_state[2]), + Arc::clone(&covar_state[3]), + Arc::clone(&var1_state[2]), + Arc::clone(&var2_state[2]), + ]) + } + + fn size(&self) -> usize { + self.covar.size() + self.var1.size() + self.var2.size() + } +} + +#[cfg(test)] +mod groups_tests { + use super::*; + use arrow::array::AsArray; + use arrow::datatypes::Float64Type; + + fn acc(legacy: bool) -> CorrelationGroupsAccumulator { + // null_on_divide_by_zero = !legacy + CorrelationGroupsAccumulator::new(!legacy) + } + + fn evaluate(a: &mut CorrelationGroupsAccumulator) -> Vec> { + a.evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect() + } + + #[test] + fn perfectly_correlated_single_group() { + let mut a = acc(true); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![2.0, 4.0, 6.0, 8.0, 10.0])); + a.update_batch(&[v1, v2], &[0, 0, 0, 0, 0], None, 1) + .unwrap(); + let r = evaluate(&mut a); + assert!((r[0].unwrap() - 1.0).abs() < 1e-12); + } + + #[test] + fn either_column_null_dropped() { + let mut a = acc(true); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(1.0), + None, + Some(3.0), + Some(5.0), + ])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(2.0), + Some(99.0), + None, + Some(10.0), + ])); + a.update_batch(&[v1, v2], &[0, 0, 0, 0], None, 1).unwrap(); + // surviving pairs (1,2) and (5,10) lie on y=2x => corr 1.0 + assert!((evaluate(&mut a)[0].unwrap() - 1.0).abs() < 1e-12); + } + + #[test] + fn empty_group_yields_null() { + let mut a = acc(true); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![3.0, 6.0])); + a.update_batch(&[v1, v2], &[0, 0], None, 2).unwrap(); + assert_eq!(evaluate(&mut a)[1], None); + } + + #[test] + fn single_row_legacy_mode_yields_nan() { + // Correlation always uses Population stats internally. With one row + // the per-row CorrelationAccumulator returns NaN when in legacy + // (null_on_divide_by_zero=false) mode and null when the flag is set. + let mut a = acc(true); // legacy + let v1: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![7.0])); + a.update_batch(&[v1, v2], &[0], None, 1).unwrap(); + let r = evaluate(&mut a); + assert!(r[0].unwrap().is_nan()); + } + + #[test] + fn single_row_ansi_mode_yields_null() { + let mut a = acc(false); // null_on_divide_by_zero = true + let v1: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![7.0])); + a.update_batch(&[v1, v2], &[0], None, 1).unwrap(); + let r = evaluate(&mut a); + assert_eq!(r[0], None); + } +} diff --git a/native/spark-expr/src/agg_funcs/covariance.rs b/native/spark-expr/src/agg_funcs/covariance.rs index 15759eb155..7007bc028e 100644 --- a/native/spark-expr/src/agg_funcs/covariance.rs +++ b/native/spark-expr/src/agg_funcs/covariance.rs @@ -17,19 +17,18 @@ * under the License. */ -use arrow::datatypes::FieldRef; -use arrow::{ - array::{ArrayRef, Float64Array}, - compute::cast, - datatypes::{DataType, Field}, -}; +use arrow::array::{Array, ArrayRef, AsArray, BooleanArray, Float64Array}; +use arrow::buffer::NullBuffer; +use arrow::compute::cast; +use arrow::datatypes::{DataType, Field, FieldRef, Float64Type}; use datafusion::common::{downcast_value, unwrap_or_internal_err, Result, ScalarValue}; use datafusion::logical_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion::logical_expr::type_coercion::aggregates::NUMERICS; -use datafusion::logical_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; +use datafusion::logical_expr::{ + Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, Signature, Volatility, +}; use datafusion::physical_expr::expressions::format_state_name; use datafusion::physical_expr::expressions::StatsType; -use std::any::Any; +use std::mem::size_of; use std::sync::Arc; /// COVAR_SAMP and COVAR_POP aggregate expression @@ -65,7 +64,7 @@ impl Covariance { assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), - signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), + signature: Signature::numeric(2, Volatility::Immutable), stats_type, null_on_divide_by_zero, } @@ -73,11 +72,6 @@ impl Covariance { } impl AggregateUDFImpl for Covariance { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { &self.name } @@ -124,6 +118,20 @@ impl AggregateUDFImpl for Covariance { )), ]) } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(CovarianceGroupsAccumulator::new( + self.stats_type, + self.null_on_divide_by_zero, + ))) + } } /// An accumulator to compute covariance @@ -202,17 +210,19 @@ impl Accumulator for CovarianceAccumulator { let value1 = unwrap_or_internal_err!(value1); let value2 = unwrap_or_internal_err!(value2); - let new_count = self.count + 1.0; - let delta1 = value1 - self.mean1; - let new_mean1 = delta1 / new_count + self.mean1; - let delta2 = value2 - self.mean2; - let new_mean2 = delta2 / new_count + self.mean2; - let new_c = delta1 * (value2 - new_mean2) + self.algo_const; - - self.count += 1.0; - self.mean1 = new_mean1; - self.mean2 = new_mean2; - self.algo_const = new_c; + + let (c, m1, m2, ac) = super::welford::covariance_update( + self.count, + self.mean1, + self.mean2, + self.algo_const, + value1, + value2, + ); + self.count = c; + self.mean1 = m1; + self.mean2 = m2; + self.algo_const = ac; } Ok(()) @@ -243,17 +253,18 @@ impl Accumulator for CovarianceAccumulator { let value1 = unwrap_or_internal_err!(value1); let value2 = unwrap_or_internal_err!(value2); - let new_count = self.count - 1.0; - let delta1 = self.mean1 - value1; - let new_mean1 = delta1 / new_count + self.mean1; - let delta2 = self.mean2 - value2; - let new_mean2 = delta2 / new_count + self.mean2; - let new_c = self.algo_const - delta1 * (new_mean2 - value2); - - self.count -= 1.0; - self.mean1 = new_mean1; - self.mean2 = new_mean2; - self.algo_const = new_c; + let (c, m1, m2, ac) = super::welford::covariance_retract( + self.count, + self.mean1, + self.mean2, + self.algo_const, + value1, + value2, + ); + self.count = c; + self.mean1 = m1; + self.mean2 = m2; + self.algo_const = ac; } Ok(()) @@ -270,14 +281,16 @@ impl Accumulator for CovarianceAccumulator { if c == 0.0 { continue; } - let new_count = self.count + c; - let new_mean1 = self.mean1 * self.count / new_count + means1.value(i) * c / new_count; - let new_mean2 = self.mean2 * self.count / new_count + means2.value(i) * c / new_count; - let delta1 = self.mean1 - means1.value(i); - let delta2 = self.mean2 - means2.value(i); - let new_c = - self.algo_const + cs.value(i) + delta1 * delta2 * self.count * c / new_count; - + let (new_count, new_mean1, new_mean2, new_c) = super::welford::covariance_merge( + self.count, + self.mean1, + self.mean2, + self.algo_const, + c, + means1.value(i), + means2.value(i), + cs.value(i), + ); self.count = new_count; self.mean1 = new_mean1; self.mean2 = new_mean2; @@ -311,3 +324,255 @@ impl Accumulator for CovarianceAccumulator { std::mem::size_of_val(self) } } + +/// Vectorized grouped covariance accumulator. Counts are `f64` to match the +/// Spark wire format used by the per-row `CovarianceAccumulator`. +#[derive(Debug)] +pub(crate) struct CovarianceGroupsAccumulator { + pub(super) counts: Vec, + pub(super) mean1s: Vec, + pub(super) mean2s: Vec, + pub(super) algo_consts: Vec, + stats_type: StatsType, + null_on_divide_by_zero: bool, +} + +impl CovarianceGroupsAccumulator { + pub(crate) fn new(stats_type: StatsType, null_on_divide_by_zero: bool) -> Self { + Self { + counts: Vec::new(), + mean1s: Vec::new(), + mean2s: Vec::new(), + algo_consts: Vec::new(), + stats_type, + null_on_divide_by_zero, + } + } + + fn resize(&mut self, total_num_groups: usize) { + self.counts.resize(total_num_groups, 0.0); + self.mean1s.resize(total_num_groups, 0.0); + self.mean2s.resize(total_num_groups, 0.0); + self.algo_consts.resize(total_num_groups, 0.0); + } + + fn finalize(&mut self, emit_to: EmitTo) -> (Vec, NullBuffer) { + let counts = emit_to.take_needed(&mut self.counts); + let _ = emit_to.take_needed(&mut self.mean1s); + let _ = emit_to.take_needed(&mut self.mean2s); + let cs = emit_to.take_needed(&mut self.algo_consts); + super::welford::finalize_moments(counts, cs, self.stats_type, self.null_on_divide_by_zero) + } +} + +impl GroupsAccumulator for CovarianceGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 2, "two arguments to update_batch"); + let v1 = cast(&values[0], &DataType::Float64)?; + let v2 = cast(&values[1], &DataType::Float64)?; + let v1 = v1.as_primitive::(); + let v2 = v2.as_primitive::(); + + self.resize(total_num_groups); + + for (idx, &group_index) in group_indices.iter().enumerate() { + if let Some(f) = opt_filter { + if !f.is_valid(idx) || !f.value(idx) { + continue; + } + } + if v1.is_null(idx) || v2.is_null(idx) { + continue; + } + let (c, m1, m2, ac) = super::welford::covariance_update( + self.counts[group_index], + self.mean1s[group_index], + self.mean2s[group_index], + self.algo_consts[group_index], + v1.value(idx), + v2.value(idx), + ); + self.counts[group_index] = c; + self.mean1s[group_index] = m1; + self.mean2s[group_index] = m2; + self.algo_consts[group_index] = ac; + } + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 4, "four arguments to merge_batch"); + let counts = downcast_value!(values[0], Float64Array); + let means1 = downcast_value!(values[1], Float64Array); + let means2 = downcast_value!(values[2], Float64Array); + let cs = downcast_value!(values[3], Float64Array); + + self.resize(total_num_groups); + + for (i, &group_index) in group_indices.iter().enumerate() { + let partial_count = counts.value(i); + if partial_count == 0.0 { + continue; + } + let (new_count, new_m1, new_m2, new_c) = super::welford::covariance_merge( + self.counts[group_index], + self.mean1s[group_index], + self.mean2s[group_index], + self.algo_consts[group_index], + partial_count, + means1.value(i), + means2.value(i), + cs.value(i), + ); + self.counts[group_index] = new_count; + self.mean1s[group_index] = new_m1; + self.mean2s[group_index] = new_m2; + self.algo_consts[group_index] = new_c; + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let (values, nulls) = self.finalize(emit_to); + Ok(Arc::new(Float64Array::new(values.into(), Some(nulls)))) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + let counts = emit_to.take_needed(&mut self.counts); + let mean1s = emit_to.take_needed(&mut self.mean1s); + let mean2s = emit_to.take_needed(&mut self.mean2s); + let cs = emit_to.take_needed(&mut self.algo_consts); + Ok(vec![ + Arc::new(Float64Array::new(counts.into(), None)), + Arc::new(Float64Array::new(mean1s.into(), None)), + Arc::new(Float64Array::new(mean2s.into(), None)), + Arc::new(Float64Array::new(cs.into(), None)), + ]) + } + + fn size(&self) -> usize { + (self.counts.capacity() + + self.mean1s.capacity() + + self.mean2s.capacity() + + self.algo_consts.capacity()) + * size_of::() + } +} + +#[cfg(test)] +mod groups_tests { + use super::*; + use arrow::array::{AsArray, Float64Array}; + + fn pop() -> CovarianceGroupsAccumulator { + CovarianceGroupsAccumulator::new(StatsType::Population, false) + } + + fn evaluate(acc: &mut CovarianceGroupsAccumulator) -> Vec> { + acc.evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect() + } + + #[test] + fn pop_covariance_single_group() { + let mut acc = pop(); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![2.0, 4.0, 6.0, 8.0, 10.0])); + acc.update_batch(&[v1, v2], &[0, 0, 0, 0, 0], None, 1) + .unwrap(); + // pop covariance of x and 2x for x in [1..5] = 4.0 + assert!((evaluate(&mut acc)[0].unwrap() - 4.0).abs() < 1e-12); + } + + #[test] + fn null_in_either_column_skipped() { + let mut acc = pop(); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(1.0), + None, + Some(3.0), + Some(5.0), + ])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(2.0), + Some(99.0), + None, + Some(10.0), + ])); + acc.update_batch(&[v1, v2], &[0, 0, 0, 0], None, 1).unwrap(); + // surviving pairs (1,2) and (5,10): mean(x)=3, mean(y)=6 + // pop covar = ((1-3)(2-6) + (5-3)(10-6))/2 = (8+8)/2 = 8.0 + assert!((evaluate(&mut acc)[0].unwrap() - 8.0).abs() < 1e-12); + } + + #[test] + fn merge_matches_singleshot() { + let single = { + let mut a = pop(); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0, 6.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![2.0, 4.0, 6.0, 8.0, 10.0, 12.0])); + a.update_batch(&[v1, v2], &[0; 6], None, 1).unwrap(); + evaluate(&mut a)[0].unwrap() + }; + + let mut left = pop(); + left.update_batch( + &[ + Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0])) as ArrayRef, + Arc::new(Float64Array::from(vec![2.0, 4.0, 6.0])) as ArrayRef, + ], + &[0, 0, 0], + None, + 1, + ) + .unwrap(); + let lstate = left.state(EmitTo::All).unwrap(); + + let mut right = pop(); + right + .update_batch( + &[ + Arc::new(Float64Array::from(vec![4.0, 5.0, 6.0])) as ArrayRef, + Arc::new(Float64Array::from(vec![8.0, 10.0, 12.0])) as ArrayRef, + ], + &[0, 0, 0], + None, + 1, + ) + .unwrap(); + let rstate = right.state(EmitTo::All).unwrap(); + + let mut merged = pop(); + merged.merge_batch(&lstate, &[0], None, 1).unwrap(); + merged.merge_batch(&rstate, &[0], None, 1).unwrap(); + let merged_result = evaluate(&mut merged)[0].unwrap(); + + assert!((single - merged_result).abs() < 1e-12); + } + + #[test] + fn empty_group_yields_null() { + let mut acc = pop(); + let v1: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0])); + let v2: ArrayRef = Arc::new(Float64Array::from(vec![10.0, 20.0])); + acc.update_batch(&[v1, v2], &[0, 0], None, 2).unwrap(); + assert_eq!(evaluate(&mut acc)[1], None); + } +} diff --git a/native/spark-expr/src/agg_funcs/mod.rs b/native/spark-expr/src/agg_funcs/mod.rs index b1027153e8..2a0322e46c 100644 --- a/native/spark-expr/src/agg_funcs/mod.rs +++ b/native/spark-expr/src/agg_funcs/mod.rs @@ -23,6 +23,7 @@ mod stddev; mod sum_decimal; mod sum_int; mod variance; +mod welford; pub use avg::Avg; pub use avg_decimal::AvgDecimal; diff --git a/native/spark-expr/src/agg_funcs/stddev.rs b/native/spark-expr/src/agg_funcs/stddev.rs index b231b8afa7..3389a86af6 100644 --- a/native/spark-expr/src/agg_funcs/stddev.rs +++ b/native/spark-expr/src/agg_funcs/stddev.rs @@ -15,18 +15,18 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, sync::Arc}; +use std::sync::Arc; -use crate::agg_funcs::variance::VarianceAccumulator; +use crate::agg_funcs::variance::{VarianceAccumulator, VarianceGroupsAccumulator}; +use arrow::array::{ArrayRef, AsArray, BooleanArray, Float64Array}; use arrow::datatypes::FieldRef; -use arrow::{ - array::ArrayRef, - datatypes::{DataType, Field}, -}; +use arrow::datatypes::{DataType, Field, Float64Type}; use datafusion::common::types::NativeType; use datafusion::common::{internal_err, Result, ScalarValue}; use datafusion::logical_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion::logical_expr::{Accumulator, AggregateUDFImpl, Coercion, Signature, Volatility}; +use datafusion::logical_expr::{ + Accumulator, AggregateUDFImpl, Coercion, EmitTo, GroupsAccumulator, Signature, Volatility, +}; use datafusion::logical_expr_common::signature; use datafusion::physical_expr::expressions::format_state_name; use datafusion::physical_expr::expressions::StatsType; @@ -78,11 +78,6 @@ impl Stddev { } impl AggregateUDFImpl for Stddev { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { &self.name } @@ -135,6 +130,20 @@ impl AggregateUDFImpl for Stddev { fn default_value(&self, _data_type: &DataType) -> Result { Ok(ScalarValue::Float64(None)) } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(StddevGroupsAccumulator::new( + self.stats_type, + self.null_on_divide_by_zero, + ))) + } } /// An accumulator to compute the standard deviation @@ -190,3 +199,126 @@ impl Accumulator for StddevAccumulator { std::mem::align_of_val(self) - std::mem::align_of_val(&self.variance) + self.variance.size() } } + +/// Stddev grouped accumulator: wraps a `VarianceGroupsAccumulator` and applies +/// `sqrt` element-wise on `evaluate`. State is identical to variance. +#[derive(Debug)] +struct StddevGroupsAccumulator { + inner: VarianceGroupsAccumulator, +} + +impl StddevGroupsAccumulator { + fn new(stats_type: StatsType, null_on_divide_by_zero: bool) -> Self { + Self { + inner: VarianceGroupsAccumulator::new(stats_type, null_on_divide_by_zero), + } + } +} + +impl GroupsAccumulator for StddevGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.inner + .update_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.inner + .merge_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let arr = self.inner.evaluate(emit_to)?; + // Run sqrt across the buffer in place via PrimitiveArray::unary + // rather than allocating an intermediate Vec. + let sqrted: Float64Array = arr.as_primitive::().unary(|v| v.sqrt()); + Ok(Arc::new(sqrted)) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + self.inner.state(emit_to) + } + + fn size(&self) -> usize { + self.inner.size() + } +} + +#[cfg(test)] +mod groups_tests { + use super::*; + use arrow::array::AsArray; + use arrow::datatypes::Float64Type; + + #[test] + fn pop_stddev_single_group() { + let mut acc = StddevGroupsAccumulator::new(StatsType::Population, false); + let values: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0])); + acc.update_batch(&[values], &[0, 0, 0, 0, 0], None, 1) + .unwrap(); + // sqrt(2.0) + let result: Vec> = acc + .evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect(); + assert!((result[0].unwrap() - 2.0_f64.sqrt()).abs() < 1e-12); + } + + #[test] + fn empty_group_yields_null() { + let mut acc = StddevGroupsAccumulator::new(StatsType::Population, false); + let values: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0])); + acc.update_batch(&[values], &[0, 0], None, 2).unwrap(); + let result: Vec> = acc + .evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect(); + assert_eq!(result[1], None); + } + + #[test] + fn sample_single_row_nan_legacy() { + // Stddev wraps variance, but pin the contract here too: legacy mode + // (null_on_divide_by_zero = false) emits sqrt(NaN) = NaN for a single + // sample-stddev row. + let mut acc = StddevGroupsAccumulator::new(StatsType::Sample, false); + let values: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); + acc.update_batch(&[values], &[0], None, 1).unwrap(); + let result: Vec> = acc + .evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect(); + assert!(result[0].unwrap().is_nan()); + } + + #[test] + fn sample_single_row_null_when_flag_set() { + let mut acc = StddevGroupsAccumulator::new(StatsType::Sample, true); + let values: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); + acc.update_batch(&[values], &[0], None, 1).unwrap(); + let result: Vec> = acc + .evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect(); + assert_eq!(result[0], None); + } +} diff --git a/native/spark-expr/src/agg_funcs/sum_decimal.rs b/native/spark-expr/src/agg_funcs/sum_decimal.rs index 180e884bf0..4ddf13c5d6 100644 --- a/native/spark-expr/src/agg_funcs/sum_decimal.rs +++ b/native/spark-expr/src/agg_funcs/sum_decimal.rs @@ -27,7 +27,7 @@ use datafusion::logical_expr::Volatility::Immutable; use datafusion::logical_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; #[derive(Debug)] pub struct SumDecimal { @@ -99,10 +99,6 @@ impl SumDecimal { } impl AggregateUDFImpl for SumDecimal { - fn as_any(&self) -> &dyn Any { - self - } - fn accumulator(&self, _args: AccumulatorArgs) -> DFResult> { Ok(Box::new(SumDecimalAccumulator::new( self.precision, diff --git a/native/spark-expr/src/agg_funcs/sum_int.rs b/native/spark-expr/src/agg_funcs/sum_int.rs index 5ce6066772..59a4371772 100644 --- a/native/spark-expr/src/agg_funcs/sum_int.rs +++ b/native/spark-expr/src/agg_funcs/sum_int.rs @@ -29,7 +29,7 @@ use datafusion::logical_expr::Volatility::Immutable; use datafusion::logical_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; #[derive(Debug, PartialEq, Eq, Hash)] pub struct SumInteger { @@ -52,10 +52,6 @@ impl SumInteger { } impl AggregateUDFImpl for SumInteger { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "sum" } diff --git a/native/spark-expr/src/agg_funcs/variance.rs b/native/spark-expr/src/agg_funcs/variance.rs index c97e664dd6..f1841729cf 100644 --- a/native/spark-expr/src/agg_funcs/variance.rs +++ b/native/spark-expr/src/agg_funcs/variance.rs @@ -15,18 +15,18 @@ // specific language governing permissions and limitations // under the License. -use arrow::datatypes::FieldRef; -use arrow::{ - array::{ArrayRef, Float64Array}, - datatypes::{DataType, Field}, -}; +use arrow::array::{Array, ArrayRef, AsArray, BooleanArray, Float64Array}; +use arrow::buffer::NullBuffer; +use arrow::datatypes::{DataType, Field, FieldRef, Float64Type}; use datafusion::common::{downcast_value, Result, ScalarValue}; use datafusion::logical_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion::logical_expr::Volatility::Immutable; -use datafusion::logical_expr::{Accumulator, AggregateUDFImpl, Signature}; +use datafusion::logical_expr::{ + Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, Signature, +}; use datafusion::physical_expr::expressions::format_state_name; use datafusion::physical_expr::expressions::StatsType; -use std::any::Any; +use std::mem::size_of; use std::sync::Arc; /// VAR_SAMP and VAR_POP aggregate expression @@ -71,11 +71,6 @@ impl Variance { } impl AggregateUDFImpl for Variance { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { &self.name } @@ -125,6 +120,20 @@ impl AggregateUDFImpl for Variance { fn default_value(&self, _data_type: &DataType) -> Result { Ok(ScalarValue::Float64(None)) } + + fn groups_accumulator_supported(&self, _args: AccumulatorArgs) -> bool { + true + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(VarianceGroupsAccumulator::new( + self.stats_type, + self.null_on_divide_by_zero, + ))) + } } /// An accumulator to compute variance @@ -175,15 +184,10 @@ impl Accumulator for VarianceAccumulator { let arr = downcast_value!(&values[0], Float64Array).iter().flatten(); for value in arr { - let new_count = self.count + 1.0; - let delta1 = value - self.mean; - let new_mean = delta1 / new_count + self.mean; - let delta2 = value - new_mean; - let new_m2 = self.m2 + delta1 * delta2; - - self.count += 1.0; - self.mean = new_mean; - self.m2 = new_m2; + let (c, m, m2) = super::welford::variance_update(self.count, self.mean, self.m2, value); + self.count = c; + self.mean = m; + self.m2 = m2; } Ok(()) @@ -193,15 +197,11 @@ impl Accumulator for VarianceAccumulator { let arr = downcast_value!(&values[0], Float64Array).iter().flatten(); for value in arr { - let new_count = self.count - 1.0; - let delta1 = self.mean - value; - let new_mean = delta1 / new_count + self.mean; - let delta2 = new_mean - value; - let new_m2 = self.m2 - delta1 * delta2; - - self.count -= 1.0; - self.mean = new_mean; - self.m2 = new_m2; + let (c, m, m2) = + super::welford::variance_retract(self.count, self.mean, self.m2, value); + self.count = c; + self.mean = m; + self.m2 = m2; } Ok(()) @@ -217,11 +217,14 @@ impl Accumulator for VarianceAccumulator { if c == 0_f64 { continue; } - let new_count = self.count + c; - let new_mean = self.mean * self.count / new_count + means.value(i) * c / new_count; - let delta = self.mean - means.value(i); - let new_m2 = self.m2 + m2s.value(i) + delta * delta * self.count * c / new_count; - + let (new_count, new_mean, new_m2) = super::welford::variance_merge( + self.count, + self.mean, + self.m2, + c, + means.value(i), + m2s.value(i), + ); self.count = new_count; self.mean = new_mean; self.m2 = new_m2; @@ -258,3 +261,261 @@ impl Accumulator for VarianceAccumulator { std::mem::size_of_val(self) } } + +/// Vectorized grouped variance accumulator. Mirrors the per-row +/// `VarianceAccumulator` but runs the Welford recurrence per group_index. +#[derive(Debug)] +pub(crate) struct VarianceGroupsAccumulator { + pub(super) counts: Vec, + pub(super) means: Vec, + pub(super) m2s: Vec, + stats_type: StatsType, + null_on_divide_by_zero: bool, +} + +impl VarianceGroupsAccumulator { + pub(crate) fn new(stats_type: StatsType, null_on_divide_by_zero: bool) -> Self { + Self { + counts: Vec::new(), + means: Vec::new(), + m2s: Vec::new(), + stats_type, + null_on_divide_by_zero, + } + } + + fn resize(&mut self, total_num_groups: usize) { + self.counts.resize(total_num_groups, 0.0); + self.means.resize(total_num_groups, 0.0); + self.m2s.resize(total_num_groups, 0.0); + } + + fn finalize(&mut self, emit_to: EmitTo) -> (Vec, NullBuffer) { + let counts = emit_to.take_needed(&mut self.counts); + let _ = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); + super::welford::finalize_moments(counts, m2s, self.stats_type, self.null_on_divide_by_zero) + } +} + +impl GroupsAccumulator for VarianceGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = values[0].as_primitive::(); + + self.resize(total_num_groups); + + for (idx, (&group_index, &value)) in + group_indices.iter().zip(values.values().iter()).enumerate() + { + if let Some(f) = opt_filter { + if !f.is_valid(idx) || !f.value(idx) { + continue; + } + } + if values.is_null(idx) { + continue; + } + let (c, m, m2) = super::welford::variance_update( + self.counts[group_index], + self.means[group_index], + self.m2s[group_index], + value, + ); + self.counts[group_index] = c; + self.means[group_index] = m; + self.m2s[group_index] = m2; + } + + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + _opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 3, "three arguments to merge_batch"); + let partial_counts = downcast_value!(values[0], Float64Array); + let partial_means = downcast_value!(values[1], Float64Array); + let partial_m2s = downcast_value!(values[2], Float64Array); + + self.resize(total_num_groups); + + for (i, &group_index) in group_indices.iter().enumerate() { + let partial_count = partial_counts.value(i); + if partial_count == 0.0 { + continue; + } + let (new_count, new_mean, new_m2) = super::welford::variance_merge( + self.counts[group_index], + self.means[group_index], + self.m2s[group_index], + partial_count, + partial_means.value(i), + partial_m2s.value(i), + ); + self.counts[group_index] = new_count; + self.means[group_index] = new_mean; + self.m2s[group_index] = new_m2; + } + + Ok(()) + } + + fn evaluate(&mut self, emit_to: EmitTo) -> Result { + let (values, nulls) = self.finalize(emit_to); + Ok(Arc::new(Float64Array::new(values.into(), Some(nulls)))) + } + + fn state(&mut self, emit_to: EmitTo) -> Result> { + let counts = emit_to.take_needed(&mut self.counts); + let means = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); + Ok(vec![ + Arc::new(Float64Array::new(counts.into(), None)), + Arc::new(Float64Array::new(means.into(), None)), + Arc::new(Float64Array::new(m2s.into(), None)), + ]) + } + + fn size(&self) -> usize { + self.counts.capacity() * size_of::() + + self.means.capacity() * size_of::() + + self.m2s.capacity() * size_of::() + } +} + +#[cfg(test)] +mod groups_tests { + use super::*; + use arrow::array::{AsArray, Float64Array}; + + fn pop_acc() -> VarianceGroupsAccumulator { + VarianceGroupsAccumulator::new(StatsType::Population, false) + } + fn sample_acc(null_on_divide_by_zero: bool) -> VarianceGroupsAccumulator { + VarianceGroupsAccumulator::new(StatsType::Sample, null_on_divide_by_zero) + } + + fn evaluate(acc: &mut VarianceGroupsAccumulator) -> Vec> { + acc.evaluate(EmitTo::All) + .unwrap() + .as_primitive::() + .iter() + .collect() + } + + #[test] + fn pop_variance_single_group() { + let mut acc = pop_acc(); + let values: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0])); + acc.update_batch(&[values], &[0, 0, 0, 0, 0], None, 1) + .unwrap(); + // population variance of [1..5] = 2.0 + assert_eq!(evaluate(&mut acc), vec![Some(2.0)]); + } + + #[test] + fn pop_variance_multi_group() { + let mut acc = pop_acc(); + let values: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 10.0, 20.0, 3.0])); + acc.update_batch(&[values], &[0, 0, 1, 1, 0], None, 2) + .unwrap(); + let result = evaluate(&mut acc); + // group 0: pop var of [1,2,3] = 2/3; group 1: pop var of [10,20] = 25 + assert!((result[0].unwrap() - 2.0_f64 / 3.0).abs() < 1e-12); + assert!((result[1].unwrap() - 25.0).abs() < 1e-12); + } + + #[test] + fn null_values_are_ignored() { + let mut acc = pop_acc(); + let values: ArrayRef = Arc::new(Float64Array::from(vec![ + Some(1.0), + None, + Some(2.0), + Some(3.0), + None, + ])); + acc.update_batch(&[values], &[0, 0, 0, 0, 0], None, 1) + .unwrap(); + // pop var of [1,2,3] = 2/3 + assert!((evaluate(&mut acc)[0].unwrap() - 2.0_f64 / 3.0).abs() < 1e-12); + } + + #[test] + fn opt_filter_applied() { + let mut acc = pop_acc(); + let values: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])); + let filter = BooleanArray::from(vec![true, false, true, false]); + acc.update_batch(&[values], &[0, 0, 0, 0], Some(&filter), 1) + .unwrap(); + // pop var of [1,3] = 1.0 + assert!((evaluate(&mut acc)[0].unwrap() - 1.0).abs() < 1e-12); + } + + #[test] + fn empty_group_yields_null() { + let mut acc = pop_acc(); + let values: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0])); + acc.update_batch(&[values], &[0, 0], None, 2).unwrap(); + let result = evaluate(&mut acc); + assert_eq!(result[1], None); + } + + #[test] + fn sample_single_row_nan_legacy() { + let mut acc = sample_acc(false); + let values: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); + acc.update_batch(&[values], &[0], None, 1).unwrap(); + let result = evaluate(&mut acc); + assert!(result[0].unwrap().is_nan()); + } + + #[test] + fn sample_single_row_null_when_flag_set() { + let mut acc = sample_acc(true); + let values: ArrayRef = Arc::new(Float64Array::from(vec![42.0])); + acc.update_batch(&[values], &[0], None, 1).unwrap(); + assert_eq!(evaluate(&mut acc), vec![None]); + } + + #[test] + fn merge_matches_singleshot() { + let values_full: ArrayRef = + Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0, 5.0, 6.0])); + let groups_full: Vec = vec![0; 6]; + + let mut single = pop_acc(); + single + .update_batch(std::slice::from_ref(&values_full), &groups_full, None, 1) + .unwrap(); + let single_result = evaluate(&mut single)[0].unwrap(); + + let mut left = pop_acc(); + let lvals: ArrayRef = Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0])); + left.update_batch(&[lvals], &[0, 0, 0], None, 1).unwrap(); + let left_state = left.state(EmitTo::All).unwrap(); + + let mut right = pop_acc(); + let rvals: ArrayRef = Arc::new(Float64Array::from(vec![4.0, 5.0, 6.0])); + right.update_batch(&[rvals], &[0, 0, 0], None, 1).unwrap(); + let right_state = right.state(EmitTo::All).unwrap(); + + let mut merged = pop_acc(); + merged.merge_batch(&left_state, &[0], None, 1).unwrap(); + merged.merge_batch(&right_state, &[0], None, 1).unwrap(); + let merged_result = evaluate(&mut merged)[0].unwrap(); + + assert!((single_result - merged_result).abs() < 1e-12); + } +} diff --git a/native/spark-expr/src/agg_funcs/welford.rs b/native/spark-expr/src/agg_funcs/welford.rs new file mode 100644 index 0000000000..bcc44b2988 --- /dev/null +++ b/native/spark-expr/src/agg_funcs/welford.rs @@ -0,0 +1,158 @@ +// 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. + +//! Welford-style online update/merge math shared by the per-row +//! `Accumulator` and the vectorized `GroupsAccumulator` implementations of +//! variance/stddev/covariance/correlation. Counts are `f64` to match +//! Spark's wire-format state. + +use arrow::buffer::NullBuffer; +use datafusion::physical_expr::expressions::StatsType; + +#[inline] +pub(crate) fn variance_update(count: f64, mean: f64, m2: f64, value: f64) -> (f64, f64, f64) { + let new_count = count + 1.0; + let delta1 = value - mean; + let new_mean = delta1 / new_count + mean; + let delta2 = value - new_mean; + let new_m2 = m2 + delta1 * delta2; + (new_count, new_mean, new_m2) +} + +#[inline] +pub(crate) fn variance_retract(count: f64, mean: f64, m2: f64, value: f64) -> (f64, f64, f64) { + let new_count = count - 1.0; + let delta1 = mean - value; + let new_mean = delta1 / new_count + mean; + let delta2 = new_mean - value; + let new_m2 = m2 - delta1 * delta2; + (new_count, new_mean, new_m2) +} + +#[inline] +pub(crate) fn variance_merge( + count_a: f64, + mean_a: f64, + m2_a: f64, + count_b: f64, + mean_b: f64, + m2_b: f64, +) -> (f64, f64, f64) { + let new_count = count_a + count_b; + let new_mean = mean_a * count_a / new_count + mean_b * count_b / new_count; + let delta = mean_a - mean_b; + let new_m2 = m2_a + m2_b + delta * delta * count_a * count_b / new_count; + (new_count, new_mean, new_m2) +} + +#[inline] +pub(crate) fn covariance_update( + count: f64, + mean1: f64, + mean2: f64, + c: f64, + v1: f64, + v2: f64, +) -> (f64, f64, f64, f64) { + let new_count = count + 1.0; + let delta1 = v1 - mean1; + let new_mean1 = delta1 / new_count + mean1; + let delta2 = v2 - mean2; + let new_mean2 = delta2 / new_count + mean2; + let new_c = delta1 * (v2 - new_mean2) + c; + (new_count, new_mean1, new_mean2, new_c) +} + +#[inline] +pub(crate) fn covariance_retract( + count: f64, + mean1: f64, + mean2: f64, + c: f64, + v1: f64, + v2: f64, +) -> (f64, f64, f64, f64) { + let new_count = count - 1.0; + let delta1 = mean1 - v1; + let new_mean1 = delta1 / new_count + mean1; + let delta2 = mean2 - v2; + let new_mean2 = delta2 / new_count + mean2; + let new_c = c - delta1 * (new_mean2 - v2); + (new_count, new_mean1, new_mean2, new_c) +} + +/// Compute the per-group result and a null buffer for a Welford-style +/// moment (variance `m2` or covariance `algo_const`) following Spark +/// semantics shared by variance/covariance grouped accumulators: +/// count == 0 => null, count == 1 && Sample => NaN or null depending on +/// `null_on_divide_by_zero`, otherwise `numerator / divisor`. +pub(crate) fn finalize_moments( + counts: Vec, + numerators: Vec, + stats_type: StatsType, + null_on_divide_by_zero: bool, +) -> (Vec, NullBuffer) { + let mut values = Vec::with_capacity(counts.len()); + let mut validity = Vec::with_capacity(counts.len()); + + for (count, numerator) in counts.into_iter().zip(numerators) { + if count == 0.0 { + values.push(0.0); + validity.push(false); + continue; + } + let divisor = match stats_type { + StatsType::Population => count, + StatsType::Sample if count > 1.0 => count - 1.0, + StatsType::Sample => { + if null_on_divide_by_zero { + values.push(0.0); + validity.push(false); + } else { + values.push(f64::NAN); + validity.push(true); + } + continue; + } + }; + values.push(numerator / divisor); + validity.push(true); + } + + (values, NullBuffer::from(validity)) +} + +#[inline] +#[allow(clippy::too_many_arguments)] +pub(crate) fn covariance_merge( + count_a: f64, + mean1_a: f64, + mean2_a: f64, + c_a: f64, + count_b: f64, + mean1_b: f64, + mean2_b: f64, + c_b: f64, +) -> (f64, f64, f64, f64) { + let new_count = count_a + count_b; + let new_mean1 = mean1_a * count_a / new_count + mean1_b * count_b / new_count; + let new_mean2 = mean2_a * count_a / new_count + mean2_b * count_b / new_count; + let delta1 = mean1_a - mean1_b; + let delta2 = mean2_a - mean2_b; + let new_c = c_a + c_b + delta1 * delta2 * count_a * count_b / new_count; + (new_count, new_mean1, new_mean2, new_c) +} diff --git a/native/spark-expr/src/array_funcs/array_compact.rs b/native/spark-expr/src/array_funcs/array_compact.rs index 5795ec64f8..18e28b9afa 100644 --- a/native/spark-expr/src/array_funcs/array_compact.rs +++ b/native/spark-expr/src/array_funcs/array_compact.rs @@ -33,7 +33,6 @@ use datafusion::common::{exec_err, utils::take_function_args, Result}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, TypeSignature, Volatility, }; -use std::any::Any; use std::sync::Arc; #[derive(Debug, PartialEq, Eq, Hash)] @@ -56,10 +55,6 @@ impl SparkArrayCompact { } impl ScalarUDFImpl for SparkArrayCompact { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "spark_array_compact" } diff --git a/native/spark-expr/src/array_funcs/array_insert.rs b/native/spark-expr/src/array_funcs/array_insert.rs index bce00483bc..e638c440fd 100644 --- a/native/spark-expr/src/array_funcs/array_insert.rs +++ b/native/spark-expr/src/array_funcs/array_insert.rs @@ -30,7 +30,6 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::hash::Hash; use std::{ - any::Any, fmt::{Debug, Display, Formatter}, sync::Arc, }; @@ -92,10 +91,6 @@ impl ArrayInsert { } impl PhysicalExpr for ArrayInsert { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/array_funcs/array_position.rs b/native/spark-expr/src/array_funcs/array_position.rs index dbcd8615df..a5841b27d3 100644 --- a/native/spark-expr/src/array_funcs/array_position.rs +++ b/native/spark-expr/src/array_funcs/array_position.rs @@ -28,7 +28,6 @@ use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, TypeSignature, Volatility, }; use num::Float; -use std::any::Any; use std::sync::Arc; /// Spark array_position() function that returns the 1-based position of an element in an array. @@ -313,10 +312,6 @@ impl SparkArrayPositionFunc { } impl ScalarUDFImpl for SparkArrayPositionFunc { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "spark_array_position" } diff --git a/native/spark-expr/src/array_funcs/arrays_overlap.rs b/native/spark-expr/src/array_funcs/arrays_overlap.rs index 662186e614..889a0314df 100644 --- a/native/spark-expr/src/array_funcs/arrays_overlap.rs +++ b/native/spark-expr/src/array_funcs/arrays_overlap.rs @@ -39,7 +39,6 @@ use datafusion::common::{exec_err, utils::take_function_args, Result, ScalarValu use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, TypeSignature, Volatility, }; -use std::any::Any; use std::sync::Arc; #[derive(Debug, PartialEq, Eq, Hash)] @@ -62,10 +61,6 @@ impl SparkArraysOverlap { } impl ScalarUDFImpl for SparkArraysOverlap { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "spark_arrays_overlap" } diff --git a/native/spark-expr/src/array_funcs/arrays_zip.rs b/native/spark-expr/src/array_funcs/arrays_zip.rs index 2126eb732c..6f0332d3d8 100644 --- a/native/spark-expr/src/array_funcs/arrays_zip.rs +++ b/native/spark-expr/src/array_funcs/arrays_zip.rs @@ -27,7 +27,6 @@ use datafusion::common::cast::{as_fixed_size_list_array, as_large_list_array, as use datafusion::common::{exec_err, Result, ScalarValue}; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; -use std::any::Any; use std::fmt::{Display, Formatter}; use std::sync::Arc; // TODO: Reuse functions from DF @@ -74,10 +73,6 @@ impl Display for SparkArraysZipFunc { } impl PhysicalExpr for SparkArraysZipFunc { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, input_schema: &Schema) -> Result { let fields = self.fields(input_schema)?; Ok(List(Arc::new(Field::new_list_field( diff --git a/native/spark-expr/src/array_funcs/get_array_struct_fields.rs b/native/spark-expr/src/array_funcs/get_array_struct_fields.rs index dc05a3b7f0..a873392499 100644 --- a/native/spark-expr/src/array_funcs/get_array_struct_fields.rs +++ b/native/spark-expr/src/array_funcs/get_array_struct_fields.rs @@ -27,7 +27,6 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::hash::Hash; use std::{ - any::Any, fmt::{Debug, Display, Formatter}, sync::Arc, }; @@ -75,10 +74,6 @@ impl GetArrayStructFields { } impl PhysicalExpr for GetArrayStructFields { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, input_schema: &Schema) -> DataFusionResult { let struct_field = self.child_field(input_schema)?; match self.child.data_type(input_schema)? { diff --git a/native/spark-expr/src/array_funcs/list_extract.rs b/native/spark-expr/src/array_funcs/list_extract.rs index d3661f496a..83fa23f757 100644 --- a/native/spark-expr/src/array_funcs/list_extract.rs +++ b/native/spark-expr/src/array_funcs/list_extract.rs @@ -26,7 +26,6 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::hash::Hash; use std::{ - any::Any, fmt::{Debug, Display, Formatter}, sync::Arc, }; @@ -113,10 +112,6 @@ impl ListExtract { } impl PhysicalExpr for ListExtract { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/array_funcs/size.rs b/native/spark-expr/src/array_funcs/size.rs index 9777553341..f206b299d6 100644 --- a/native/spark-expr/src/array_funcs/size.rs +++ b/native/spark-expr/src/array_funcs/size.rs @@ -21,7 +21,6 @@ use datafusion::common::{exec_err, DataFusionError, Result as DataFusionResult, use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::any::Any; use std::sync::Arc; /// Spark size() function that returns the size of arrays or maps. @@ -73,10 +72,6 @@ impl SparkSizeFunc { } impl ScalarUDFImpl for SparkSizeFunc { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "size" } diff --git a/native/spark-expr/src/bloom_filter/bloom_filter_agg.rs b/native/spark-expr/src/bloom_filter/bloom_filter_agg.rs index 4ac236e6fc..03f7223d4c 100644 --- a/native/spark-expr/src/bloom_filter/bloom_filter_agg.rs +++ b/native/spark-expr/src/bloom_filter/bloom_filter_agg.rs @@ -17,7 +17,7 @@ use arrow::datatypes::{Field, FieldRef}; use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; use crate::bloom_filter::spark_bloom_filter; use crate::bloom_filter::spark_bloom_filter::{SparkBloomFilter, SparkBloomFilterVersion}; @@ -45,7 +45,7 @@ pub struct BloomFilterAgg { #[inline] fn extract_i32_from_literal(expr: Arc) -> i32 { - match expr.as_any().downcast_ref::().unwrap().value() { + match expr.downcast_ref::().unwrap().value() { ScalarValue::Int64(scalar_value) => scalar_value.unwrap() as i32, _ => { unreachable!() @@ -81,10 +81,6 @@ impl BloomFilterAgg { } impl AggregateUDFImpl for BloomFilterAgg { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "bloom_filter_agg" } diff --git a/native/spark-expr/src/bloom_filter/bloom_filter_might_contain.rs b/native/spark-expr/src/bloom_filter/bloom_filter_might_contain.rs index ea246dfb25..66168444d9 100644 --- a/native/spark-expr/src/bloom_filter/bloom_filter_might_contain.rs +++ b/native/spark-expr/src/bloom_filter/bloom_filter_might_contain.rs @@ -22,7 +22,6 @@ use datafusion::error::DataFusionError; use datafusion::logical_expr::{ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::ColumnarValue; -use std::any::Any; use std::sync::Arc; use crate::bloom_filter::spark_bloom_filter::SparkBloomFilter; @@ -63,10 +62,6 @@ fn evaluate_bloom_filter( } impl ScalarUDFImpl for BloomFilterMightContain { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "might_contain" } diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index 9ecb11dc52..8916cc92d6 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -23,10 +23,11 @@ use crate::math_funcs::log::spark_log; use crate::math_funcs::modulo_expr::spark_modulo; use crate::{ spark_ceil, spark_decimal_div, spark_decimal_integral_div, spark_floor, spark_isnan, - spark_lpad, spark_make_decimal, spark_read_side_padding, spark_round, spark_rpad, spark_unhex, - spark_unscaled_value, EvalMode, SparkArrayCompact, SparkArrayPositionFunc, SparkArraysOverlap, - SparkContains, SparkDateDiff, SparkDateFromUnixDate, SparkDateTrunc, SparkMakeDate, - SparkSecondsToTimestamp, SparkSizeFunc, + spark_lpad, spark_make_decimal, spark_read_side_padding, spark_round, spark_rpad, + spark_to_time, spark_unhex, spark_unscaled_value, EvalMode, SparkArrayCompact, + SparkArrayPositionFunc, SparkArraysOverlap, SparkContains, SparkDateDiff, + SparkDateFromUnixDate, SparkDateTrunc, SparkMakeDate, SparkMakeTime, SparkSecondsToTimestamp, + SparkSizeFunc, }; use arrow::datatypes::DataType; use datafusion::common::{DataFusionError, Result as DataFusionResult}; @@ -36,7 +37,6 @@ use datafusion::logical_expr::{ Volatility, }; use datafusion::physical_plan::ColumnarValue; -use std::any::Any; use std::fmt::Debug; use std::sync::Arc; @@ -196,6 +196,9 @@ pub fn create_comet_physical_fun_with_eval_mode( let func = Arc::new(spark_map_sort); make_comet_scalar_udf!("spark_map_sort", func, without data_type) } + "to_time" => { + make_comet_scalar_udf!("to_time", spark_to_time, without data_type, fail_on_error) + } _ => registry.udf(fun_name).map_err(|e| { DataFusionError::Execution(format!( "Function {fun_name} not found in the registry: {e}", @@ -214,6 +217,7 @@ fn all_scalar_functions() -> Vec> { Arc::new(ScalarUDF::new_from_impl(SparkDateFromUnixDate::default())), Arc::new(ScalarUDF::new_from_impl(SparkDateTrunc::default())), Arc::new(ScalarUDF::new_from_impl(SparkMakeDate::default())), + Arc::new(ScalarUDF::new_from_impl(SparkMakeTime::default())), Arc::new(ScalarUDF::new_from_impl(SparkSecondsToTimestamp::default())), Arc::new(ScalarUDF::new_from_impl(SparkSizeFunc::default())), ] @@ -283,10 +287,6 @@ impl CometScalarFunction { } impl ScalarUDFImpl for CometScalarFunction { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { self.name.as_str() } diff --git a/native/spark-expr/src/conditional_funcs/if_expr.rs b/native/spark-expr/src/conditional_funcs/if_expr.rs index 6b1291fbbe..5e57c54a22 100644 --- a/native/spark-expr/src/conditional_funcs/if_expr.rs +++ b/native/spark-expr/src/conditional_funcs/if_expr.rs @@ -24,7 +24,7 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::{expressions::CaseExpr, PhysicalExpr}; use std::fmt::{Display, Formatter}; use std::hash::Hash; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; /// IfExpr is a wrapper around CaseExpr, because `IF(a, b, c)` is semantically equivalent to /// `CASE WHEN a THEN b ELSE c END`. @@ -83,11 +83,6 @@ impl IfExpr { } impl PhysicalExpr for IfExpr { - /// Return a reference to Any that can be used for down-casting - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/conversion_funcs/cast.rs b/native/spark-expr/src/conversion_funcs/cast.rs index 1f574f1231..dadcd1048e 100644 --- a/native/spark-expr/src/conversion_funcs/cast.rs +++ b/native/spark-expr/src/conversion_funcs/cast.rs @@ -63,7 +63,6 @@ use datafusion::common::{internal_err, DataFusionError, Result as DataFusionResu use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::ColumnarValue; use std::{ - any::Any, fmt::{Debug, Display, Formatter}, hash::Hash, sync::Arc, @@ -740,10 +739,6 @@ impl Display for Cast { } impl PhysicalExpr for Cast { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/csv_funcs/to_csv.rs b/native/spark-expr/src/csv_funcs/to_csv.rs index f41cb7f918..01fdc901cb 100644 --- a/native/spark-expr/src/csv_funcs/to_csv.rs +++ b/native/spark-expr/src/csv_funcs/to_csv.rs @@ -23,7 +23,6 @@ use arrow::datatypes::{DataType, Schema}; use datafusion::common::Result; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; -use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::Hash; use std::sync::Arc; @@ -77,10 +76,6 @@ impl Display for ToCsv { } impl PhysicalExpr for ToCsv { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, _: &Schema) -> Result { Ok(DataType::Utf8) } diff --git a/native/spark-expr/src/datetime_funcs/date_diff.rs b/native/spark-expr/src/datetime_funcs/date_diff.rs index ca148c103a..be3c1d3552 100644 --- a/native/spark-expr/src/datetime_funcs/date_diff.rs +++ b/native/spark-expr/src/datetime_funcs/date_diff.rs @@ -22,7 +22,6 @@ use datafusion::common::{utils::take_function_args, DataFusionError, Result}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::any::Any; use std::sync::Arc; /// Spark-compatible date_diff function. @@ -52,10 +51,6 @@ impl Default for SparkDateDiff { } impl ScalarUDFImpl for SparkDateDiff { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "date_diff" } diff --git a/native/spark-expr/src/datetime_funcs/date_from_unix_date.rs b/native/spark-expr/src/datetime_funcs/date_from_unix_date.rs index 1c88fc47ab..0e624e6472 100644 --- a/native/spark-expr/src/datetime_funcs/date_from_unix_date.rs +++ b/native/spark-expr/src/datetime_funcs/date_from_unix_date.rs @@ -21,7 +21,6 @@ use datafusion::common::{utils::take_function_args, DataFusionError, Result, Sca use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::any::Any; use std::sync::Arc; /// Spark-compatible date_from_unix_date function. @@ -48,10 +47,6 @@ impl Default for SparkDateFromUnixDate { } impl ScalarUDFImpl for SparkDateFromUnixDate { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "date_from_unix_date" } diff --git a/native/spark-expr/src/datetime_funcs/date_trunc.rs b/native/spark-expr/src/datetime_funcs/date_trunc.rs index aeae18e36f..7ceb5234e1 100644 --- a/native/spark-expr/src/datetime_funcs/date_trunc.rs +++ b/native/spark-expr/src/datetime_funcs/date_trunc.rs @@ -22,7 +22,6 @@ use datafusion::common::{ use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::any::Any; use crate::kernels::temporal::{date_trunc_array_fmt_dyn, date_trunc_dyn}; @@ -51,10 +50,6 @@ impl Default for SparkDateTrunc { } impl ScalarUDFImpl for SparkDateTrunc { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "date_trunc" } diff --git a/native/spark-expr/src/datetime_funcs/extract_date_part.rs b/native/spark-expr/src/datetime_funcs/extract_date_part.rs index acb7d2266e..7344a3953a 100644 --- a/native/spark-expr/src/datetime_funcs/extract_date_part.rs +++ b/native/spark-expr/src/datetime_funcs/extract_date_part.rs @@ -22,7 +22,7 @@ use datafusion::common::{internal_datafusion_err, DataFusionError}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::{any::Any, fmt::Debug}; +use std::fmt::Debug; macro_rules! extract_date_part { ($struct_name:ident, $fn_name:expr, $date_part_variant:ident) => { @@ -44,10 +44,6 @@ macro_rules! extract_date_part { } impl ScalarUDFImpl for $struct_name { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { $fn_name } diff --git a/native/spark-expr/src/datetime_funcs/hours.rs b/native/spark-expr/src/datetime_funcs/hours.rs index ea3ef742a4..26ec35ef25 100644 --- a/native/spark-expr/src/datetime_funcs/hours.rs +++ b/native/spark-expr/src/datetime_funcs/hours.rs @@ -32,7 +32,7 @@ use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; use num::integer::div_floor; -use std::{any::Any, fmt::Debug, sync::Arc}; +use std::{fmt::Debug, sync::Arc}; const MICROS_PER_HOUR: i64 = 3_600_000_000; @@ -56,10 +56,6 @@ impl Default for SparkHoursTransform { } impl ScalarUDFImpl for SparkHoursTransform { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "hours_transform" } diff --git a/native/spark-expr/src/datetime_funcs/make_date.rs b/native/spark-expr/src/datetime_funcs/make_date.rs index 58e4108580..ef29431703 100644 --- a/native/spark-expr/src/datetime_funcs/make_date.rs +++ b/native/spark-expr/src/datetime_funcs/make_date.rs @@ -23,7 +23,6 @@ use datafusion::common::{utils::take_function_args, DataFusionError, Result}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::any::Any; use std::sync::Arc; /// Spark-compatible make_date function. @@ -75,10 +74,6 @@ fn make_date(year: i32, month: i32, day: i32) -> Option { } impl ScalarUDFImpl for SparkMakeDate { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "make_date" } diff --git a/native/spark-expr/src/datetime_funcs/make_time.rs b/native/spark-expr/src/datetime_funcs/make_time.rs new file mode 100644 index 0000000000..ec95d19044 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/make_time.rs @@ -0,0 +1,227 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{Array, Decimal128Array, Int32Array, Time64NanosecondArray}; +use arrow::compute::cast; +use arrow::datatypes::{DataType, TimeUnit}; +use datafusion::common::{utils::take_function_args, DataFusionError, Result}; +use datafusion::logical_expr::{ + ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, +}; +use std::sync::Arc; + +const MICROS_PER_SECOND: i128 = 1_000_000; +const NANOS_PER_MICRO: i64 = 1_000; +const NANOS_PER_SECOND: i64 = 1_000_000_000; + +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct SparkMakeTime { + signature: Signature, +} + +impl SparkMakeTime { + pub fn new() -> Self { + Self { + signature: Signature::any(3, Volatility::Immutable), + } + } +} + +impl Default for SparkMakeTime { + fn default() -> Self { + Self::new() + } +} + +/// Converts hours, minutes, and fractional seconds (Decimal(16,6)) to nanoseconds from midnight. +/// Returns an error for invalid inputs (matching Spark's always-throw behavior). +fn make_time(hours: i32, minutes: i32, secs_and_micros_unscaled: i128) -> Result { + let full_secs = secs_and_micros_unscaled.div_euclid(MICROS_PER_SECOND); + let frac_micros = secs_and_micros_unscaled.rem_euclid(MICROS_PER_SECOND); + + if full_secs > i32::MAX as i128 || full_secs < 0 { + return Err(DataFusionError::Execution(format!( + "Invalid value for SecondOfMinute (valid values 0 - 59): {}", + secs_and_micros_unscaled / MICROS_PER_SECOND + ))); + } + + let secs = full_secs as i32; + let nanos = (frac_micros as i64) * NANOS_PER_MICRO; + + if !(0..=23).contains(&hours) { + return Err(DataFusionError::Execution(format!( + "Invalid value for HourOfDay (valid values 0 - 23): {hours}" + ))); + } + if !(0..=59).contains(&minutes) { + return Err(DataFusionError::Execution(format!( + "Invalid value for MinuteOfHour (valid values 0 - 59): {minutes}" + ))); + } + if !(0..=59).contains(&secs) { + return Err(DataFusionError::Execution(format!( + "Invalid value for SecondOfMinute (valid values 0 - 59): {secs}" + ))); + } + + let total_nanos = hours as i64 * 3_600 * NANOS_PER_SECOND + + minutes as i64 * 60 * NANOS_PER_SECOND + + secs as i64 * NANOS_PER_SECOND + + nanos; + + Ok(total_nanos) +} + +impl ScalarUDFImpl for SparkMakeTime { + fn name(&self) -> &str { + "make_time" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _: &[DataType]) -> Result { + Ok(DataType::Time64(TimeUnit::Nanosecond)) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + let [hours, minutes, secs_and_micros] = take_function_args(self.name(), args.args)?; + + let num_rows = [&hours, &minutes, &secs_and_micros] + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + + let hours_arr = hours.into_array(num_rows)?; + let minutes_arr = minutes.into_array(num_rows)?; + let secs_arr = secs_and_micros.into_array(num_rows)?; + + let hours_arr = cast_to_int32(&hours_arr)?; + let minutes_arr = cast_to_int32(&minutes_arr)?; + + let hours_array = hours_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("make_time: failed to cast hours to Int32".to_string()) + })?; + + let minutes_array = minutes_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("make_time: failed to cast minutes to Int32".to_string()) + })?; + + let secs_array = secs_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution( + "make_time: expected Decimal128 for seconds argument".to_string(), + ) + })?; + + let len = hours_array.len(); + let mut builder = Time64NanosecondArray::builder(len); + + for i in 0..len { + if hours_array.is_null(i) || minutes_array.is_null(i) || secs_array.is_null(i) { + builder.append_null(); + } else { + let h = hours_array.value(i); + let m = minutes_array.value(i); + let s = secs_array.value(i); + + let nanos = make_time(h, m, s)?; + builder.append_value(nanos); + } + } + + Ok(ColumnarValue::Array(Arc::new(builder.finish()))) + } +} + +fn cast_to_int32(arr: &Arc) -> Result> { + if arr.data_type() == &DataType::Int32 { + Ok(Arc::clone(arr)) + } else { + cast(arr.as_ref(), &DataType::Int32) + .map_err(|e| DataFusionError::Execution(format!("Failed to cast to Int32: {e}"))) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_make_time_valid() { + // Midnight + assert_eq!(make_time(0, 0, 0).unwrap(), 0); + // 1 hour + assert_eq!(make_time(1, 0, 0).unwrap(), 3_600_000_000_000); + // 1 minute + assert_eq!(make_time(0, 1, 0).unwrap(), 60_000_000_000); + // 1 second (unscaled: 1_000_000) + assert_eq!(make_time(0, 0, 1_000_000).unwrap(), 1_000_000_000); + // 1.5 seconds (unscaled: 1_500_000) + assert_eq!(make_time(0, 0, 1_500_000).unwrap(), 1_500_000_000); + // 23:59:59.999999 (unscaled: 59_999_999) + assert_eq!(make_time(23, 59, 59_999_999).unwrap(), 86_399_999_999_000); + // 12:30:45.123456 (unscaled: 45_123_456) + assert_eq!( + make_time(12, 30, 45_123_456).unwrap(), + 12 * 3_600_000_000_000 + 30 * 60_000_000_000 + 45_123_456_000 + ); + } + + #[test] + fn test_make_time_invalid_hours() { + assert!(make_time(24, 0, 0).is_err()); + assert!(make_time(25, 0, 0).is_err()); + assert!(make_time(-1, 0, 0).is_err()); + } + + #[test] + fn test_make_time_invalid_minutes() { + assert!(make_time(0, 60, 0).is_err()); + assert!(make_time(0, -1, 0).is_err()); + } + + #[test] + fn test_make_time_invalid_seconds() { + // 60 seconds (unscaled: 60_000_000) + assert!(make_time(0, 0, 60_000_000).is_err()); + // 100.5 seconds (unscaled: 100_500_000) + assert!(make_time(0, 0, 100_500_000).is_err()); + // negative seconds (unscaled: -1_000_000) + assert!(make_time(0, 0, -1_000_000).is_err()); + } + + #[test] + fn test_make_time_overflow_seconds() { + // Very large value that overflows i32 + let large = (i32::MAX as i128 + 1) * MICROS_PER_SECOND; + assert!(make_time(0, 0, large).is_err()); + } +} diff --git a/native/spark-expr/src/datetime_funcs/mod.rs b/native/spark-expr/src/datetime_funcs/mod.rs index a94bf16ce2..53a1f185ff 100644 --- a/native/spark-expr/src/datetime_funcs/mod.rs +++ b/native/spark-expr/src/datetime_funcs/mod.rs @@ -21,8 +21,10 @@ mod date_trunc; mod extract_date_part; mod hours; mod make_date; +mod make_time; mod seconds_to_timestamp; mod timestamp_trunc; +mod to_time; mod unix_timestamp; pub use date_diff::SparkDateDiff; @@ -33,6 +35,8 @@ pub use extract_date_part::SparkMinute; pub use extract_date_part::SparkSecond; pub use hours::SparkHoursTransform; pub use make_date::SparkMakeDate; +pub use make_time::SparkMakeTime; pub use seconds_to_timestamp::SparkSecondsToTimestamp; pub use timestamp_trunc::TimestampTruncExpr; +pub use to_time::spark_to_time; pub use unix_timestamp::SparkUnixTimestamp; diff --git a/native/spark-expr/src/datetime_funcs/seconds_to_timestamp.rs b/native/spark-expr/src/datetime_funcs/seconds_to_timestamp.rs index 2da1ac73b0..ab9388692b 100644 --- a/native/spark-expr/src/datetime_funcs/seconds_to_timestamp.rs +++ b/native/spark-expr/src/datetime_funcs/seconds_to_timestamp.rs @@ -24,7 +24,6 @@ use datafusion::common::{utils::take_function_args, DataFusionError, Result, Sca use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, TypeSignature, Volatility, }; -use std::any::Any; use std::sync::Arc; const MICROS_PER_SECOND: i64 = 1_000_000; @@ -61,10 +60,6 @@ impl Default for SparkSecondsToTimestamp { } impl ScalarUDFImpl for SparkSecondsToTimestamp { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "seconds_to_timestamp" } diff --git a/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs b/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs index 3435d3ee50..c26c5ef08e 100644 --- a/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs +++ b/native/spark-expr/src/datetime_funcs/timestamp_trunc.rs @@ -23,7 +23,6 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::hash::Hash; use std::{ - any::Any, fmt::{Debug, Display, Formatter}, sync::Arc, }; @@ -85,10 +84,6 @@ impl Display for TimestampTruncExpr { } impl PhysicalExpr for TimestampTruncExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/datetime_funcs/to_time.rs b/native/spark-expr/src/datetime_funcs/to_time.rs new file mode 100644 index 0000000000..727998fdf8 --- /dev/null +++ b/native/spark-expr/src/datetime_funcs/to_time.rs @@ -0,0 +1,482 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{Array, StringArray, Time64NanosecondArray}; +use datafusion::common::{DataFusionError, Result}; +use datafusion::physical_plan::ColumnarValue; +use std::sync::Arc; + +const NANOS_PER_MICRO: i64 = 1_000; +const NANOS_PER_SECOND: i64 = 1_000_000_000; +const NANOS_PER_MINUTE: i64 = 60 * NANOS_PER_SECOND; +const NANOS_PER_HOUR: i64 = 60 * NANOS_PER_MINUTE; + +/// Spark-compatible to_time: parse a string to time (nanoseconds from midnight). +/// When fail_on_error is true (to_time), returns an error for unparseable input. +/// When fail_on_error is false (try_to_time), returns null for unparseable input. +pub fn spark_to_time(args: &[ColumnarValue], fail_on_error: bool) -> Result { + if args.is_empty() { + return Err(DataFusionError::Execution( + "to_time requires at least 1 argument".to_string(), + )); + } + + let num_rows = args + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + + let str_arr = args[0].clone().into_array(num_rows)?; + let str_array = str_arr + .as_any() + .downcast_ref::() + .ok_or_else(|| { + DataFusionError::Execution("to_time: expected String argument".to_string()) + })?; + + let len = str_array.len(); + let mut builder = Time64NanosecondArray::builder(len); + + for i in 0..len { + if str_array.is_null(i) { + builder.append_null(); + } else { + let s = str_array.value(i); + match string_to_time(s) { + Some(nanos) => builder.append_value(nanos), + None => { + if fail_on_error { + return Err(DataFusionError::Execution(format!( + "The input string '{}' cannot be parsed to a TIME value", + s + ))); + } + builder.append_null(); + } + } + } + } + + Ok(ColumnarValue::Array(Arc::new(builder.finish()))) +} + +/// Parse a time string to nanoseconds from midnight, matching Spark's stringToTime behavior. +/// Returns None for invalid input. +fn string_to_time(s: &str) -> Option { + let trimmed = s.trim(); + if trimmed.is_empty() { + return None; + } + + // Spark's parseTimestampString gates the T-prefix branch on j == 0 (start of + // the trimmed string), so " T12:30" is rejected even though leading whitespace + // is trimmed: the original segment start differs from the trimmed position. + if trimmed.as_bytes()[0] == b'T' && s.as_bytes()[0].is_ascii_whitespace() { + return None; + } + + let bytes = trimmed.as_bytes(); + let num_chars = bytes.len(); + + // Detect AM/PM suffix + let (is_am, is_pm, has_suffix) = if num_chars > 2 { + let last = bytes[num_chars - 1]; + if last == b'M' || last == b'm' { + let second_last = bytes[num_chars - 2]; + let am = second_last == b'A' || second_last == b'a'; + let pm = second_last == b'P' || second_last == b'p'; + (am, pm, am || pm) + } else { + (false, false, false) + } + } else { + (false, false, false) + }; + + // Strip AM/PM suffix (and optional space before it) + let time_str = if has_suffix { + let end = num_chars - 2; + let s = &trimmed[..end]; + s.trim_end() + } else { + trimmed + }; + + // Parse the time components + let (hour, minute, second, micros) = parse_time_components(time_str)?; + + // Validate and convert hours + let hr = if !has_suffix { + if hour > 23 { + return None; + } + hour + } else { + if !(1..=12).contains(&hour) { + return None; + } + if is_am { + if hour == 12 { + 0 + } else { + hour + } + } else if is_pm { + if hour == 12 { + 12 + } else { + hour + 12 + } + } else { + return None; + } + }; + + // Validate minutes and seconds + if minute > 59 || second > 59 { + return None; + } + + let nanos = hr as i64 * NANOS_PER_HOUR + + minute as i64 * NANOS_PER_MINUTE + + second as i64 * NANOS_PER_SECOND + + micros as i64 * NANOS_PER_MICRO; + + Some(nanos) +} + +/// Parse time components from a string like "HH:mm:ss.ffffff" or "T HH:mm:ss". +/// Returns (hour, minute, second, microseconds) or None if invalid. +fn parse_time_components(s: &str) -> Option<(i32, i32, i32, i32)> { + let bytes = s.as_bytes(); + if bytes.is_empty() { + return None; + } + + let mut pos = 0; + + // Skip optional 'T' prefix + if bytes[pos] == b'T' { + pos += 1; + } + + // Parse hour + let (hour, new_pos) = parse_digits(bytes, pos)?; + pos = new_pos; + if hour < 0 { + return None; + } + + // Expect ':' + if pos >= bytes.len() || bytes[pos] != b':' { + return None; + } + pos += 1; + + // Parse minute + let (minute, new_pos) = parse_digits(bytes, pos)?; + pos = new_pos; + + // Optional seconds + if pos >= bytes.len() { + return Some((hour, minute, 0, 0)); + } + + if bytes[pos] != b':' { + return None; + } + pos += 1; + + // Parse seconds + let (second, new_pos) = parse_digits(bytes, pos)?; + pos = new_pos; + + // Optional fractional seconds + if pos >= bytes.len() { + return Some((hour, minute, second, 0)); + } + + if bytes[pos] != b'.' { + // No more content allowed (timezone would invalidate) + return None; + } + pos += 1; + + // Parse fractional seconds (up to 6 digits, pad with zeros) + let (micros, new_pos) = parse_fractional(bytes, pos)?; + pos = new_pos; + + // Nothing should follow the fractional seconds (timezone not allowed for time) + if pos < bytes.len() { + return None; + } + + Some((hour, minute, second, micros)) +} + +/// Parse consecutive digits starting at pos. Returns (value, new_pos). +/// At least 1 digit is required. +fn parse_digits(bytes: &[u8], start: usize) -> Option<(i32, usize)> { + let mut pos = start; + let mut value: i32 = 0; + let mut count = 0; + + while pos < bytes.len() { + let b = bytes[pos]; + if b.is_ascii_digit() { + value = value * 10 + (b - b'0') as i32; + count += 1; + pos += 1; + } else { + break; + } + } + + if count == 0 || count > 2 { + return None; + } + + Some((value, pos)) +} + +/// Parse fractional seconds (microseconds). Up to 6 digits, padded with zeros. +/// Returns (microseconds, new_pos). +fn parse_fractional(bytes: &[u8], start: usize) -> Option<(i32, usize)> { + let mut pos = start; + let mut value: i32 = 0; + let mut count = 0; + + while pos < bytes.len() && count < 6 { + let b = bytes[pos]; + if b.is_ascii_digit() { + value = value * 10 + (b - b'0') as i32; + count += 1; + pos += 1; + } else { + break; + } + } + + if count == 0 { + return None; + } + + // Skip any remaining digits beyond 6 (truncation) + while pos < bytes.len() && bytes[pos].is_ascii_digit() { + pos += 1; + } + + // Pad with zeros to 6 digits + while count < 6 { + value *= 10; + count += 1; + } + + Some((value, pos)) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_basic_time_parsing() { + // HH:mm + assert_eq!(string_to_time("00:00"), Some(0)); + assert_eq!( + string_to_time("12:30"), + Some(12 * NANOS_PER_HOUR + 30 * NANOS_PER_MINUTE) + ); + assert_eq!( + string_to_time("23:59"), + Some(23 * NANOS_PER_HOUR + 59 * NANOS_PER_MINUTE) + ); + + // HH:mm:ss + assert_eq!( + string_to_time("12:30:45"), + Some(12 * NANOS_PER_HOUR + 30 * NANOS_PER_MINUTE + 45 * NANOS_PER_SECOND) + ); + assert_eq!(string_to_time("00:00:00"), Some(0)); + assert_eq!( + string_to_time("23:59:59"), + Some(23 * NANOS_PER_HOUR + 59 * NANOS_PER_MINUTE + 59 * NANOS_PER_SECOND) + ); + } + + #[test] + fn test_fractional_seconds() { + // 1 digit + assert_eq!( + string_to_time("00:00:00.1"), + Some(100_000 * NANOS_PER_MICRO) + ); + // 3 digits + assert_eq!( + string_to_time("00:00:00.001"), + Some(1_000 * NANOS_PER_MICRO) + ); + // 6 digits + assert_eq!(string_to_time("00:00:00.000001"), Some(NANOS_PER_MICRO)); + // >6 digits truncated to microseconds + assert_eq!( + string_to_time("00:00:00.1234567"), + Some(123_456 * NANOS_PER_MICRO) + ); + // Full precision + assert_eq!( + string_to_time("23:59:59.999999"), + Some( + 23 * NANOS_PER_HOUR + + 59 * NANOS_PER_MINUTE + + 59 * NANOS_PER_SECOND + + 999_999 * NANOS_PER_MICRO + ) + ); + } + + #[test] + fn test_single_digit_components() { + // Single digit hour, minute, second + assert_eq!( + string_to_time("1:2:3"), + Some(NANOS_PER_HOUR + 2 * NANOS_PER_MINUTE + 3 * NANOS_PER_SECOND) + ); + assert_eq!( + string_to_time("1:2:3.04"), + Some( + NANOS_PER_HOUR + + 2 * NANOS_PER_MINUTE + + 3 * NANOS_PER_SECOND + + 40_000 * NANOS_PER_MICRO + ) + ); + } + + #[test] + fn test_t_prefix() { + assert_eq!( + string_to_time("T1:02:3.04"), + Some( + NANOS_PER_HOUR + + 2 * NANOS_PER_MINUTE + + 3 * NANOS_PER_SECOND + + 40_000 * NANOS_PER_MICRO + ) + ); + assert_eq!( + string_to_time("T12:30:45"), + Some(12 * NANOS_PER_HOUR + 30 * NANOS_PER_MINUTE + 45 * NANOS_PER_SECOND) + ); + } + + #[test] + fn test_am_pm() { + // 12:00:00 AM = midnight + assert_eq!(string_to_time("12:00:00 AM"), Some(0)); + // 1:00:00 AM + assert_eq!(string_to_time("1:00:00 AM"), Some(NANOS_PER_HOUR)); + // 11:59:59 AM + assert_eq!( + string_to_time("11:59:59 AM"), + Some(11 * NANOS_PER_HOUR + 59 * NANOS_PER_MINUTE + 59 * NANOS_PER_SECOND) + ); + // 12:00:00 PM = noon + assert_eq!(string_to_time("12:00:00 PM"), Some(12 * NANOS_PER_HOUR)); + // 1:00:00 PM = 13:00 + assert_eq!(string_to_time("1:00:00 PM"), Some(13 * NANOS_PER_HOUR)); + // 11:59:59 PM = 23:59:59 + assert_eq!( + string_to_time("11:59:59 PM"), + Some(23 * NANOS_PER_HOUR + 59 * NANOS_PER_MINUTE + 59 * NANOS_PER_SECOND) + ); + // Case insensitive + assert_eq!(string_to_time("12:00:00 am"), Some(0)); + assert_eq!(string_to_time("12:00:00 pm"), Some(12 * NANOS_PER_HOUR)); + // No space before AM/PM + assert_eq!(string_to_time("12:00:00AM"), Some(0)); + assert_eq!(string_to_time("1:00:00PM"), Some(13 * NANOS_PER_HOUR)); + // With fractional seconds + assert_eq!( + string_to_time("12:59:59.999999 PM"), + Some( + 12 * NANOS_PER_HOUR + + 59 * NANOS_PER_MINUTE + + 59 * NANOS_PER_SECOND + + 999_999 * NANOS_PER_MICRO + ) + ); + } + + #[test] + fn test_invalid_am_pm() { + // Hour 0 invalid in 12-hour format + assert_eq!(string_to_time("0:00:00 AM"), None); + // Hour 13 invalid in 12-hour format + assert_eq!(string_to_time("13:00:00 AM"), None); + assert_eq!(string_to_time("13:00:00 PM"), None); + } + + #[test] + fn test_invalid_inputs() { + assert_eq!(string_to_time(""), None); + assert_eq!(string_to_time(" "), None); + assert_eq!(string_to_time("XYZ"), None); + assert_eq!(string_to_time("24:00:00"), None); + assert_eq!(string_to_time("23:60:00"), None); + assert_eq!(string_to_time("23:00:60"), None); + // Date component present + assert_eq!(string_to_time("2025-03-09 00:00:00"), None); + // Timezone present + assert_eq!(string_to_time("00:01:02 UTC"), None); + // Just digits without separators + assert_eq!(string_to_time("120000"), None); + } + + #[test] + fn test_trailing_whitespace() { + assert_eq!(string_to_time("12:30:45 "), string_to_time("12:30:45")); + assert_eq!(string_to_time("1:00:00 AM "), string_to_time("1:00:00 AM")); + } + + #[test] + fn test_three_digit_components() { + // 3-digit hour/minute/second must be rejected (Spark requires 1-2 digits) + assert_eq!(string_to_time("001:02:03"), None); + assert_eq!(string_to_time("12:001:03"), None); + assert_eq!(string_to_time("12:02:003"), None); + } + + #[test] + fn test_leading_whitespace() { + assert_eq!(string_to_time(" 12:30"), string_to_time("12:30")); + assert_eq!(string_to_time(" 12:30:45"), string_to_time("12:30:45")); + assert_eq!(string_to_time(" 12:30:45 "), string_to_time("12:30:45")); + assert_eq!(string_to_time(" 1:00:00 AM"), string_to_time("1:00:00 AM")); + // Tabs and newlines are also trimmed (Spark's isWhitespaceOrISOControl) + assert_eq!(string_to_time("\t12:30:45"), string_to_time("12:30:45")); + assert_eq!(string_to_time("\n12:30:45"), string_to_time("12:30:45")); + // T-prefix is rejected when preceded by whitespace because Spark's + // parseTimestampString gates the T-prefix branch on j == 0 (start of + // the already-trimmed segment), so leading whitespace moves j past 0. + assert_eq!(string_to_time(" T12:30:45"), None); + assert_eq!(string_to_time(" T12:30"), None); + } +} diff --git a/native/spark-expr/src/datetime_funcs/unix_timestamp.rs b/native/spark-expr/src/datetime_funcs/unix_timestamp.rs index f8e932f396..bd62563a6b 100644 --- a/native/spark-expr/src/datetime_funcs/unix_timestamp.rs +++ b/native/spark-expr/src/datetime_funcs/unix_timestamp.rs @@ -24,7 +24,7 @@ use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; use num::integer::div_floor; -use std::{any::Any, fmt::Debug, sync::Arc}; +use std::{fmt::Debug, sync::Arc}; const MICROS_PER_SECOND: i64 = 1_000_000; @@ -46,10 +46,6 @@ impl SparkUnixTimestamp { } impl ScalarUDFImpl for SparkUnixTimestamp { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "unix_timestamp" } diff --git a/native/spark-expr/src/json_funcs/from_json.rs b/native/spark-expr/src/json_funcs/from_json.rs index 685ea3c8ec..eaca6db016 100644 --- a/native/spark-expr/src/json_funcs/from_json.rs +++ b/native/spark-expr/src/json_funcs/from_json.rs @@ -86,10 +86,6 @@ impl PartialEq for FromJson { } impl PhysicalExpr for FromJson { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/json_funcs/to_json.rs b/native/spark-expr/src/json_funcs/to_json.rs index 3cc827f210..6dcd8bc045 100644 --- a/native/spark-expr/src/json_funcs/to_json.rs +++ b/native/spark-expr/src/json_funcs/to_json.rs @@ -79,10 +79,6 @@ impl PartialEq for ToJson { } impl PhysicalExpr for ToJson { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/jvm_udf/mod.rs b/native/spark-expr/src/jvm_udf/mod.rs index 4ed25de6ee..0ca603ac9b 100644 --- a/native/spark-expr/src/jvm_udf/mod.rs +++ b/native/spark-expr/src/jvm_udf/mod.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; @@ -59,6 +58,10 @@ impl JvmScalarUdfExpr { return_nullable: bool, task_context: Option>>>, ) -> Self { + debug_assert!( + !class_name.is_empty(), + "JvmScalarUdfExpr requires a non-empty class name" + ); Self { class_name, args, @@ -103,10 +106,6 @@ impl PartialEq for JvmScalarUdfExpr { impl Eq for JvmScalarUdfExpr {} impl PhysicalExpr for JvmScalarUdfExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } @@ -120,10 +119,10 @@ impl PhysicalExpr for JvmScalarUdfExpr { } fn evaluate(&self, batch: &RecordBatch) -> DFResult { - // Step 1: evaluate child expressions to get Arrow arrays. Scalar children - // (e.g. literal patterns) are sent as length-1 vectors rather than expanded - // to batch-row count, so the JVM bridge does not pay an O(rows) copy for - // values that never vary across the batch. + // Scalar children (e.g. literal patterns) are sent as length-1 vectors rather than + // expanded to batch-row count, so the JVM bridge does not pay an O(rows) copy for + // values that never vary across the batch. The JVM side gets `numRows` directly via + // the bridge so it doesn't need the scalar to carry batch length. let arrays: Vec = self .args .iter() @@ -133,7 +132,6 @@ impl PhysicalExpr for JvmScalarUdfExpr { }) .collect::>()?; - // Step 2: allocate FFI structs on the Rust heap and collect their raw pointers. // The JVM writes into the out_array/out_schema slots and reads from the in_ slots. let in_ffi_arrays: Vec> = arrays .iter() @@ -157,7 +155,13 @@ impl PhysicalExpr for JvmScalarUdfExpr { .map(|b| b.as_ref() as *const FFI_ArrowSchema as i64) .collect(); - // Allocate output FFI slots. + debug_assert!(!self.class_name.is_empty(), "class_name must not be empty"); + debug_assert_eq!( + in_arr_ptrs.len(), + in_sch_ptrs.len(), + "input array and schema pointer counts must match" + ); + let mut out_array = Box::new(FFI_ArrowArray::empty()); let mut out_schema = Box::new(FFI_ArrowSchema::empty()); let out_arr_ptr = out_array.as_mut() as *mut FFI_ArrowArray as i64; @@ -166,7 +170,6 @@ impl PhysicalExpr for JvmScalarUdfExpr { let class_name = self.class_name.clone(); let n_args = arrays.len(); - // Step 3: attach a JNI env for this thread and call the static bridge method. JVMClasses::with_env(|env| { let bridge = JVMClasses::get().comet_udf_bridge.as_ref().ok_or_else(|| { CometError::from(ExecutionError::GeneralError( @@ -176,12 +179,10 @@ impl PhysicalExpr for JvmScalarUdfExpr { )) })?; - // Build the JVM String for the class name. let jclass_name = env .new_string(&class_name) .map_err(|e| CometError::JNI { source: e })?; - // Build the long[] arrays for input pointers. let in_arr_java = env .new_long_array(n_args) .map_err(|e| CometError::JNI { source: e })?; @@ -196,9 +197,10 @@ impl PhysicalExpr for JvmScalarUdfExpr { .set_region(env, 0, &in_sch_ptrs) .map_err(|e| CometError::JNI { source: e })?; - // Pass a null jobject when no TaskContext was propagated so the bridge's null-guard - // leaves the worker thread's current TaskContext.get() in place. The borrow must - // outlive `call_static_method_unchecked`. + // Resolve the TaskContext reference once before building the arg array so the + // borrow lives until `call_static_method_unchecked` returns. When no TaskContext + // was propagated, pass a null object so the bridge's null-guard leaves the thread- + // local alone. let null_task_context = JObject::null(); let task_context_ref: &JObject = match &self.task_context { Some(gref) => gref.as_obj(), @@ -229,7 +231,6 @@ impl PhysicalExpr for JvmScalarUdfExpr { Ok(()) })?; - // Step 4: import the result from the FFI slots filled by the JVM. // SAFETY: `*out_array` moves the FFI_ArrowArray out of the Box (the heap // allocation is freed by the move), and `from_ffi` wraps it in an Arc that // keeps the JVM-installed release callback alive until the resulting diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 4f2d5da469..f60f01f48e 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -75,9 +75,9 @@ pub use comet_scalar_funcs::{ }; pub use csv_funcs::*; pub use datetime_funcs::{ - SparkDateDiff, SparkDateFromUnixDate, SparkDateTrunc, SparkHour, SparkHoursTransform, - SparkMakeDate, SparkMinute, SparkSecond, SparkSecondsToTimestamp, SparkUnixTimestamp, - TimestampTruncExpr, + spark_to_time, SparkDateDiff, SparkDateFromUnixDate, SparkDateTrunc, SparkHour, + SparkHoursTransform, SparkMakeDate, SparkMakeTime, SparkMinute, SparkSecond, + SparkSecondsToTimestamp, SparkUnixTimestamp, TimestampTruncExpr, }; pub use error::{decimal_overflow_error, SparkError, SparkErrorWithContext, SparkResult}; pub use hash_funcs::*; diff --git a/native/spark-expr/src/math_funcs/internal/checkoverflow.rs b/native/spark-expr/src/math_funcs/internal/checkoverflow.rs index f1fb9c2f02..f867fe0c4d 100644 --- a/native/spark-expr/src/math_funcs/internal/checkoverflow.rs +++ b/native/spark-expr/src/math_funcs/internal/checkoverflow.rs @@ -28,7 +28,6 @@ use std::hash::Hash; use crate::SparkError; use std::{ - any::Any, fmt::{Display, Formatter}, sync::Arc, }; @@ -91,10 +90,6 @@ impl Display for CheckOverflow { } impl PhysicalExpr for CheckOverflow { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } @@ -274,9 +269,6 @@ mod tests { } impl PhysicalExpr for ScalarChild { - fn as_any(&self) -> &dyn Any { - self - } fn data_type(&self, _: &Schema) -> datafusion::common::Result { Ok(DataType::Decimal128(self.1, self.2)) } diff --git a/native/spark-expr/src/math_funcs/internal/decimal_rescale_check.rs b/native/spark-expr/src/math_funcs/internal/decimal_rescale_check.rs index 1322404951..667f76939a 100644 --- a/native/spark-expr/src/math_funcs/internal/decimal_rescale_check.rs +++ b/native/spark-expr/src/math_funcs/internal/decimal_rescale_check.rs @@ -29,7 +29,6 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::hash::Hash; use std::{ - any::Any, fmt::{Display, Formatter}, sync::Arc, }; @@ -154,10 +153,6 @@ fn rescale_and_check( } impl PhysicalExpr for DecimalRescaleCheckOverflow { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } @@ -392,9 +387,6 @@ mod tests { } impl PhysicalExpr for ScalarChild { - fn as_any(&self) -> &dyn Any { - self - } fn data_type(&self, _: &Schema) -> datafusion::common::Result { Ok(DataType::Decimal128(self.1, self.2)) } diff --git a/native/spark-expr/src/math_funcs/internal/normalize_nan.rs b/native/spark-expr/src/math_funcs/internal/normalize_nan.rs index b3838f64f4..165ae3acc9 100644 --- a/native/spark-expr/src/math_funcs/internal/normalize_nan.rs +++ b/native/spark-expr/src/math_funcs/internal/normalize_nan.rs @@ -26,7 +26,6 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::hash::Hash; use std::{ - any::Any, fmt::{Display, Formatter}, sync::Arc, }; @@ -57,10 +56,6 @@ impl NormalizeNaNAndZero { } impl PhysicalExpr for NormalizeNaNAndZero { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/math_funcs/negative.rs b/native/spark-expr/src/math_funcs/negative.rs index a268894086..650fa401ef 100644 --- a/native/spark-expr/src/math_funcs/negative.rs +++ b/native/spark-expr/src/math_funcs/negative.rs @@ -29,7 +29,7 @@ use datafusion::{ }; use std::fmt::{Display, Formatter}; use std::hash::Hash; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; pub fn create_negate_expr( expr: Arc, @@ -96,11 +96,6 @@ impl std::fmt::Display for NegativeExpr { } impl PhysicalExpr for NegativeExpr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, input_schema: &Schema) -> Result { self.arg.data_type(input_schema) } diff --git a/native/spark-expr/src/math_funcs/wide_decimal_binary_expr.rs b/native/spark-expr/src/math_funcs/wide_decimal_binary_expr.rs index 7ff78713be..ca4869357e 100644 --- a/native/spark-expr/src/math_funcs/wide_decimal_binary_expr.rs +++ b/native/spark-expr/src/math_funcs/wide_decimal_binary_expr.rs @@ -31,7 +31,7 @@ use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::fmt::{Display, Formatter}; use std::hash::Hash; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; /// The arithmetic operation to perform. #[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)] @@ -165,10 +165,6 @@ fn max_for_precision(precision: u8) -> i256 { } impl PhysicalExpr for WideDecimalBinaryExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, _input_schema: &Schema) -> Result { Ok(DataType::Decimal128( self.output_precision, diff --git a/native/spark-expr/src/nondetermenistic_funcs/monotonically_increasing_id.rs b/native/spark-expr/src/nondetermenistic_funcs/monotonically_increasing_id.rs index 49a5066a38..e7404e00c1 100644 --- a/native/spark-expr/src/nondetermenistic_funcs/monotonically_increasing_id.rs +++ b/native/spark-expr/src/nondetermenistic_funcs/monotonically_increasing_id.rs @@ -20,7 +20,6 @@ use arrow::datatypes::{DataType, Schema}; use datafusion::common::Result; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; -use std::any::Any; use std::fmt::{Debug, Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::atomic::{AtomicI64, Ordering}; @@ -66,10 +65,6 @@ impl Hash for MonotonicallyIncreasingId { } impl PhysicalExpr for MonotonicallyIncreasingId { - fn as_any(&self) -> &dyn Any { - self - } - fn evaluate(&self, batch: &RecordBatch) -> Result { let start = self .current_offset diff --git a/native/spark-expr/src/nondetermenistic_funcs/rand.rs b/native/spark-expr/src/nondetermenistic_funcs/rand.rs index e23a83d84e..8f07f37344 100644 --- a/native/spark-expr/src/nondetermenistic_funcs/rand.rs +++ b/native/spark-expr/src/nondetermenistic_funcs/rand.rs @@ -23,7 +23,6 @@ use arrow::datatypes::{DataType, Schema}; use datafusion::common::Result; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; -use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::{Arc, Mutex}; @@ -120,10 +119,6 @@ impl Hash for RandExpr { } impl PhysicalExpr for RandExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, _input_schema: &Schema) -> Result { Ok(DataType::Float64) } diff --git a/native/spark-expr/src/nondetermenistic_funcs/randn.rs b/native/spark-expr/src/nondetermenistic_funcs/randn.rs index 40fafedc20..7d50d24811 100644 --- a/native/spark-expr/src/nondetermenistic_funcs/randn.rs +++ b/native/spark-expr/src/nondetermenistic_funcs/randn.rs @@ -22,7 +22,6 @@ use arrow::array::RecordBatch; use arrow::datatypes::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; -use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::{Arc, Mutex}; @@ -131,10 +130,6 @@ impl Hash for RandnExpr { } impl PhysicalExpr for RandnExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, _input_schema: &Schema) -> datafusion::common::Result { Ok(DataType::Float64) } diff --git a/native/spark-expr/src/predicate_funcs/rlike.rs b/native/spark-expr/src/predicate_funcs/rlike.rs index ed5970a6a2..ee005dd1ac 100644 --- a/native/spark-expr/src/predicate_funcs/rlike.rs +++ b/native/spark-expr/src/predicate_funcs/rlike.rs @@ -25,7 +25,6 @@ use datafusion::common::{internal_err, Result, ScalarValue}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::ColumnarValue; use regex::Regex; -use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; @@ -102,10 +101,6 @@ impl Display for RLike { } impl PhysicalExpr for RLike { - fn as_any(&self) -> &dyn Any { - self - } - fn data_type(&self, _input_schema: &Schema) -> Result { Ok(DataType::Boolean) } diff --git a/native/spark-expr/src/string_funcs/contains.rs b/native/spark-expr/src/string_funcs/contains.rs index bc34ce9cba..537227efdf 100644 --- a/native/spark-expr/src/string_funcs/contains.rs +++ b/native/spark-expr/src/string_funcs/contains.rs @@ -27,7 +27,6 @@ use datafusion::common::{exec_err, Result, ScalarValue}; use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -use std::any::Any; use std::sync::Arc; /// Spark-optimized contains function. @@ -53,10 +52,6 @@ impl SparkContains { } impl ScalarUDFImpl for SparkContains { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { "contains" } diff --git a/native/spark-expr/src/string_funcs/substring.rs b/native/spark-expr/src/string_funcs/substring.rs index dff32c10d7..9226241d86 100644 --- a/native/spark-expr/src/string_funcs/substring.rs +++ b/native/spark-expr/src/string_funcs/substring.rs @@ -24,7 +24,6 @@ use arrow::record_batch::RecordBatch; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::{ - any::Any, fmt::{Display, Formatter}, hash::Hash, sync::Arc, @@ -68,10 +67,6 @@ impl Display for SubstringExpr { } impl PhysicalExpr for SubstringExpr { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/struct_funcs/create_named_struct.rs b/native/spark-expr/src/struct_funcs/create_named_struct.rs index 70e03ad0c0..1a63cb1cc9 100644 --- a/native/spark-expr/src/struct_funcs/create_named_struct.rs +++ b/native/spark-expr/src/struct_funcs/create_named_struct.rs @@ -22,7 +22,6 @@ use datafusion::common::Result as DataFusionResult; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::{ - any::Any, fmt::{Display, Formatter}, hash::Hash, sync::Arc, @@ -53,10 +52,6 @@ impl CreateNamedStruct { } impl PhysicalExpr for CreateNamedStruct { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/struct_funcs/get_struct_field.rs b/native/spark-expr/src/struct_funcs/get_struct_field.rs index 7929cea483..6684967c3a 100644 --- a/native/spark-expr/src/struct_funcs/get_struct_field.rs +++ b/native/spark-expr/src/struct_funcs/get_struct_field.rs @@ -22,7 +22,6 @@ use datafusion::common::{DataFusionError, Result as DataFusionResult, ScalarValu use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr::PhysicalExpr; use std::{ - any::Any, fmt::{Display, Formatter}, hash::Hash, sync::Arc, @@ -62,10 +61,6 @@ impl GetStructField { } impl PhysicalExpr for GetStructField { - fn as_any(&self) -> &dyn Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/native/spark-expr/src/unbound.rs b/native/spark-expr/src/unbound.rs index cf0adafa91..69187dbc02 100644 --- a/native/spark-expr/src/unbound.rs +++ b/native/spark-expr/src/unbound.rs @@ -59,11 +59,6 @@ impl std::fmt::Display for UnboundColumn { } impl PhysicalExpr for UnboundColumn { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn std::any::Any { - self - } - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { Display::fmt(self, f) } diff --git a/spark-integration/pom.xml b/spark-integration/pom.xml index 356f91adfd..9aacf8db23 100644 --- a/spark-integration/pom.xml +++ b/spark-integration/pom.xml @@ -45,11 +45,19 @@ under the License. comet-spark-spark${spark.version.short}_${scala.binary.version} ${project.version} - - - org.apache.datafusion - comet-common-spark${spark.version.short}_${scala.binary.version} - + + + org.apache.datafusion + comet-common-spark${spark.version.short}_${scala.binary.version} + + + + org.apache.arrow + * + diff --git a/spark/pom.xml b/spark/pom.xml index d3c18ccf87..6d97ea831f 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -43,17 +43,27 @@ under the License. org.apache.datafusion comet-common-spark${spark.version.short}_${scala.binary.version} ${project.version} - - - org.apache.arrow - * - - org.apache.spark spark-sql_${scala.binary.version} + + org.apache.parquet + parquet-column + + + org.apache.parquet + parquet-format-structures + + + org.apache.arrow + arrow-vector + + + org.scala-lang.modules + scala-collection-compat_${scala.binary.version} + org.scala-lang scala-library @@ -128,18 +138,16 @@ under the License. - + org.apache.arrow arrow-memory-unsafe - test org.apache.arrow arrow-c-data - test org.apache.hadoop @@ -355,6 +363,31 @@ under the License. + + io.github.git-commit-id + git-commit-id-maven-plugin + ${git-commit-id-maven-plugin.version} + + + get-the-git-infos + + revision + + initialize + + + + true + ${project.build.outputDirectory}/comet-git-info.properties + full + + ^git.branch$ + ^git.build.*$ + ^git.commit.id.(abbrev|full)$ + ^git.remote.*$ + + + com.github.os72 protoc-jar-maven-plugin @@ -394,11 +427,13 @@ under the License. true + org.apache.datafusion:comet-common-spark${spark.version.short}_${scala.binary.version} + + org.apache.arrow:* com.google.protobuf:protobuf-java com.google.guava:guava - org.scala-lang.modules:scala-collection-compat_${scala.binary.version} @@ -410,9 +445,17 @@ under the License. git.properties log4j.properties log4j2.properties + arrow-git.properties **/SparkFilterApi.* + + org.apache.arrow:arrow-vector + + + codegen/** + + org.apache.parquet:parquet-hadoop:tests @@ -422,6 +465,20 @@ under the License. + + org.apache.arrow + ${comet.shade.packageName}.arrow + + + org/apache/arrow/c/jni/JniWrapper + org/apache/arrow/c/jni/PrivateData + org/apache/arrow/c/jni/CDataJniException + + org/apache/arrow/c/ArrayStreamExporter$ExportedArrayStreamPrivateData + + com.google.protobuf ${comet.shade.packageName}.protobuf @@ -492,6 +549,34 @@ under the License. + + + ${project.basedir}/src/main/resources + + + ${project.basedir}/../native/target/x86_64-apple-darwin/release + + libcomet.dylib + + org/apache/comet/darwin/x86_64 + + + ${project.basedir}/../native/target/aarch64-apple-darwin/release + + libcomet.dylib + + org/apache/comet/darwin/aarch64 + + + ${jni.dir} + + libcomet.dylib + libcomet.so + comet.dll + + org/apache/comet/${platform}/${arch} + + diff --git a/common/src/main/java/org/apache/arrow/c/ArrowImporter.java b/spark/src/main/java/org/apache/arrow/c/ArrowImporter.java similarity index 100% rename from common/src/main/java/org/apache/arrow/c/ArrowImporter.java rename to spark/src/main/java/org/apache/arrow/c/ArrowImporter.java diff --git a/common/src/main/java/org/apache/comet/NativeBase.java b/spark/src/main/java/org/apache/comet/NativeBase.java similarity index 100% rename from common/src/main/java/org/apache/comet/NativeBase.java rename to spark/src/main/java/org/apache/comet/NativeBase.java diff --git a/spark/src/main/java/org/apache/comet/codegen/CometBatchKernel.java b/spark/src/main/java/org/apache/comet/codegen/CometBatchKernel.java new file mode 100644 index 0000000000..a515cbe32d --- /dev/null +++ b/spark/src/main/java/org/apache/comet/codegen/CometBatchKernel.java @@ -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.codegen; + +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.ValueVector; + +/** + * Abstract base extended by the Janino-compiled batch kernel emitted by {@code + * CometBatchKernelCodegen}. The generated subclass extends {@code CometInternalRow} (so Spark's + * {@code BoundReference.genCode} can call {@code this.getUTF8String(ord)} directly) and carries + * typed input fields baked at codegen time, one per input column. Expression evaluation plus Arrow + * read/write fuse into one method per expression tree. + */ +public abstract class CometBatchKernel extends CometInternalRow { + + protected final Object[] references; + + protected CometBatchKernel(Object[] references) { + this.references = references; + } + + /** + * Run partition-dependent initialization. The generated subclass overrides this to execute + * statements collected via {@code CodegenContext.addPartitionInitializationStatement}, e.g. + * reseeding {@code Rand}'s {@code XORShiftRandom} from {@code seed + partitionIndex}. + * Deterministic expressions leave this as a no-op. + * + *

The caller invokes this before the first {@code process} call of each partition. The + * generated subclass is not thread-safe across concurrent {@code process} calls. The dispatcher + * allocates one per partition and serializes calls. + */ + public void init(int partitionIndex) {} + + /** + * Process one batch. + * + * @param inputs Arrow input vectors. Length and concrete classes match the schema the kernel was + * compiled against. + * @param output Arrow output vector. Caller allocates to the expression's {@code dataType}. + * @param numRows number of rows in this batch + */ + public abstract void process(ValueVector[] inputs, FieldVector output, int numRows); +} diff --git a/common/src/main/java/org/apache/comet/parquet/CometFileKeyUnwrapper.java b/spark/src/main/java/org/apache/comet/parquet/CometFileKeyUnwrapper.java similarity index 100% rename from common/src/main/java/org/apache/comet/parquet/CometFileKeyUnwrapper.java rename to spark/src/main/java/org/apache/comet/parquet/CometFileKeyUnwrapper.java diff --git a/common/src/main/java/org/apache/comet/udf/CometUdfBridge.java b/spark/src/main/java/org/apache/comet/udf/CometUdfBridge.java similarity index 55% rename from common/src/main/java/org/apache/comet/udf/CometUdfBridge.java rename to spark/src/main/java/org/apache/comet/udf/CometUdfBridge.java index 5e76819810..9e97ef2226 100644 --- a/common/src/main/java/org/apache/comet/udf/CometUdfBridge.java +++ b/spark/src/main/java/org/apache/comet/udf/CometUdfBridge.java @@ -29,18 +29,48 @@ import org.apache.arrow.vector.ValueVector; import org.apache.spark.TaskContext; import org.apache.spark.comet.CometTaskContextShim; +import org.apache.spark.util.TaskCompletionListener; /** * JNI entry point for native execution to invoke a {@link CometUDF}. Matches the static-method * pattern used by CometScalarSubquery so the native side can dispatch via * call_static_method_unchecked. + * + *

Cache invariants: + * + *

    + *
  1. For each live Spark task attempt there is at most one {@link CometUDF} instance per class + * name. + *
  2. A {@link CometUDF} instance is visible only within the Spark task attempt that instantiated + * it. Two task attempts observing the same class name receive distinct instances. + *
  3. At any instant at most one thread is inside {@code evaluate()} for a given {@code + * taskAttemptId}. This follows from Spark executing one native future per partition and Tokio + * polling one future per worker at a time. + *
  4. All instances for a task are dropped by the {@link TaskCompletionListener} registered on + * the first cache miss for that task. No cache entry outlives its task. + *
  5. When {@code taskContext} is {@code null} (unit tests, direct native driver) the fallback + * key {@code -1L} is used; that bucket is never evicted because no task-completion event will + * fire. + *
+ * + *

Keying by {@code taskAttemptId} rather than by thread keeps the cache correct under Tokio + * work-stealing: on the scan-free execution path the same Spark task can be polled by different + * Tokio workers across batches, so a thread-local cache would lose per-task state on migration. The + * task attempt ID is stable for the life of the task regardless of which worker is polling. */ public class CometUdfBridge { - // Process-wide cache of UDF instances keyed by class name. CometUDF - // implementations are required to be stateless (see CometUDF), so a - // single shared instance per class is safe across native worker threads. - private static final ConcurrentHashMap INSTANCES = new ConcurrentHashMap<>(); + /** + * Task-scoped cache of {@link CometUDF} instances. Outer map keys are Spark task attempt IDs (or + * {@code -1L} when no {@link TaskContext} is available). Inner maps hold one instance per UDF + * class name for the task's lifetime. Entries are removed by the {@link TaskCompletionListener} + * registered on the first cache miss per task. + */ + private static final ConcurrentHashMap> INSTANCES = + new ConcurrentHashMap<>(); + + /** Sentinel key for calls that carry no {@link TaskContext} (unit tests, direct driver). */ + private static final long NO_TASK_ID = -1L; /** * Called from native via JNI. @@ -58,7 +88,9 @@ public class CometUdfBridge { * thread-local on entry, with the prior value (if any) saved and restored in {@code finally}. * Lets partition-sensitive built-ins ({@code Rand}, {@code Uuid}, {@code * MonotonicallyIncreasingID}) work from Tokio workers and avoids reusing a stale TaskContext - * left on a worker by a previous task. + * left on a worker by a previous task. Its task attempt ID also keys the UDF-instance cache, + * so a UDF holding per-task state in fields sees a consistent instance for every call within + * the task regardless of which Tokio worker is polling. */ public static void evaluate( String udfClassName, @@ -68,16 +100,33 @@ public static void evaluate( long outSchemaPtr, int numRows, TaskContext taskContext) { - // Save-and-restore rather than only-install-if-null: the propagated context is the ground - // truth for this call. Any value already on the thread is either (a) the same object on a - // Spark task thread, or (b) stale from a prior task on a reused Tokio worker. + assert udfClassName != null && !udfClassName.isEmpty() : "udfClassName must be non-empty"; + assert inputArrayPtrs != null && inputSchemaPtrs != null + : "input pointer arrays must be non-null"; + assert inputArrayPtrs.length == inputSchemaPtrs.length + : "input array pointer count must equal schema pointer count"; + assert numRows >= 0 : "numRows must be non-negative"; + assert outArrayPtr != 0L : "outArrayPtr must be a valid FFI pointer"; + assert outSchemaPtr != 0L : "outSchemaPtr must be a valid FFI pointer"; + + // Save-and-restore rather than only-install-if-null: the propagated `taskContext` is the + // ground truth for this call. Any value already on the thread is either (a) the same object + // on a Spark task thread, or (b) stale from a prior task on a reused Tokio worker. TaskContext prior = TaskContext.get(); if (taskContext != null) { CometTaskContextShim.set(taskContext); + assert TaskContext.get() == taskContext + : "TaskContext install did not take effect on this thread"; } try { evaluateInternal( - udfClassName, inputArrayPtrs, inputSchemaPtrs, outArrayPtr, outSchemaPtr, numRows); + udfClassName, + inputArrayPtrs, + inputSchemaPtrs, + outArrayPtr, + outSchemaPtr, + numRows, + taskContext); } finally { if (taskContext != null) { if (prior != null) { @@ -95,9 +144,34 @@ private static void evaluateInternal( long[] inputSchemaPtrs, long outArrayPtr, long outSchemaPtr, - int numRows) { - CometUDF udf = + int numRows, + TaskContext taskContext) { + long taskAttemptId = (taskContext != null) ? taskContext.taskAttemptId() : NO_TASK_ID; + + ConcurrentHashMap perTask = INSTANCES.computeIfAbsent( + taskAttemptId, + id -> { + ConcurrentHashMap fresh = new ConcurrentHashMap<>(); + if (taskContext != null) { + // computeIfAbsent runs this lambda at most once per key, so the listener is + // registered exactly once per task attempt. + taskContext.addTaskCompletionListener( + (TaskCompletionListener) + ctx -> { + ConcurrentHashMap removed = INSTANCES.remove(id); + assert removed != null + : "task-completion listener fired but cache already removed " + + "entry for task " + + id; + }); + } + return fresh; + }); + assert perTask != null : "per-task cache must be non-null after computeIfAbsent"; + + CometUDF udf = + perTask.computeIfAbsent( udfClassName, name -> { try { @@ -113,6 +187,7 @@ private static void evaluateInternal( throw new RuntimeException("Failed to instantiate CometUDF: " + name, e); } }); + assert udf != null : "reflective instantiation returned null for " + udfClassName; BufferAllocator allocator = org.apache.comet.package$.MODULE$.CometArrowAllocator(); diff --git a/common/src/main/java/org/apache/comet/vector/CometDecodedVector.java b/spark/src/main/java/org/apache/comet/vector/CometDecodedVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometDecodedVector.java rename to spark/src/main/java/org/apache/comet/vector/CometDecodedVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometDelegateVector.java b/spark/src/main/java/org/apache/comet/vector/CometDelegateVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometDelegateVector.java rename to spark/src/main/java/org/apache/comet/vector/CometDelegateVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionary.java b/spark/src/main/java/org/apache/comet/vector/CometDictionary.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometDictionary.java rename to spark/src/main/java/org/apache/comet/vector/CometDictionary.java diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java b/spark/src/main/java/org/apache/comet/vector/CometDictionaryVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometDictionaryVector.java rename to spark/src/main/java/org/apache/comet/vector/CometDictionaryVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometListVector.java b/spark/src/main/java/org/apache/comet/vector/CometListVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometListVector.java rename to spark/src/main/java/org/apache/comet/vector/CometListVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometMapVector.java b/spark/src/main/java/org/apache/comet/vector/CometMapVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometMapVector.java rename to spark/src/main/java/org/apache/comet/vector/CometMapVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometPlainVector.java b/spark/src/main/java/org/apache/comet/vector/CometPlainVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometPlainVector.java rename to spark/src/main/java/org/apache/comet/vector/CometPlainVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometSelectionVector.java b/spark/src/main/java/org/apache/comet/vector/CometSelectionVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometSelectionVector.java rename to spark/src/main/java/org/apache/comet/vector/CometSelectionVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometStructVector.java b/spark/src/main/java/org/apache/comet/vector/CometStructVector.java similarity index 100% rename from common/src/main/java/org/apache/comet/vector/CometStructVector.java rename to spark/src/main/java/org/apache/comet/vector/CometStructVector.java diff --git a/common/src/main/java/org/apache/comet/vector/CometVector.java b/spark/src/main/java/org/apache/comet/vector/CometVector.java similarity index 98% rename from common/src/main/java/org/apache/comet/vector/CometVector.java rename to spark/src/main/java/org/apache/comet/vector/CometVector.java index 6dda765d51..f922f2281c 100644 --- a/common/src/main/java/org/apache/comet/vector/CometVector.java +++ b/spark/src/main/java/org/apache/comet/vector/CometVector.java @@ -39,10 +39,7 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; -import org.apache.comet.IcebergApi; - /** Base class for all Comet column vector implementations. */ -@IcebergApi public abstract class CometVector extends ColumnVector { private static final int DECIMAL_BYTE_WIDTH = 16; private final byte[] DECIMAL_BYTES = new byte[DECIMAL_BYTE_WIDTH]; @@ -61,7 +58,6 @@ public abstract class CometVector extends ColumnVector { } } - @IcebergApi public CometVector(DataType type, boolean useDecimal128) { super(type); this.useDecimal128 = useDecimal128; @@ -71,18 +67,15 @@ public CometVector(DataType type, boolean useDecimal128) { * Sets the number of nulls in this vector to be 'numNulls'. This is used when the vector is * reused across batches. */ - @IcebergApi public abstract void setNumNulls(int numNulls); /** * Sets the number of values (including both nulls and non-nulls) in this vector to be * 'numValues'. This is used when the vector is reused across batches. */ - @IcebergApi public abstract void setNumValues(int numValues); /** Returns the number of values in this vector. */ - @IcebergApi public abstract int numValues(); /** Whether the elements of this vector are of fixed length. */ @@ -222,7 +215,6 @@ public DictionaryProvider getDictionaryProvider() { throw new UnsupportedOperationException("Not implemented"); } - @IcebergApi public abstract ValueVector getValueVector(); /** @@ -232,7 +224,6 @@ public DictionaryProvider getDictionaryProvider() { * @param length the length of the new vector * @return the new vector */ - @IcebergApi public abstract CometVector slice(int offset, int length); /** diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/spark/src/main/scala/org/apache/comet/CometConf.scala similarity index 97% rename from common/src/main/scala/org/apache/comet/CometConf.scala rename to spark/src/main/scala/org/apache/comet/CometConf.scala index 9b376837f7..fdd1ae2073 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/spark/src/main/scala/org/apache/comet/CometConf.scala @@ -111,24 +111,6 @@ object CometConf extends ShimCometConf { .booleanConf .createWithEnvVarOrDefault("ENABLE_COMET_WRITE", false) - val SCAN_NATIVE_DATAFUSION = "native_datafusion" - val SCAN_NATIVE_ICEBERG_COMPAT = "native_iceberg_compat" - val SCAN_AUTO = "auto" - - val COMET_NATIVE_SCAN_IMPL: ConfigEntry[String] = conf("spark.comet.scan.impl") - .category(CATEGORY_PARQUET) - .doc( - "The implementation of Comet's Parquet scan to use. Available scans are " + - s"`$SCAN_NATIVE_DATAFUSION`, and `$SCAN_NATIVE_ICEBERG_COMPAT`. " + - s"`$SCAN_NATIVE_DATAFUSION` is a fully native implementation, and " + - s"`$SCAN_NATIVE_ICEBERG_COMPAT` is a hybrid implementation that supports some " + - "additional features, such as row indexes and field ids. " + - s"`$SCAN_AUTO` (default) chooses the best available scan based on the scan schema.") - .stringConf - .transform(_.toLowerCase(Locale.ROOT)) - .checkValues(Set(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) - .createWithEnvVarOrDefault("COMET_PARQUET_SCAN_IMPL", SCAN_AUTO) - val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.icebergNative.enabled") .category(CATEGORY_SCAN) @@ -380,6 +362,17 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_SCALA_UDF_CODEGEN_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.exec.scalaUDF.codegen.enabled") + .category(CATEGORY_EXEC) + .doc("Experimental. Whether to route Spark `ScalaUDF` expressions through Comet's " + + "Arrow-direct codegen dispatcher. When enabled, a supported ScalaUDF is compiled into " + + "a per-batch kernel that reads and writes Arrow vectors directly from native " + + "execution. When disabled, plans containing a ScalaUDF fall back to Spark for the " + + "enclosing operator.") + .booleanConf + .createWithDefault(false) + val COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.native.shuffle.partitioning.hash.enabled") .category(CATEGORY_SHUFFLE) @@ -727,16 +720,6 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(true) - val COMET_SCHEMA_EVOLUTION_ENABLED: ConfigEntry[Boolean] = - conf("spark.comet.schemaEvolution.enabled") - .internal() - .category(CATEGORY_SCAN) - .doc("Whether to enable schema evolution in Comet. For instance, promoting a integer " + - "column to a long column, a float column to a double column, etc. This is automatically" + - "enabled when reading from Iceberg tables.") - .booleanConf - .createWithDefault(COMET_SCHEMA_EVOLUTION_ENABLED_DEFAULT) - val COMET_ENABLE_PARTIAL_HASH_AGGREGATE: ConfigEntry[Boolean] = conf("spark.comet.testing.aggregate.partialMode.enabled") .internal() @@ -793,6 +776,19 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(true) + val COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER: ConfigEntry[Boolean] = + conf("spark.comet.scan.allowDisabledParquetVectorizedReader") + .category(CATEGORY_SCAN) + .doc( + "Whether to allow Comet's native scan to replace the Parquet scan when Spark's " + + s"${SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key} is set to false. By default " + + "(false), Comet falls back to Spark in that case, because Comet's native readers " + + "mirror Spark's vectorized reader semantics rather than Spark's parquet-mr " + + "(non-vectorized) semantics, which permit silent overflow / null-on-narrowing " + + s"that Comet has no equivalent for. $COMPAT_GUIDE.") + .booleanConf + .createWithDefault(false) + val COMET_EXEC_STRICT_FLOATING_POINT: ConfigEntry[Boolean] = conf("spark.comet.exec.strictFloatingPoint") .category(CATEGORY_EXEC) diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 679005d9b1..1ae90e1845 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -127,6 +127,16 @@ object CometSparkSessionExtensions extends Logging { return false } + if (COMET_EXEC_SHUFFLE_ENABLED.get(conf) && !isCometShuffleManagerEnabled(conf)) { + logWarning( + "Comet extension is disabled because spark.shuffle.manager is not set to " + + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager. " + + "Comet provides limited benefit without its shuffle manager. " + + s"Set ${COMET_EXEC_SHUFFLE_ENABLED.key}=false to keep Comet enabled with " + + "Spark's default shuffle manager.") + return false + } + // We don't support INT96 timestamps written by Apache Impala in a different timezone yet if (conf.getConf(SQLConf.PARQUET_INT96_TIMESTAMP_CONVERSION)) { logWarning( diff --git a/common/src/main/scala/org/apache/comet/Constants.scala b/spark/src/main/scala/org/apache/comet/Constants.scala similarity index 100% rename from common/src/main/scala/org/apache/comet/Constants.scala rename to spark/src/main/scala/org/apache/comet/Constants.scala diff --git a/spark/src/main/scala/org/apache/comet/GenerateDocs.scala b/spark/src/main/scala/org/apache/comet/GenerateDocs.scala index 870fb5e47d..43f0016d79 100644 --- a/spark/src/main/scala/org/apache/comet/GenerateDocs.scala +++ b/spark/src/main/scala/org/apache/comet/GenerateDocs.scala @@ -129,6 +129,16 @@ object GenerateDocs { serde.getCompatibleNotes(), serde.getIncompatibleReasons(), serde.getUnsupportedReasons()) + })), + "url" -> (( + "compatibility/expressions/url.md", + () => + QueryPlanSerde.urlExpressions.toSeq.map { case (cls, serde) => + ( + cls.getSimpleName, + serde.getCompatibleNotes(), + serde.getIncompatibleReasons(), + serde.getUnsupportedReasons()) }))) def main(args: Array[String]): Unit = { diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometArrayData.scala b/spark/src/main/scala/org/apache/comet/codegen/CometArrayData.scala new file mode 100644 index 0000000000..308d1e9d96 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometArrayData.scala @@ -0,0 +1,99 @@ +/* + * 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.codegen + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} + +import org.apache.comet.shims.CometInternalRowShim + +/** + * Throwing-default `ArrayData` base for the codegen kernel. Subclasses override only the getters + * their element type needs. + * + * Consumer: per-column `InputArray_${path}` nested classes that back `getArray(ord)` plus the + * recursion for `Array>` and array-typed map keys / struct fields. + * + * `ArrayData` and `InternalRow` are sibling abstract classes, so a base aimed at one cannot serve + * the other. The shared `get(ordinal, dataType)` dispatch lives in + * [[CometSpecializedGettersDispatch]]. Mixes in [[CometInternalRowShim]] so Spark 4.x's + * `getVariant` / `getGeography` / `getGeometry` get throwing defaults. + */ +abstract class CometArrayData extends ArrayData with CometInternalRowShim { + + override def getInterval(ordinal: Int): CalendarInterval = unsupported("getInterval") + + override def get(ordinal: Int, dataType: DataType): AnyRef = + CometSpecializedGettersDispatch.get(this, ordinal, dataType) + + override def isNullAt(ordinal: Int): Boolean = unsupported("isNullAt") + + override def getBoolean(ordinal: Int): Boolean = unsupported("getBoolean") + + override def getByte(ordinal: Int): Byte = unsupported("getByte") + + override def getShort(ordinal: Int): Short = unsupported("getShort") + + override def getInt(ordinal: Int): Int = unsupported("getInt") + + override def getLong(ordinal: Int): Long = unsupported("getLong") + + override def getFloat(ordinal: Int): Float = unsupported("getFloat") + + override def getDouble(ordinal: Int): Double = unsupported("getDouble") + + override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = + unsupported("getDecimal") + + override def getUTF8String(ordinal: Int): UTF8String = unsupported("getUTF8String") + + override def getBinary(ordinal: Int): Array[Byte] = unsupported("getBinary") + + override def getStruct(ordinal: Int, numFields: Int): InternalRow = unsupported("getStruct") + + override def getArray(ordinal: Int): ArrayData = unsupported("getArray") + + override def getMap(ordinal: Int): MapData = unsupported("getMap") + + override def setNullAt(i: Int): Unit = unsupported("setNullAt") + + protected def unsupported(method: String): Nothing = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: $method not implemented for this array shape") + + override def update(i: Int, value: Any): Unit = unsupported("update") + + override def copy(): ArrayData = unsupported("copy") + + override def array: Array[Any] = unsupported("array") + + override def toString(): String = { + val n = + try numElements().toString + catch { + case _: Throwable => "?" + } + s"${getClass.getSimpleName}(numElements=$n)" + } + + override def numElements(): Int = unsupported("numElements") +} diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegen.scala b/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegen.scala new file mode 100644 index 0000000000..2795911da3 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegen.scala @@ -0,0 +1,494 @@ +/* + * 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.codegen + +import org.apache.arrow.vector._ +import org.apache.arrow.vector.complex.{ListVector, MapVector, StructVector} +import org.apache.arrow.vector.types.pojo.Field +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, HigherOrderFunction, LambdaFunction, Literal, NamedLambdaVariable, Unevaluable} +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import org.apache.comet.shims.CometExprTraitShim + +/** + * Compiles a bound [[Expression]] plus an Arrow input schema into a [[CometBatchKernel]] that + * fuses Arrow input reads, Spark expression evaluation, and Arrow output writes into one + * Janino-compiled method per `(expression, schema)` pair. + * + * The kernel compiles any bound Catalyst expression. The tree need not be rooted at a `ScalaUDF`. + * Today's only consumer is [[org.apache.comet.udf.codegen.CometScalaUDFCodegen]]. + * + * Constraints: one output vector per kernel, per-row scalar evaluation only (aggregate, window, + * generator are rejected by [[canHandle]]). + * + * Input- and output-side emission live in [[CometBatchKernelCodegenInput]] and + * [[CometBatchKernelCodegenOutput]]. This file owns the [[ArrowColumnSpec]] vocabulary, the + * [[canHandle]] / [[allocateOutput]] / [[compile]] / [[generateSource]] entry points, and + * cross-cutting kernel-shape decisions (NullIntolerant short-circuit, CSE variant). + * + * The generated kernel is the `InternalRow` that Spark's `BoundReference.genCode` reads from. See + * [[generateSource]] for how the wiring is set up. + */ +object CometBatchKernelCodegen extends Logging with CometExprTraitShim { + + /** + * Resolve an Arrow vector class by simple name through the codegen object's own classloader. + * Tests use this to refer to vector classes via the same classloader the codegen pattern- + * matches against, in case the test classpath ever diverges from the codegen's (e.g. through + * future shading rearrangement). + */ + def vectorClassBySimpleName(name: String): Class[_ <: ValueVector] = name match { + case "BitVector" => classOf[BitVector] + case "TinyIntVector" => classOf[TinyIntVector] + case "SmallIntVector" => classOf[SmallIntVector] + case "IntVector" => classOf[IntVector] + case "BigIntVector" => classOf[BigIntVector] + case "Float4Vector" => classOf[Float4Vector] + case "Float8Vector" => classOf[Float8Vector] + case "DecimalVector" => classOf[DecimalVector] + case "DateDayVector" => classOf[DateDayVector] + case "TimeStampMicroVector" => classOf[TimeStampMicroVector] + case "TimeStampMicroTZVector" => classOf[TimeStampMicroTZVector] + case "VarCharVector" => classOf[VarCharVector] + case "VarBinaryVector" => classOf[VarBinaryVector] + case other => throw new IllegalArgumentException(s"unknown Arrow vector class: $other") + } + + /** + * Type surface the kernel covers on both input and output sides. Recursive: complex types are + * supported when their children are. + */ + def isSupportedDataType(dt: DataType): Boolean = dt match { + case BooleanType | ByteType | ShortType | IntegerType | LongType => true + case FloatType | DoubleType => true + case _: DecimalType => true + case _: StringType | _: BinaryType => true + case DateType | TimestampType | TimestampNTZType => true + case ArrayType(inner, _) => isSupportedDataType(inner) + case st: StructType => st.fields.forall(f => isSupportedDataType(f.dataType)) + case mt: MapType => isSupportedDataType(mt.keyType) && isSupportedDataType(mt.valueType) + case _ => false + } + + /** + * Mirrors `WholeStageCodegenExec.numOfNestedFields` so [[canHandle]] can reuse + * `spark.sql.codegen.maxFields`. + */ + private def numOfNestedFields(dataType: DataType): Int = dataType match { + case st: StructType => st.fields.map(f => numOfNestedFields(f.dataType)).sum + case m: MapType => numOfNestedFields(m.keyType) + numOfNestedFields(m.valueType) + case a: ArrayType => numOfNestedFields(a.elementType) + case _ => 1 + } + + /** + * Plan-time predicate. `None` greenlights the serde to emit the codegen proto; `Some(reason)` + * forces a Spark fallback (typically `withInfo(...) + None`) so the operator falls back cleanly + * rather than crashing the Janino compile at execute time. + * + * Checks every `BoundReference`'s data type and the root `expr.dataType` against + * [[isSupportedDataType]], rejects aggregates / generators / `CodegenFallback` (other than + * HOFs, which are admitted), and gates total nested-field count on + * `spark.sql.codegen.maxFields`. + */ + def canHandle(boundExpr: Expression): Option[String] = { + if (!isSupportedDataType(boundExpr.dataType)) { + return Some(s"codegen dispatch: unsupported output type ${boundExpr.dataType}") + } + // Mirror WSCG's `spark.sql.codegen.maxFields` gate. Wide schemas blow the generated class's + // typed input field count, the typed-getter switch, and the constant pool. Refuse here so the + // operator falls back to Spark cleanly rather than tripping a Janino compile failure + // mid-execution (Comet has no recovery for that). + val maxFields = SQLConf.get.wholeStageMaxNumFields + val totalFields = numOfNestedFields(boundExpr.dataType) + + boundExpr.collect { case b: BoundReference => numOfNestedFields(b.dataType) }.sum + if (totalFields > maxFields) { + return Some( + s"codegen dispatch: too many nested fields ($totalFields > " + + s"spark.sql.codegen.maxFields=$maxFields)") + } + // HOFs are `CodegenFallback` but admitted: `CodegenFallback.doGenCode` emits one + // `((Expression) references[N]).eval(row)` call site per HOF. The kernel dispatches to the + // HOF's interpreted `eval`, which mutates `NamedLambdaVariable.value` per element and reads + // the input array through the kernel's typed Arrow getters. Per-task `boundExpr` isolation + // in `CometScalaUDFCodegen.kernelCache` prevents concurrent partitions from racing on the + // lambda variable's `AtomicReference`. See `CometCodegenHOFSuite`. + // + // Nondeterministic / stateful expressions are accepted: each cache entry holds one kernel + // instance with a single `init(partitionIndex)` call, so `Rand` / `MonotonicallyIncreasingID` + // state advances correctly across batches. + // + // `ExecSubqueryExpression` (`ScalarSubquery`, `InSubqueryExec`) is accepted: the surrounding + // Comet operator's inherited `SparkPlan.waitForSubqueries` populates the subquery's + // `result` field before evaluation. The closure serializer captures that value into the + // arg-0 bytes, and the dispatcher keys its compile cache on those bytes, so distinct subquery + // results produce distinct cache entries. + // + // `Unevaluable`: rejected by default. `isCodegenInertUnevaluable` exempts version-specific + // leaves that are `Unevaluable` but never invoked by codegen (e.g. Spark 4.0's + // `ResolvedCollation` in `Collate.collation`, where `Collate.genCode` delegates to its child). + boundExpr.find { + case _: org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction => true + case _: org.apache.spark.sql.catalyst.expressions.Generator => true + case _: HigherOrderFunction => false + case _: LambdaFunction => false + case _: NamedLambdaVariable => false + case _: CodegenFallback => true + case u: Unevaluable if isCodegenInertUnevaluable(u) => false + case _: Unevaluable => true + case _ => false + } match { + case Some(bad) => + return Some( + s"codegen dispatch: expression ${bad.getClass.getSimpleName} not supported " + + "(aggregate, generator, codegen-fallback, or unevaluable)") + case None => + } + val badRef = boundExpr.collectFirst { + case b: BoundReference if !isSupportedDataType(b.dataType) => + b + } + badRef.map(b => + s"codegen dispatch: unsupported input type ${b.dataType} at ordinal ${b.ordinal}") + } + + /** + * Allocate an Arrow output vector from a pre-built `Field`. Forwards to + * [[CometBatchKernelCodegenOutput.allocateOutput]]. + */ + def allocateOutput(field: Field, numRows: Int, estimatedBytes: Int): FieldVector = + CometBatchKernelCodegenOutput.allocateOutput(field, numRows, estimatedBytes) + + /** + * Spark `DataType` to an Arrow `Field`, resolving mismatches between Arrow Java's default field + * labels and what Spark / Arrow Rust expect on the FFI boundary. + */ + def toFfiArrowField(name: String, dataType: DataType, nullable: Boolean): Field = + CometBatchKernelCodegenOutput.toFfiArrowField(name, dataType, nullable) + + def compile(boundExpr: Expression, inputSchema: Seq[ArrowColumnSpec]): CompiledKernel = { + val src = generateSource(boundExpr, inputSchema) + val (clazz, _) = + try { + CodeGenerator.compile(src.code) + } catch { + case t: Throwable => + logError( + s"CometBatchKernelCodegen: compile failed for ${boundExpr.getClass.getSimpleName}. " + + s"Generated source follows:\n${CodeFormatter.format(src.code)}", + t) + throw t + } + logInfo( + s"CometBatchKernelCodegen: compiled ${boundExpr.getClass.getSimpleName} " + + s"-> ${boundExpr.dataType} inputs=" + + inputSchema + .map(s => s"${s.vectorClass.getSimpleName}${if (s.nullable) "?" else ""}") + .mkString(",")) + // ScalaUDF embeds stateful `ExpressionEncoder` serializers via `ctx.addReferenceObj` that + // reuse internal `UnsafeRow` / `byte[]` buffers per `apply`. Each kernel instance needs its + // own copy. The closure regenerates the references array per call so the dispatcher can hand + // a fresh array to every kernel it allocates from this `CompiledKernel`. + val freshReferences: () => Array[Any] = () => + generateSource(boundExpr, inputSchema).references + CompiledKernel(clazz, freshReferences) + } + + /** + * Generate the Java source without compiling it. Tests assert on emitted source (null short- + * circuit present, non-nullable `isNullAt` returns literal `false`, etc.) without paying for + * Janino. + */ + def generateSource( + boundExpr: Expression, + inputSchema: Seq[ArrowColumnSpec]): GeneratedSource = { + canHandle(boundExpr).foreach(reason => + throw new IllegalArgumentException(s"CometBatchKernelCodegen.generateSource: $reason")) + val ctx = new CodegenContext + // `BoundReference.genCode` emits `${ctx.INPUT_ROW}.getUTF8String(ord)`. Aliasing `row` to + // `this` at the top of `process` routes those reads to the kernel's typed getters (final + // class, JIT devirtualizes + folds the switch). `row` rather than `this` because Spark's + // `splitExpressions` uses `INPUT_ROW` as the parameter name of helper methods it emits; + // `this` is a reserved keyword and Janino rejects it as a parameter name. + ctx.INPUT_ROW = "row" + + val baseClass = classOf[CometBatchKernel].getName + // Resolve Arrow class names at runtime so the generated source matches the method signature + // the running classloader sees. The packaged Comet jar relocates `org.apache.arrow` to + // `org.apache.comet.shaded.arrow` (see `spark/pom.xml`); `.getName` picks the right name + // regardless of whether we run against the shaded jar or the unshaded build output. + val valueVectorClass = classOf[ValueVector].getName + val fieldVectorClass = classOf[FieldVector].getName + + // `outputSetup` holds once-per-batch declarations (typed child-vector casts for complex + // outputs) that `emitOutputWriter` factors out of the per-row body. Scalar outputs return an + // empty string here. + // + // TODO(method-size): perRowBody is inlined inside process's for-loop and not split. + // Sufficiently deep trees can exceed Janino's 64KB method size. Wrap in + // ctx.splitExpressionsWithCurrentInputs when hit. + val (concreteOutClass, outputSetup, perRowBody) = { + // Class-field CSE. `generateExpressions` runs `subexpressionElimination` under the hood, + // populating `ctx.subexprFunctions` with per-row helper calls that write common subtree + // results into `addMutableState` fields. The returned `ExprCode` references those fields. + // `subexprFunctionsCode` is the concatenated helper invocation block, spliced into the + // per-row body by `defaultBody`. + val ev = if (SQLConf.get.subexpressionEliminationEnabled) { + ctx.generateExpressions(Seq(boundExpr), doSubexpressionElimination = true).head + } else { + boundExpr.genCode(ctx) + } + val subExprsCode = ctx.subexprFunctionsCode + val (cls, setup, snippet) = + CometBatchKernelCodegenOutput.emitOutputWriter(boundExpr.dataType, ev.value, ctx) + (cls, setup, defaultBody(boundExpr, ev, snippet, subExprsCode)) + } + + val typedFieldDecls = CometBatchKernelCodegenInput.emitInputFieldDecls(inputSchema) + val typedInputCasts = CometBatchKernelCodegenInput.emitInputCasts(inputSchema) + val decimalTypeByOrdinal = CometBatchKernelCodegenInput.decimalPrecisionByOrdinal(boundExpr) + val getters = + CometBatchKernelCodegenInput.emitTypedGetters(inputSchema, decimalTypeByOrdinal) + val nested = CometBatchKernelCodegenInput.emitNestedClasses(inputSchema) + val getArrayMethod = CometBatchKernelCodegenInput.emitGetArrayMethod(inputSchema) + val getStructMethod = CometBatchKernelCodegenInput.emitGetStructMethod(inputSchema) + val getMapMethod = CometBatchKernelCodegenInput.emitGetMapMethod(inputSchema) + + val codeBody = + s""" + |public java.lang.Object generate(Object[] references) { + | return new SpecificCometBatchKernel(references); + |} + | + |final class SpecificCometBatchKernel extends $baseClass { + | + | ${ctx.declareMutableStates()} + | + | $typedFieldDecls + | private int rowIdx; + | + | public SpecificCometBatchKernel(Object[] references) { + | super(references); + | ${ctx.initMutableStates()} + | } + | + | @Override + | public void init(int partitionIndex) { + | ${ctx.initPartition()} + | } + | + | $getters + | $getArrayMethod + | $getStructMethod + | $getMapMethod + | + | @Override + | public void process( + | $valueVectorClass[] inputs, + | $fieldVectorClass outRaw, + | int numRows) { + | $concreteOutClass output = ($concreteOutClass) outRaw; + | $typedInputCasts + | $outputSetup + | // Alias the kernel as `row` so Spark-generated `${ctx.INPUT_ROW}.method()` reads + | // resolve to the kernel's typed getters. Helper methods that Spark splits via + | // `splitExpressions` also take `InternalRow row` as a parameter; `this` flows + | // implicitly via INPUT_ROW. + | org.apache.spark.sql.catalyst.InternalRow row = this; + | for (int i = 0; i < numRows; i++) { + | this.rowIdx = i; + | $perRowBody + | } + | } + | + | ${ctx.declareAddedFunctions()} + | + |$nested + |} + """.stripMargin + + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) + GeneratedSource(code.body, code, ctx.references.toArray) + } + + /** + * Per-row body. For `NullIntolerant` expressions where the entire tree propagates nulls, + * prepends a short-circuit on the union of input ordinals so the whole `ev.code` cost is + * skipped on null rows. Otherwise the standard shape: run `ev.code`, then `setNull` or write + * based on `ev.isNull`. + * + * `subExprsCode` is the CSE helper-invocation block. It must run before `ev.code`. Inside the + * short-circuit it lives in the else branch so null rows skip CSE too. + */ + private def defaultBody( + boundExpr: Expression, + ev: ExprCode, + writeSnippet: String, + subExprsCode: String): String = { + boundExpr match { + case _ if isNullIntolerant(boundExpr) && allNullIntolerant(boundExpr) => + // Every node from root to leaf is `NullIntolerant` or a leaf, so "any BoundReference null + // -> whole expression null". A non-null-propagating node like `coalesce` or `if` would + // make this incorrect (`coalesce(null, x)` is `x`); `allNullIntolerant` rejects those. + val inputOrdinals = + boundExpr.collect { case b: BoundReference => b.ordinal }.distinct + val nullCheck = + if (inputOrdinals.isEmpty) "false" + else inputOrdinals.map(ord => s"this.col$ord.isNull(i)").mkString(" || ") + s""" + |if ($nullCheck) { + | output.setNull(i); + |} else { + | $subExprsCode + | ${ev.code} + | $writeSnippet + |} + """.stripMargin + case _ => + // NonNullableOutputShortCircuit: when `nullable = false`, drop the `if (ev.isNull)` + // guard at source level rather than relying on JIT folding. + if (!boundExpr.nullable) { + s""" + |$subExprsCode + |${ev.code} + |$writeSnippet + """.stripMargin + } else { + s""" + |$subExprsCode + |${ev.code} + |if (${ev.isNull}) { + | output.setNull(i); + |} else { + | $writeSnippet + |} + """.stripMargin + } + } + } + + /** + * True iff every node in the tree propagates nulls (`NullIntolerant`, `BoundReference`, or + * `Literal`). Gates the [[defaultBody]] short-circuit, which is only correct when no node + * (`Coalesce`, `If`, `CaseWhen`, `Concat`, ...) breaks the propagation chain. + */ + private def allNullIntolerant(expr: Expression): Boolean = + !expr.exists { + case _: BoundReference | _: Literal => false + case other => !isNullIntolerant(other) + } + + /** + * Per-column compile-time invariants. The concrete Arrow vector class and the nullability flag + * are baked into the generated kernel and form part of the cache key: different vector classes + * or nullability produce different kernels. The dispatcher hardcodes top-level `nullable=true` + * (per-batch null density is not part of the cache key); tests reach the non-nullable codegen + * path by constructing specs directly. + */ + sealed trait ArrowColumnSpec { + def vectorClass: Class[_ <: ValueVector] + + def nullable: Boolean + } + + /** Scalar column: one Arrow vector class per row slot, no nested structure. */ + final case class ScalarColumnSpec(vectorClass: Class[_ <: ValueVector], nullable: Boolean) + extends ArrowColumnSpec + + /** + * Array column: an Arrow `ListVector` wrapping a child spec. `elementSparkType` lets the + * nested-class emitter pick the right read template, and the child carries the Arrow vector + * class. Nested arrays compose recursively. + */ + final case class ArrayColumnSpec( + nullable: Boolean, + elementSparkType: DataType, + element: ArrowColumnSpec) + extends ArrowColumnSpec { + override def vectorClass: Class[_ <: ValueVector] = classOf[ListVector] + } + + /** + * Struct column: an Arrow `StructVector` over N typed children. Each [[StructFieldSpec]] + * carries the Spark name (cache-key identity), the Spark `DataType`, the child + * `ArrowColumnSpec`, and the per-field `nullable` bit (lets non-nullable fields elide their + * per-row null check). + */ + final case class StructColumnSpec(nullable: Boolean, fields: Seq[StructFieldSpec]) + extends ArrowColumnSpec { + override def vectorClass: Class[_ <: ValueVector] = classOf[StructVector] + } + + /** One field entry on a [[StructColumnSpec]]. */ + final case class StructFieldSpec( + name: String, + sparkType: DataType, + nullable: Boolean, + child: ArrowColumnSpec) + + /** + * Map column: an Arrow `MapVector` (subclass of `ListVector`) whose data vector is a + * `StructVector` with key at child 0 and value at child 1. Nested keys and values compose + * recursively. The child specs' `nullable` field is unused on the read path. Output-side null + * guards for map values come from `MapType.valueContainsNull` on the Spark `DataType`. + */ + final case class MapColumnSpec( + nullable: Boolean, + keySparkType: DataType, + valueSparkType: DataType, + key: ArrowColumnSpec, + value: ArrowColumnSpec) + extends ArrowColumnSpec { + override def vectorClass: Class[_ <: ValueVector] = classOf[MapVector] + } + + /** + * Compiled kernel handle. `freshReferences` regenerates the references array per kernel + * allocation because `ScalaUDF` embeds stateful `ExpressionEncoder` serializers that cannot be + * shared. + */ + final case class CompiledKernel(factory: GeneratedClass, freshReferences: () => Array[Any]) { + def newInstance(): CometBatchKernel = + factory.generate(freshReferences()).asInstanceOf[CometBatchKernel] + } + + /** + * Output of [[generateSource]]. Tests inspect `body` to assert the shape of the generated + * source. See `CometCodegenSourceSuite`. + */ + final case class GeneratedSource(body: String, code: CodeAndComment, references: Array[Any]) + + object ArrowColumnSpec { + + /** Convenience constructor for the scalar case. */ + def apply(vectorClass: Class[_ <: ValueVector], nullable: Boolean): ArrowColumnSpec = + ScalarColumnSpec(vectorClass, nullable) + + /** Trait-level extractor that destructures only the scalar case. */ + def unapply(spec: ArrowColumnSpec): Option[(Class[_ <: ValueVector], Boolean)] = spec match { + case ScalarColumnSpec(c, n) => Some((c, n)) + case _ => None + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegenInput.scala b/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegenInput.scala new file mode 100644 index 0000000000..9a4f4bcc57 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegenInput.scala @@ -0,0 +1,963 @@ +/* + * 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.codegen + +import scala.collection.mutable + +import org.apache.arrow.vector._ +import org.apache.arrow.vector.complex.{ListVector, MapVector, StructVector} +import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression} +import org.apache.spark.sql.types._ + +import org.apache.comet.codegen.CometBatchKernelCodegen.{ArrayColumnSpec, ArrowColumnSpec, MapColumnSpec, ScalarColumnSpec, StructColumnSpec} +import org.apache.comet.vector.CometPlainVector + +/** + * Input-side emitters for the codegen kernel: typed field declarations, per-batch input casts, + * top-level typed-getter switches, nested `InputArray_${path}` / `InputStruct_${path}` / + * `InputMap_${path}` classes per complex level. Paired with [[CometBatchKernelCodegenOutput]]. + * + * Path encoding. Each position in the spec tree has a unique path string used as a suffix on + * vector fields and nested classes. From a column ordinal: root `col${ord}`, array element + * `${P}_e`, struct field `fi` `${P}_f${fi}`, map key `${P}_k`, map value `${P}_v`. + * + * Nested-class composition. Each instance is allocated fresh per `getArray(i)` / `getStruct(i, + * n)` / `getMap(i)` call, with `final` slice fields. Matches Spark's `ColumnarRow` / + * `ColumnarArray` model: retain-by-reference consumers (e.g. `ArrayDistinct.nullSafeEval` + * stashing references in an `OpenHashSet`) get distinct identities, and JIT escape analysis + * usually scalarizes the allocation when the value is consumed locally. + */ +private[codegen] object CometBatchKernelCodegenInput { + + /** + * Primitive Arrow vector classes wrapped in [[CometPlainVector]] at input-cast time so per-row + * reads go through `Platform.get*` against a cached buffer address (JIT inlines to branchless + * reads). Decimal/VarChar/VarBinary stay on the typed Arrow field with cached buffer addresses + * for inline unsafe reads. + */ + private val primitiveArrowClasses: Set[Class[_]] = Set( + classOf[BitVector], + classOf[TinyIntVector], + classOf[SmallIntVector], + classOf[IntVector], + classOf[BigIntVector], + classOf[Float4Vector], + classOf[Float8Vector], + classOf[DateDayVector], + classOf[TimeStampMicroVector], + classOf[TimeStampMicroTZVector]) + private val cometPlainVectorName: String = classOf[CometPlainVector].getName + + /** Emit kernel typed-vector field declarations for every level of every input column. */ + def emitInputFieldDecls(inputSchema: Seq[ArrowColumnSpec]): String = { + val lines = new mutable.ArrayBuffer[String]() + inputSchema.zipWithIndex.foreach { case (spec, ord) => + val path = s"col$ord" + collectVectorFieldDecls(path, spec, lines) + } + lines.mkString("\n ") + } + + /** + * Emit per-batch cast statements, recursing through complex types via `getDataVector` / etc. + */ + def emitInputCasts(inputSchema: Seq[ArrowColumnSpec]): String = { + val lines = new mutable.ArrayBuffer[String]() + inputSchema.zipWithIndex.foreach { case (spec, ord) => + val path = s"col$ord" + collectCasts(path, spec, s"inputs[$ord]", lines) + } + lines.mkString("\n ") + } + + /** + * Emit typed-getter overrides. Each switches on column ordinal. With the inlined constant + * ordinal from `BoundReference.genCode`, JIT folds the switch to one branch. + * + * `decimalTypeByOrdinal` lets the decimal getter specialize per ordinal: when only a + * `DecimalType(precision <= 18)` `BoundReference` reads the ordinal, the case skips the + * `BigDecimal` allocation and reads the unscaled long directly. + */ + def emitTypedGetters( + inputSchema: Seq[ArrowColumnSpec], + decimalTypeByOrdinal: Map[Int, Option[DecimalType]]): String = { + val withOrd = inputSchema.zipWithIndex + + val isNullCases = withOrd.map { case (spec, ord) => + if (!spec.nullable) { + s" case $ord: return false;" + } else { + // CometPlainVector exposes `isNullAt`; Arrow-typed fields expose `isNull`. Same semantics. + val method = spec.vectorClass match { + case cls if wrapsInCometPlainVector(cls) => "isNullAt" + case _ => "isNull" + } + s" case $ord: return this.col$ord.$method(this.rowIdx);" + } + } + + val booleanCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[BitVector] => + s" case $ord: return this.col$ord.getBoolean(this.rowIdx);" + } + val byteCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[TinyIntVector] => + s" case $ord: return this.col$ord.getByte(this.rowIdx);" + } + val shortCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[SmallIntVector] => + s" case $ord: return this.col$ord.getShort(this.rowIdx);" + } + val intCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) + if cls == classOf[IntVector] || cls == classOf[DateDayVector] => + s" case $ord: return this.col$ord.getInt(this.rowIdx);" + } + val longCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) + if cls == classOf[BigIntVector] || + cls == classOf[TimeStampMicroVector] || + cls == classOf[TimeStampMicroTZVector] => + s" case $ord: return this.col$ord.getLong(this.rowIdx);" + } + val floatCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[Float4Vector] => + s" case $ord: return this.col$ord.getFloat(this.rowIdx);" + } + val doubleCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[Float8Vector] => + s" case $ord: return this.col$ord.getDouble(this.rowIdx);" + } + val decimalCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[DecimalVector] => + val known = decimalTypeByOrdinal.getOrElse(ord, None) + val valueAddr = s"this.col${ord}_valueAddr" + val slowField = s"this.col$ord" + val fastPath = emitDecimalFastBodyUnsafe(valueAddr, "this.rowIdx", " ") + val slowPath = emitDecimalSlowBody(slowField, "this.rowIdx", " ") + val body = known match { + case Some(dt) if dt.precision <= Decimal.MAX_LONG_DIGITS => fastPath + case Some(_) => slowPath + case None => + s""" if (precision <= ${Decimal.MAX_LONG_DIGITS}) { + |$fastPath + | } else { + |$slowPath + | }""".stripMargin + } + s""" case $ord: { + |$body + | }""".stripMargin + } + val binaryCases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[VarBinaryVector] => + s""" case $ord: { + |${emitBinaryBodyUnsafe( + s"this.col${ord}_valueAddr", + s"this.col${ord}_offsetAddr", + "this.rowIdx", + " ")} + | }""".stripMargin + } + val utf8Cases = withOrd.collect { + case (ArrowColumnSpec(cls, _), ord) if cls == classOf[VarCharVector] => + s""" case $ord: { + |${emitUtf8BodyUnsafe( + s"this.col${ord}_valueAddr", + s"this.col${ord}_offsetAddr", + "this.rowIdx", + " ")} + | }""".stripMargin + } + + Seq( + emitOrdinalSwitch("public boolean isNullAt(int ordinal)", "isNullAt", isNullCases), + emitOrdinalSwitch("public boolean getBoolean(int ordinal)", "getBoolean", booleanCases), + emitOrdinalSwitch("public byte getByte(int ordinal)", "getByte", byteCases), + emitOrdinalSwitch("public short getShort(int ordinal)", "getShort", shortCases), + emitOrdinalSwitch("public int getInt(int ordinal)", "getInt", intCases), + emitOrdinalSwitch("public long getLong(int ordinal)", "getLong", longCases), + emitOrdinalSwitch("public float getFloat(int ordinal)", "getFloat", floatCases), + emitOrdinalSwitch("public double getDouble(int ordinal)", "getDouble", doubleCases), + emitOrdinalSwitch( + "public org.apache.spark.sql.types.Decimal getDecimal(" + + "int ordinal, int precision, int scale)", + "getDecimal", + decimalCases), + emitOrdinalSwitch("public byte[] getBinary(int ordinal)", "getBinary", binaryCases), + emitOrdinalSwitch( + "public org.apache.spark.unsafe.types.UTF8String getUTF8String(int ordinal)", + "getUTF8String", + utf8Cases)).mkString + } + + private def wrapsInCometPlainVector(cls: Class[_]): Boolean = + primitiveArrowClasses.contains(cls) + + private def emitOrdinalSwitch(methodSig: String, label: String, cases: Seq[String]): String = { + if (cases.isEmpty) { + "" + } else { + s""" + | @Override + | $methodSig { + | switch (ordinal) { + |${cases.mkString("\n")} + | default: throw new UnsupportedOperationException( + | "$label out of range: " + ordinal); + | } + | } + """.stripMargin + } + } + + private def emitDecimalSlowBody(field: String, idx: String, ind: String): String = { + val cont = ind + " " + s"""${ind}java.math.BigDecimal bd = $field.getObject($idx); + |${ind}return org.apache.spark.sql.types.Decimal$$.MODULE$$ + |$cont.apply(bd, precision, scale);""".stripMargin + } + + private def emitDecimalFastBodyUnsafe(valueAddr: String, idx: String, ind: String): String = { + val cont = ind + " " + val i = castableIdx(idx) + s"""${ind}long unscaled = org.apache.spark.unsafe.Platform.getLong(null, + |$cont$valueAddr + (long) $i * 16L); + |${ind}return org.apache.spark.sql.types.Decimal$$.MODULE$$ + |$cont.createUnsafe(unscaled, precision, scale);""".stripMargin + } + + private def emitUtf8BodyUnsafe( + valueAddr: String, + offsetAddr: String, + idx: String, + ind: String): String = { + val cont = ind + " " + val i = castableIdx(idx) + s"""${ind}int s = org.apache.spark.unsafe.Platform.getInt(null, + |$cont$offsetAddr + (long) $i * 4L); + |${ind}int e = org.apache.spark.unsafe.Platform.getInt(null, + |$cont$offsetAddr + ((long) $i + 1L) * 4L); + |${ind}return org.apache.spark.unsafe.types.UTF8String + |$cont.fromAddress(null, $valueAddr + s, e - s);""".stripMargin + } + + /** Parenthesize `idx` when it contains whitespace, to keep `(long) idx * 16L` well-formed. */ + private def castableIdx(idx: String): String = if (idx.contains(' ')) s"($idx)" else idx + + private def emitBinaryBodyUnsafe( + valueAddr: String, + offsetAddr: String, + idx: String, + ind: String): String = { + val cont = ind + " " + val i = castableIdx(idx) + s"""${ind}int s = org.apache.spark.unsafe.Platform.getInt(null, + |$cont$offsetAddr + (long) $i * 4L); + |${ind}int e = org.apache.spark.unsafe.Platform.getInt(null, + |$cont$offsetAddr + ((long) $i + 1L) * 4L); + |${ind}int len = e - s; + |${ind}byte[] out = new byte[len]; + |${ind}org.apache.spark.unsafe.Platform.copyMemory(null, $valueAddr + s, out, + |${cont}org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET, len); + |${ind}return out;""".stripMargin + } + + /** + * Per-ordinal map of the `DecimalType` observed on `BoundReference`s. Used by + * [[emitTypedGetters]] to emit a precision-specialized `getDecimal` case per ordinal. + */ + def decimalPrecisionByOrdinal(boundExpr: Expression): Map[Int, Option[DecimalType]] = { + boundExpr + .collect { + case b: BoundReference if b.dataType.isInstanceOf[DecimalType] => + b.ordinal -> b.dataType.asInstanceOf[DecimalType] + } + .groupBy(_._1) + .map { case (ord, pairs) => + val distinct = pairs.map(_._2).toSet + ord -> (if (distinct.size == 1) Some(distinct.head) else None) + } + } + + /** + * Emit nested classes for every complex level of every input column: `InputArray_${path}` for + * arrays, `InputStruct_${path}` for structs, `InputMap_${path}` plus `InputArray` views for the + * key/value slices for maps (Spark's `MapData.keyArray()` / `valueArray()` return `ArrayData`). + */ + def emitNestedClasses(inputSchema: Seq[ArrowColumnSpec]): String = { + val out = new mutable.ArrayBuffer[String]() + inputSchema.zipWithIndex.foreach { case (spec, ord) => + collectNestedClasses(s"col$ord", spec, out) + } + out.mkString("\n") + } + + /** + * Top-level `getArray(int ordinal)` switch. Each case reads `(start, length)` from the outer + * `ListVector` offsets and allocates a fresh `InputArray_col${ord}` view. + */ + def emitGetArrayMethod(inputSchema: Seq[ArrowColumnSpec]): String = { + val cases = inputSchema.zipWithIndex.collect { case (_: ArrayColumnSpec, ord) => + s""" case $ord: { + | int __idx = this.rowIdx; + | int __s = this.col$ord.getElementStartIndex(__idx); + | int __e = this.col$ord.getElementEndIndex(__idx); + | return new InputArray_col$ord(__s, __e - __s); + | }""".stripMargin + } + if (cases.isEmpty) { + "" + } else { + s""" + | @Override + | public org.apache.spark.sql.catalyst.util.ArrayData getArray(int ordinal) { + | switch (ordinal) { + |${cases.mkString("\n")} + | default: throw new UnsupportedOperationException( + | "getArray out of range: " + ordinal); + | } + | } + |""".stripMargin + } + } + + /** Top-level `getMap(int ordinal)` switch when the schema has at least one map column. */ + def emitGetMapMethod(inputSchema: Seq[ArrowColumnSpec]): String = { + val cases = inputSchema.zipWithIndex.collect { case (_: MapColumnSpec, ord) => + s""" case $ord: { + | int __idx = this.rowIdx; + | int __s = this.col$ord.getElementStartIndex(__idx); + | int __e = this.col$ord.getElementEndIndex(__idx); + | return new InputMap_col$ord(__s, __e - __s); + | }""".stripMargin + } + if (cases.isEmpty) { + "" + } else { + s""" + | @Override + | public org.apache.spark.sql.catalyst.util.MapData getMap(int ordinal) { + | switch (ordinal) { + |${cases.mkString("\n")} + | default: throw new UnsupportedOperationException( + | "getMap out of range: " + ordinal); + | } + | } + |""".stripMargin + } + } + + /** Top-level `getStruct(int ordinal, int numFields)` switch when the schema has any struct. */ + def emitGetStructMethod(inputSchema: Seq[ArrowColumnSpec]): String = { + val cases = inputSchema.zipWithIndex.collect { case (_: StructColumnSpec, ord) => + s""" case $ord: return new InputStruct_col$ord(this.rowIdx);""".stripMargin + } + if (cases.isEmpty) { + "" + } else { + s""" + | @Override + | public org.apache.spark.sql.catalyst.InternalRow getStruct(int ordinal, int numFields) { + | switch (ordinal) { + |${cases.mkString("\n")} + | default: throw new UnsupportedOperationException( + | "getStruct out of range: " + ordinal); + | } + | } + |""".stripMargin + } + } + + /** + * Scalar columns that need a cached data-buffer address for inline unsafe reads. + * `DecimalVector` uses it for the short-precision fast path (`Platform.getLong`); + * `VarCharVector` / `VarBinaryVector` use it as the base for `UTF8String.fromAddress` / + * `Platform.copyMemory`. + */ + private def needsValueAddrField(cls: Class[_]): Boolean = + cls == classOf[DecimalVector] || + cls == classOf[VarCharVector] || + cls == classOf[VarBinaryVector] + + /** Variable-width columns also cache the offset-buffer address for `Platform.getInt`. */ + private def needsOffsetAddrField(cls: Class[_]): Boolean = + cls == classOf[VarCharVector] || cls == classOf[VarBinaryVector] + + /** + * Java method name for the per-column null check. Primitive scalars wrapped in + * [[CometPlainVector]] expose `isNullAt`; Arrow typed fields expose `isNull`. Same semantics. + */ + private def nullCheckMethod(spec: ArrowColumnSpec): String = spec match { + case sc: ScalarColumnSpec if wrapsInCometPlainVector(sc.vectorClass) => "isNullAt" + case _ => "isNull" + } + + private def collectVectorFieldDecls( + path: String, + spec: ArrowColumnSpec, + out: mutable.ArrayBuffer[String]): Unit = spec match { + case sc: ScalarColumnSpec => + // Primitive scalars wrap in CometPlainVector for JIT-inlined Platform.get* against a + // cached buffer address. Decimal/VarChar/VarBinary stay on the Arrow typed field with + // cached data- (and offset-) buffer addresses for inline unsafe reads. + val fieldClass = + if (wrapsInCometPlainVector(sc.vectorClass)) cometPlainVectorName + else sc.vectorClass.getName + out += s"private $fieldClass $path;" + if (needsValueAddrField(sc.vectorClass)) { + out += s"private long ${path}_valueAddr;" + } + if (needsOffsetAddrField(sc.vectorClass)) { + out += s"private long ${path}_offsetAddr;" + } + case ar: ArrayColumnSpec => + out += s"private ${classOf[ListVector].getName} $path;" + collectVectorFieldDecls(s"${path}_e", ar.element, out) + case st: StructColumnSpec => + out += s"private ${classOf[StructVector].getName} $path;" + st.fields.zipWithIndex.foreach { case (f, fi) => + collectVectorFieldDecls(s"${path}_f$fi", f.child, out) + } + case mp: MapColumnSpec => + out += s"private ${classOf[MapVector].getName} $path;" + // Key/value vectors live at `${P}_k_e` / `${P}_v_e` so the synthetic `InputArray_${P}_k` / + // `InputArray_${P}_v` classes (which follow the array-element convention of reading from + // `${path}_e`) resolve correctly. + collectVectorFieldDecls(s"${path}_k_e", mp.key, out) + collectVectorFieldDecls(s"${path}_v_e", mp.value, out) + } + + private def collectCasts( + path: String, + spec: ArrowColumnSpec, + source: String, + out: mutable.ArrayBuffer[String]): Unit = spec match { + case sc: ScalarColumnSpec => + if (wrapsInCometPlainVector(sc.vectorClass)) { + // `useDecimal128 = true` matches Spark's 128-bit decimal storage. + out += s"this.$path = new $cometPlainVectorName($source, true);" + } else { + out += s"this.$path = (${sc.vectorClass.getName}) $source;" + } + if (needsValueAddrField(sc.vectorClass)) { + out += s"this.${path}_valueAddr = this.$path.getDataBuffer().memoryAddress();" + } + if (needsOffsetAddrField(sc.vectorClass)) { + out += s"this.${path}_offsetAddr = this.$path.getOffsetBuffer().memoryAddress();" + } + case ar: ArrayColumnSpec => + out += s"this.$path = (${classOf[ListVector].getName}) $source;" + collectCasts(s"${path}_e", ar.element, s"this.$path.getDataVector()", out) + case st: StructColumnSpec => + out += s"this.$path = (${classOf[StructVector].getName}) $source;" + st.fields.zipWithIndex.foreach { case (f, fi) => + collectCasts(s"${path}_f$fi", f.child, s"this.$path.getChildByOrdinal($fi)", out) + } + case mp: MapColumnSpec => + // MapVector's data vector is a StructVector with key at child 0 and value at child 1. + val structLocal = s"${path}__mapStruct" + out += s"this.$path = (${classOf[MapVector].getName}) $source;" + out += s"${classOf[StructVector].getName} $structLocal = " + + s"(${classOf[StructVector].getName}) this.$path.getDataVector();" + collectCasts(s"${path}_k_e", mp.key, s"$structLocal.getChildByOrdinal(0)", out) + collectCasts(s"${path}_v_e", mp.value, s"$structLocal.getChildByOrdinal(1)", out) + } + + private def collectNestedClasses( + path: String, + spec: ArrowColumnSpec, + out: mutable.ArrayBuffer[String]): Unit = spec match { + case _: ScalarColumnSpec => () + case ar: ArrayColumnSpec => + out += emitArrayClass(path, ar) + collectNestedClasses(s"${path}_e", ar.element, out) + case st: StructColumnSpec => + out += emitStructClass(path, st) + st.fields.zipWithIndex.foreach { case (f, fi) => + collectNestedClasses(s"${path}_f$fi", f.child, out) + } + case mp: MapColumnSpec => + out += emitMapClass(path) + // Emit `InputArray_${path}_k` / `InputArray_${path}_v` (the views returned by + // `keyArray()` / `valueArray()`). Each reads from `${classPath}_e` per the array-element + // convention, mapping to the key/value vector at `${path}_k_e` / `${path}_v_e`. + out += emitArrayClass( + s"${path}_k", + ArrayColumnSpec(nullable = true, elementSparkType = mp.keySparkType, element = mp.key)) + out += emitArrayClass( + s"${path}_v", + ArrayColumnSpec( + nullable = true, + elementSparkType = mp.valueSparkType, + element = mp.value)) + collectNestedClasses(s"${path}_k_e", mp.key, out) + collectNestedClasses(s"${path}_v_e", mp.value, out) + } + + /** + * Emit one `InputArray_${path}` nested class. Constructor takes `(startIdx, length)` and stores + * both in `final` fields. Map key/value arrays share this shape. + */ + private def emitArrayClass(path: String, spec: ArrayColumnSpec): String = { + val baseClassName = classOf[CometArrayData].getName + val elemPath = s"${path}_e" + val isNullAt = + s""" @Override + | public boolean isNullAt(int i) { + | return $elemPath.${nullCheckMethod(spec.element)}(startIndex + i); + | }""".stripMargin + val elementGetter = emitArrayElementGetter(path, spec) + s""" private final class InputArray_$path extends $baseClassName { + | private final int startIndex; + | private final int length; + | + | InputArray_$path(int startIdx, int len) { + | this.startIndex = startIdx; + | this.length = len; + | } + | + | @Override + | public int numElements() { + | return length; + | } + | + |$isNullAt + | + |$elementGetter + | } + |""".stripMargin + } + + /** + * Element-getter body for a nested array. Scalar -> direct typed read. Complex -> allocate a + * fresh inner view. + * + * Reference-typed getters (`getDecimal` / `getUTF8String` / `getBinary` / `getStruct` / + * `getArray` / `getMap`) prepend `if (isNullAt(i)) return null;` when the element is nullable, + * because Spark's `CodeGenerator.setArrayElement` only emits the caller-side `isNullAt` check + * for primitive elements (it relies on the source's getter to return null for reference types, + * matching `ColumnarArray.getBinary`). Without this guard, expressions like `Flatten.doGenCode` + * write empty bytes / garbage decimals where Spark expects null. + */ + private def emitArrayElementGetter(path: String, spec: ArrayColumnSpec): String = { + val elemPath = s"${path}_e" + val nullGuard = + if (spec.element.nullable) " if (isNullAt(i)) return null;\n" + else "" + spec.element match { + case _: ScalarColumnSpec => + emitArrayElementScalarGetter(spec.elementSparkType, elemPath, spec.element.nullable) + case _: ArrayColumnSpec => + s""" @Override + | public org.apache.spark.sql.catalyst.util.ArrayData getArray(int i) { + |$nullGuard int __idx = startIndex + i; + | int __s = $elemPath.getElementStartIndex(__idx); + | int __e = $elemPath.getElementEndIndex(__idx); + | return new InputArray_$elemPath(__s, __e - __s); + | }""".stripMargin + case _: StructColumnSpec => + s""" @Override + | public org.apache.spark.sql.catalyst.InternalRow getStruct(int i, int numFields) { + |$nullGuard return new InputStruct_$elemPath(startIndex + i); + | }""".stripMargin + case _: MapColumnSpec => + s""" @Override + | public org.apache.spark.sql.catalyst.util.MapData getMap(int i) { + |$nullGuard int __idx = startIndex + i; + | int __s = $elemPath.getElementStartIndex(__idx); + | int __e = $elemPath.getElementEndIndex(__idx); + | return new InputMap_$elemPath(__s, __e - __s); + | }""".stripMargin + } + } + + /** + * Scalar-element getter override. Only the getter matching the element type is overridden; + * other getters inherit the base class's `UnsupportedOperationException`. Reference-typed + * getters (Decimal / String / Binary) prepend the null guard documented on + * [[emitArrayElementGetter]]. + */ + private def emitArrayElementScalarGetter( + elemType: DataType, + childField: String, + elementNullable: Boolean): String = { + val nullGuard = + if (elementNullable) " if (isNullAt(i)) return null;\n" + else "" + elemType match { + case BooleanType => + s""" @Override + | public boolean getBoolean(int i) { + | return $childField.getBoolean(startIndex + i); + | }""".stripMargin + case ByteType => + s""" @Override + | public byte getByte(int i) { + | return $childField.getByte(startIndex + i); + | }""".stripMargin + case ShortType => + s""" @Override + | public short getShort(int i) { + | return $childField.getShort(startIndex + i); + | }""".stripMargin + case IntegerType | DateType => + s""" @Override + | public int getInt(int i) { + | return $childField.getInt(startIndex + i); + | }""".stripMargin + case LongType | TimestampType | TimestampNTZType => + s""" @Override + | public long getLong(int i) { + | return $childField.getLong(startIndex + i); + | }""".stripMargin + case FloatType => + s""" @Override + | public float getFloat(int i) { + | return $childField.getFloat(startIndex + i); + | }""".stripMargin + case DoubleType => + s""" @Override + | public double getDouble(int i) { + | return $childField.getDouble(startIndex + i); + | }""".stripMargin + case dt: DecimalType => + val body = + if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + emitDecimalFastBodyUnsafe(s"${childField}_valueAddr", "startIndex + i", " ") + } else { + emitDecimalSlowBody(childField, "startIndex + i", " ") + } + s""" @Override + | public org.apache.spark.sql.types.Decimal getDecimal( + | int i, int precision, int scale) { + |$nullGuard$body + | }""".stripMargin + case _: StringType => + s""" @Override + | public org.apache.spark.unsafe.types.UTF8String getUTF8String(int i) { + |$nullGuard${emitUtf8BodyUnsafe( + s"${childField}_valueAddr", + s"${childField}_offsetAddr", + "startIndex + i", + " ")} + | }""".stripMargin + case BinaryType => + s""" @Override + | public byte[] getBinary(int i) { + |$nullGuard${emitBinaryBodyUnsafe( + s"${childField}_valueAddr", + s"${childField}_offsetAddr", + "startIndex + i", + " ")} + | }""".stripMargin + case other => + throw new UnsupportedOperationException( + s"nested ArrayData: unsupported element type $other") + } + } + + /** + * Emit one `InputStruct_${path}` nested class. Constructor takes `rowIdx` and stores it in a + * `final` field. Scalar getters switch on field ordinal. Complex getters allocate fresh inner + * views (offsets computed for array/map children, rowIdx passed through for struct children). + */ + private def emitStructClass(path: String, spec: StructColumnSpec): String = { + val baseClassName = classOf[CometInternalRow].getName + val isNullCases = spec.fields.zipWithIndex.map { + case (f, fi) if !f.nullable => + s" case $fi: return false;" + case (f, fi) => + s" case $fi: return ${path}_f$fi.${nullCheckMethod(f.child)}(this.rowIdx);" + } + val scalarGetters = emitStructScalarGetters(path, spec) + val complexGetters = emitStructComplexGetters(path, spec) + s""" private final class InputStruct_$path extends $baseClassName { + | private final int rowIdx; + | + | InputStruct_$path(int outerRowIdx) { + | this.rowIdx = outerRowIdx; + | } + | + | @Override + | public int numFields() { + | return ${spec.fields.length}; + | } + | + | @Override + | public boolean isNullAt(int ordinal) { + | switch (ordinal) { + |${isNullCases.mkString("\n")} + | default: throw new UnsupportedOperationException( + | "InputStruct_$path.isNullAt out of range: " + ordinal); + | } + | } + | + |$scalarGetters + |$complexGetters + | } + |""".stripMargin + } + + // Scalar-read body templates parameterized on row-index expression (`idx`), cached buffer + // addresses (`valueAddr`, `offsetAddr`) for unsafe reads, or the Arrow field for the decimal + // slow path. `ind` is the per-line indent. + // + // TODO(#4280, #4279): once offset-address caching and validity-bitmap byte cache land in + // CometPlainVector, replace the VarChar/VarBinary unsafe emitters with CometPlainVector reads. + + private def emitStructScalarGetters(path: String, spec: StructColumnSpec): String = { + val withOrd = spec.fields.zipWithIndex + val scalarOrd = withOrd.filter { case (f, _) => f.child.isInstanceOf[ScalarColumnSpec] } + + // For nullable reference-typed struct fields, prepend the null guard so `getX(ord)` returns + // null on null positions (Spark contract for reference types). Same rationale as the array + // element getter. + def nullGuardForCase(fi: Int, fieldNullable: Boolean): String = + if (fieldNullable) s" if (isNullAt($fi)) return null;\n" + else "" + + def fieldReadScalar(fi: Int, dt: DataType, fieldNullable: Boolean): String = { + val guard = nullGuardForCase(fi, fieldNullable) + dt match { + case BooleanType => + s" case $fi: return ${path}_f$fi.getBoolean(this.rowIdx);" + case ByteType => + s" case $fi: return ${path}_f$fi.getByte(this.rowIdx);" + case ShortType => + s" case $fi: return ${path}_f$fi.getShort(this.rowIdx);" + case IntegerType | DateType => + s" case $fi: return ${path}_f$fi.getInt(this.rowIdx);" + case LongType | TimestampType | TimestampNTZType => + s" case $fi: return ${path}_f$fi.getLong(this.rowIdx);" + case FloatType => + s" case $fi: return ${path}_f$fi.getFloat(this.rowIdx);" + case DoubleType => + s" case $fi: return ${path}_f$fi.getDouble(this.rowIdx);" + case BinaryType => + s""" case $fi: { + |$guard${emitBinaryBodyUnsafe( + s"${path}_f${fi}_valueAddr", + s"${path}_f${fi}_offsetAddr", + "this.rowIdx", + " ")} + | }""".stripMargin + case _: StringType => + s""" case $fi: { + |$guard${emitUtf8BodyUnsafe( + s"${path}_f${fi}_valueAddr", + s"${path}_f${fi}_offsetAddr", + "this.rowIdx", + " ")} + | }""".stripMargin + case _: DecimalType => + throw new IllegalStateException("decimal handled separately") + case other => + throw new UnsupportedOperationException( + s"nested InputStruct getter: unsupported field type $other") + } + } + + val booleanCases = + scalarOrd.collect { + case (f, fi) if f.sparkType == BooleanType => + fieldReadScalar(fi, BooleanType, f.nullable) + } + val byteCases = + scalarOrd.collect { + case (f, fi) if f.sparkType == ByteType => + fieldReadScalar(fi, ByteType, f.nullable) + } + val shortCases = + scalarOrd.collect { + case (f, fi) if f.sparkType == ShortType => + fieldReadScalar(fi, ShortType, f.nullable) + } + val intCases = scalarOrd.collect { + case (f, fi) if f.sparkType == IntegerType || f.sparkType == DateType => + fieldReadScalar(fi, IntegerType, f.nullable) + } + val longCases = scalarOrd.collect { + case (f, fi) + if f.sparkType == LongType || f.sparkType == TimestampType || + f.sparkType == TimestampNTZType => + fieldReadScalar(fi, LongType, f.nullable) + } + val floatCases = + scalarOrd.collect { + case (f, fi) if f.sparkType == FloatType => + fieldReadScalar(fi, FloatType, f.nullable) + } + val doubleCases = + scalarOrd.collect { + case (f, fi) if f.sparkType == DoubleType => + fieldReadScalar(fi, DoubleType, f.nullable) + } + val binaryCases = + scalarOrd.collect { + case (f, fi) if f.sparkType == BinaryType => + fieldReadScalar(fi, BinaryType, f.nullable) + } + val utf8Cases = scalarOrd.collect { + case (f, fi) if f.sparkType.isInstanceOf[StringType] => + fieldReadScalar(fi, f.sparkType, f.nullable) + } + + val decimalCases = scalarOrd.collect { + case (f, fi) if f.sparkType.isInstanceOf[DecimalType] => + val dt = f.sparkType.asInstanceOf[DecimalType] + val field = s"${path}_f$fi" + val body = + if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + emitDecimalFastBodyUnsafe(s"${field}_valueAddr", "this.rowIdx", " ") + } else { + emitDecimalSlowBody(field, "this.rowIdx", " ") + } + val guard = nullGuardForCase(fi, f.nullable) + s""" case $fi: { + |$guard$body + | }""".stripMargin + } + + Seq( + structSwitch("public boolean getBoolean(int ordinal)", "getBoolean", booleanCases), + structSwitch("public byte getByte(int ordinal)", "getByte", byteCases), + structSwitch("public short getShort(int ordinal)", "getShort", shortCases), + structSwitch("public int getInt(int ordinal)", "getInt", intCases), + structSwitch("public long getLong(int ordinal)", "getLong", longCases), + structSwitch("public float getFloat(int ordinal)", "getFloat", floatCases), + structSwitch("public double getDouble(int ordinal)", "getDouble", doubleCases), + structSwitch( + "public org.apache.spark.sql.types.Decimal getDecimal(" + + "int ordinal, int precision, int scale)", + "getDecimal", + decimalCases), + structSwitch("public byte[] getBinary(int ordinal)", "getBinary", binaryCases), + structSwitch( + "public org.apache.spark.unsafe.types.UTF8String getUTF8String(int ordinal)", + "getUTF8String", + utf8Cases)).mkString + } + + private def emitStructComplexGetters(path: String, spec: StructColumnSpec): String = { + // Same null-guard rationale as `emitArrayElementGetter`. + def guardLine(fi: Int, fieldNullable: Boolean): String = + if (fieldNullable) s" if (isNullAt($fi)) return null;\n" + else "" + val getArrayCases = spec.fields.zipWithIndex.collect { + case (f, fi) if f.child.isInstanceOf[ArrayColumnSpec] => + val fieldPath = s"${path}_f$fi" + s""" case $fi: { + |${guardLine(fi, f.nullable)} int __idx = this.rowIdx; + | int __s = $fieldPath.getElementStartIndex(__idx); + | int __e = $fieldPath.getElementEndIndex(__idx); + | return new InputArray_$fieldPath(__s, __e - __s); + | }""".stripMargin + } + val getStructCases = spec.fields.zipWithIndex.collect { + case (f, fi) if f.child.isInstanceOf[StructColumnSpec] => + val fieldPath = s"${path}_f$fi" + if (f.nullable) { + s""" case $fi: { + |${guardLine( + fi, + f.nullable)} return new InputStruct_$fieldPath(this.rowIdx); + | }""".stripMargin + } else { + s" case $fi: return new InputStruct_$fieldPath(this.rowIdx);" + } + } + val getMapCases = spec.fields.zipWithIndex.collect { + case (f, fi) if f.child.isInstanceOf[MapColumnSpec] => + val fieldPath = s"${path}_f$fi" + s""" case $fi: { + |${guardLine(fi, f.nullable)} int __idx = this.rowIdx; + | int __s = $fieldPath.getElementStartIndex(__idx); + | int __e = $fieldPath.getElementEndIndex(__idx); + | return new InputMap_$fieldPath(__s, __e - __s); + | }""".stripMargin + } + Seq( + structSwitch( + "public org.apache.spark.sql.catalyst.util.ArrayData getArray(int ordinal)", + "getArray", + getArrayCases), + structSwitch( + "public org.apache.spark.sql.catalyst.InternalRow getStruct(int ordinal, int numFields)", + "getStruct", + getStructCases), + structSwitch( + "public org.apache.spark.sql.catalyst.util.MapData getMap(int ordinal)", + "getMap", + getMapCases)).mkString + } + + /** + * Emit one `InputMap_${path}` nested class. Constructor takes `(start, length)`; `keyArray()` / + * `valueArray()` allocate fresh `InputArray_${path}_k` / `InputArray_${path}_v` views. + */ + private def emitMapClass(path: String): String = { + val baseClassName = classOf[CometMapData].getName + val keyPath = s"${path}_k" + val valPath = s"${path}_v" + s""" private final class InputMap_$path extends $baseClassName { + | private final int startIndex; + | private final int length; + | + | InputMap_$path(int startIdx, int len) { + | this.startIndex = startIdx; + | this.length = len; + | } + | + | @Override + | public int numElements() { + | return length; + | } + | + | @Override + | public org.apache.spark.sql.catalyst.util.ArrayData keyArray() { + | return new InputArray_$keyPath(this.startIndex, this.length); + | } + | + | @Override + | public org.apache.spark.sql.catalyst.util.ArrayData valueArray() { + | return new InputArray_$valPath(this.startIndex, this.length); + | } + | } + |""".stripMargin + } + + private def structSwitch(methodSig: String, label: String, cases: Seq[String]): String = { + if (cases.isEmpty) { + "" + } else { + s""" + | @Override + | $methodSig { + | switch (ordinal) { + |${cases.mkString("\n")} + | default: throw new UnsupportedOperationException( + | "$label out of range: " + ordinal); + | } + | } + """.stripMargin + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegenOutput.scala b/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegenOutput.scala new file mode 100644 index 0000000000..7a6b02237d --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometBatchKernelCodegenOutput.scala @@ -0,0 +1,394 @@ +/* + * 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.codegen + +import scala.jdk.CollectionConverters._ +import scala.util.control.NonFatal + +import org.apache.arrow.memory.BufferAllocator +import org.apache.arrow.vector._ +import org.apache.arrow.vector.complex.{ListVector, MapVector, StructVector} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field} +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.types._ + +import org.apache.comet.CometArrowAllocator + +/** + * Output-side emitters for the codegen kernel: [[allocateOutput]], [[emitOutputWriter]] + * (top-level write entry), [[emitWrite]] (recursive per-type write), the output vector-class + * lookup. Paired with [[CometBatchKernelCodegenInput]] on the read side. + */ +private[codegen] object CometBatchKernelCodegenOutput { + + /** + * Spark `DataType` to an Arrow `Field` with names Comet expects on FFI export. Spark's + * `Utils.toArrowField` names list children `"element"`; this rewrites them to `"item"`. Pair + * with the [[RenamedListVector]] / [[RenamedMapVector]] / [[RenamedStructVector]] subclasses in + * [[allocateOutput]], which pin `getField()` so the cached Field actually reaches export. + */ + def toFfiArrowField(name: String, dataType: DataType, nullable: Boolean): Field = + renameForArrowRustFfi(Utils.toArrowField(name, dataType, nullable, "UTC")) + + private def renameForArrowRustFfi(field: Field): Field = { + val children = field.getChildren.asScala + if (children.isEmpty) return field + field.getType match { + case _: ArrowType.List | _: ArrowType.LargeList | _: ArrowType.FixedSizeList => + val child = children.head + val renamedChild = renameForArrowRustFfi( + new Field("item", child.getFieldType, child.getChildren)) + new Field( + field.getName, + field.getFieldType, + java.util.Collections.singletonList(renamedChild)) + case _ => + val renamedChildren = children.map(renameForArrowRustFfi).toList.asJava + new Field(field.getName, field.getFieldType, renamedChildren) + } + } + + /** + * Allocate an Arrow output vector from a pre-built `Field`. Callers cache the Field per + * `(expression, schema)` and pass it on every batch. + * + * Complex top-level types route through a [[RenamedListVector]] / [[RenamedMapVector]] / + * [[RenamedStructVector]] (see those for the runtime-vs-export naming gap). + * + * `estimatedBytes` pre-sizes the data buffer for variable-length scalar outputs. Ignored for + * other root types, and not propagated into nested var-width children (their `allocateNew` runs + * through the parent's `allocateNew`, which resets child buffers). + * + * TODO(nested-varwidth-sizing): thread the estimate into nested var-width children. + * + * TODO(cached-write-buffer-addrs): cache buffer addresses at `process` setup and emit + * `Platform.putByte` / `Platform.copyMemory` for VarChar / VarBinary / Decimal scalar outputs, + * bypassing `setSafe`'s realloc check. Depends on pre-allocated buffers. + * + * Closes the vector on any failure so a partially-initialized tree doesn't leak buffers. + */ + def allocateOutput(field: Field, numRows: Int, estimatedBytes: Int): FieldVector = { + val vec: FieldVector = field.getType match { + case _: ArrowType.List | _: ArrowType.LargeList | _: ArrowType.FixedSizeList => + val v = new RenamedListVector(field, CometArrowAllocator) + v.initializeChildrenFromFields(field.getChildren) + v + case _: ArrowType.Map => + val v = new RenamedMapVector(field, CometArrowAllocator) + v.initializeChildrenFromFields(field.getChildren) + v + case _: ArrowType.Struct => + val v = new RenamedStructVector(field, CometArrowAllocator) + v.initializeChildrenFromFields(field.getChildren) + v + case _ => + field.createVector(CometArrowAllocator).asInstanceOf[FieldVector] + } + try { + vec.setInitialCapacity(numRows) + vec match { + case v: BaseVariableWidthVector if estimatedBytes > 0 => + v.allocateNew(estimatedBytes.toLong, numRows) + case _ => + vec.allocateNew() + } + vec + } catch { + case t: Throwable => + try vec.close() + catch { + case NonFatal(_) => () + } + throw t + } + } + + /** + * Pin `getField()` to the cached Field so FFI export carries the names Comet expects. + * `ListVector.getField` rebuilds child labels from the runtime data vector, which + * `addOrGetVector` hardcodes to `"$data$"`. Applied to `MapVector` and `StructVector` too + * because their `getField` recurses and can pick up a buried `ListVector`'s `"$data$"`. + */ + private final class RenamedListVector(exportField: Field, allocator: BufferAllocator) + extends ListVector(exportField, allocator, null) { + override def getField: Field = exportField + } + + private final class RenamedMapVector(exportField: Field, allocator: BufferAllocator) + extends MapVector(exportField, allocator, null) { + override def getField: Field = exportField + } + + private final class RenamedStructVector(exportField: Field, allocator: BufferAllocator) + extends StructVector(exportField, allocator, null) { + override def getField: Field = exportField + } + + /** + * Returns `(concreteVectorClassName, batchSetup, perRowSnippet)`. `output` is cast to the + * concrete class in `process`'s prelude so `emitWrite`'s complex-type branches can hoist child + * casts off `output` without re-casting per row. + */ + def emitOutputWriter( + dataType: DataType, + valueTerm: String, + ctx: CodegenContext): (String, String, String) = { + val cls = outputVectorClass(dataType) + val emit = emitWrite("output", "i", valueTerm, dataType, ctx) + (cls, emit.setup, emit.perRow) + } + + /** Concrete Arrow vector class name for the output type, used to cast `outRaw` once. */ + private def outputVectorClass(dataType: DataType): String = dataType match { + case BooleanType => classOf[BitVector].getName + case ByteType => classOf[TinyIntVector].getName + case ShortType => classOf[SmallIntVector].getName + case IntegerType => classOf[IntVector].getName + case LongType => classOf[BigIntVector].getName + case FloatType => classOf[Float4Vector].getName + case DoubleType => classOf[Float8Vector].getName + case _: DecimalType => classOf[DecimalVector].getName + case _: StringType => classOf[VarCharVector].getName + case BinaryType => classOf[VarBinaryVector].getName + case DateType => classOf[DateDayVector].getName + case TimestampType => classOf[TimeStampMicroTZVector].getName + case TimestampNTZType => classOf[TimeStampMicroVector].getName + case _: ArrayType => classOf[ListVector].getName + case _: StructType => classOf[StructVector].getName + case _: MapType => classOf[MapVector].getName + case other => + throw new UnsupportedOperationException( + s"CometBatchKernelCodegen.outputVectorClass: unsupported output type $other") + } + + /** + * Composable write emitter. Returns an [[OutputEmit]] whose `setup` declares once-per-batch + * typed child-vector casts and whose `perRow` writes `source` into `targetVec` at `idx`. + * `targetVec` is assumed pre-cast to the right Arrow class (root prelude or a parent's setup). + * + * Scalars emit `perRow` only. Complex types emit both. Inner setup bubbles up so deep child + * casts land at the batch prelude. + */ + private def emitWrite( + targetVec: String, + idx: String, + source: String, + dataType: DataType, + ctx: CodegenContext): OutputEmit = dataType match { + case BooleanType => + OutputEmit("", s"$targetVec.set($idx, $source ? 1 : 0);") + case ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | DateType | + TimestampType | TimestampNTZType => + // Spark codegen emits the matching primitive Java type; Arrow `set` overloads accept it. + OutputEmit("", s"$targetVec.set($idx, $source);") + case dt: DecimalType => + // DecimalOutputShortFastPath: precision <= 18 fits in a signed long, so pass the unscaled + // value to `setSafe(int, long)` and skip the BigDecimal allocation. + val write = + if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + s"$targetVec.setSafe($idx, $source.toUnscaledLong());" + } else { + s"$targetVec.setSafe($idx, $source.toJavaBigDecimal());" + } + OutputEmit("", write) + case _: StringType => + // Utf8OutputOnHeapShortcut: when the UTF8String is on-heap (Spark's string functions + // allocate results on-heap), pass its backing byte[] directly to `setSafe`, skipping the + // `getBytes()` allocation. Off-heap falls back to `getBytes()`. + // + // TODO(utf8-unsafe-write): output-side equivalent of `UTF8String.fromAddress`. Coupled + // with `cached-write-buffer-addrs` and a pre-allocated buffer. + val bBase = ctx.freshName("utfBase") + val bLen = ctx.freshName("utfLen") + val bArr = ctx.freshName("utfArr") + OutputEmit( + "", + s"""Object $bBase = $source.getBaseObject(); + |int $bLen = $source.numBytes(); + |if ($bBase instanceof byte[]) { + | $targetVec.setSafe($idx, (byte[]) $bBase, + | (int) ($source.getBaseOffset() + | - org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET), + | $bLen); + |} else { + | byte[] $bArr = $source.getBytes(); + | $targetVec.setSafe($idx, $bArr, 0, $bArr.length); + |}""".stripMargin) + case BinaryType => + OutputEmit("", s"$targetVec.setSafe($idx, $source, 0, $source.length);") + case ArrayType(elementType, containsNull) => + // Spark's `doGenCode` for ArrayType produces an `ArrayData` value. Iterate elements, + // write each into the `ListVector`'s child, bracket with `startNewValue`/`endValue`. The + // element write recurses through `emitWrite` on the child vector so any supported scalar + // becomes a valid element. Nested complex types compose. `targetVec` is a `ListVector` at + // the call site, and only its data vector needs casting (in setup). + // + // NullableElementElision: when `containsNull == false` drop the `isNullAt` guard at + // source level rather than relying on JIT folding. + val childVar = ctx.freshName("outListChild") + val childClass = outputVectorClass(elementType) + val arrVar = ctx.freshName("arr") + val nVar = ctx.freshName("n") + val childIdx = ctx.freshName("cidx") + val jVar = ctx.freshName("j") + val elemSource = emitSpecializedGetterExpr(arrVar, jVar, elementType) + val inner = emitWrite(childVar, s"$childIdx + $jVar", elemSource, elementType, ctx) + val setup = + (s"$childClass $childVar = ($childClass) $targetVec.getDataVector();" +: + Seq(inner.setup).filter(_.nonEmpty)).mkString("\n") + val elementWrite = if (containsNull) { + s"""if ($arrVar.isNullAt($jVar)) { + | $childVar.setNull($childIdx + $jVar); + | } else { + | ${inner.perRow} + | }""".stripMargin + } else { + inner.perRow + } + val perRow = + s"""org.apache.spark.sql.catalyst.util.ArrayData $arrVar = $source; + |int $nVar = $arrVar.numElements(); + |int $childIdx = $targetVec.startNewValue($idx); + |for (int $jVar = 0; $jVar < $nVar; $jVar++) { + | $elementWrite + |} + |$targetVec.endValue($idx, $nVar);""".stripMargin + OutputEmit(setup, perRow) + case st: StructType => + // Spark's `doGenCode` for StructType produces an `InternalRow`. Typed child-vector casts + // hoist to setup, and the per-row body references the hoisted names. + // + // For non-nullable fields, drop the `row.isNullAt($fi)` guard at source level so HotSpot + // emits a straight write path per field rather than a branch. + val rowVar = ctx.freshName("row") + val perField = st.fields.zipWithIndex.map { case (field, fi) => + val childVar = ctx.freshName("outStructChild") + val childClass = outputVectorClass(field.dataType) + val childDecl = + s"$childClass $childVar = ($childClass) $targetVec.getChildByOrdinal($fi);" + val fieldSource = emitSpecializedGetterExpr(rowVar, fi.toString, field.dataType) + val inner = emitWrite(childVar, idx, fieldSource, field.dataType, ctx) + val write = + if (!field.nullable) { + inner.perRow + } else { + s"""if ($rowVar.isNullAt($fi)) { + | $childVar.setNull($idx); + |} else { + | ${inner.perRow} + |}""".stripMargin + } + val perFieldSetup = (Seq(childDecl) ++ Seq(inner.setup).filter(_.nonEmpty)).mkString("\n") + (perFieldSetup, write) + } + val setup = perField.map(_._1).mkString("\n") + val perFieldWrites = perField.map(_._2).mkString("\n") + val perRow = + s"""org.apache.spark.sql.catalyst.InternalRow $rowVar = $source; + |$targetVec.setIndexDefined($idx); + |$perFieldWrites""".stripMargin + OutputEmit(setup, perRow) + case mt: MapType => + // Spark's `doGenCode` for MapType produces a `MapData`. Typed child-vector casts for the + // entries struct and the key/value children hoist to setup. + // + // Per-row: read keyArray/valueArray, open via `startNewValue(idx)`, write each pair into + // the entries struct (key always non-null per Spark/Arrow invariant, value guarded on + // `valueContainsNull`), close via `endValue(idx, n)`. + val entriesVar = ctx.freshName("outMapEntries") + val keyVar = ctx.freshName("outMapKey") + val valVar = ctx.freshName("outMapVal") + val mapSrc = ctx.freshName("mapSrc") + val keyArr = ctx.freshName("keyArr") + val valArr = ctx.freshName("valArr") + val nVar = ctx.freshName("n") + val childIdx = ctx.freshName("cidx") + val jVar = ctx.freshName("j") + val structClass = classOf[StructVector].getName + val keyClass = outputVectorClass(mt.keyType) + val valClass = outputVectorClass(mt.valueType) + val keySrcExpr = emitSpecializedGetterExpr(keyArr, jVar, mt.keyType) + val valSrcExpr = emitSpecializedGetterExpr(valArr, jVar, mt.valueType) + val keyEmit = emitWrite(keyVar, s"$childIdx + $jVar", keySrcExpr, mt.keyType, ctx) + val valEmit = emitWrite(valVar, s"$childIdx + $jVar", valSrcExpr, mt.valueType, ctx) + val setup = + (Seq( + s"$structClass $entriesVar = ($structClass) $targetVec.getDataVector();", + s"$keyClass $keyVar = ($keyClass) $entriesVar.getChildByOrdinal(0);", + s"$valClass $valVar = ($valClass) $entriesVar.getChildByOrdinal(1);") ++ + Seq(keyEmit.setup, valEmit.setup).filter(_.nonEmpty)).mkString("\n") + val valueWrite = if (mt.valueContainsNull) { + s"""if ($valArr.isNullAt($jVar)) { + | $valVar.setNull($childIdx + $jVar); + | } else { + | ${valEmit.perRow} + | }""".stripMargin + } else { + valEmit.perRow + } + val perRow = + s"""org.apache.spark.sql.catalyst.util.MapData $mapSrc = $source; + |org.apache.spark.sql.catalyst.util.ArrayData $keyArr = $mapSrc.keyArray(); + |org.apache.spark.sql.catalyst.util.ArrayData $valArr = $mapSrc.valueArray(); + |int $nVar = $mapSrc.numElements(); + |int $childIdx = $targetVec.startNewValue($idx); + |for (int $jVar = 0; $jVar < $nVar; $jVar++) { + | $entriesVar.setIndexDefined($childIdx + $jVar); + | ${keyEmit.perRow} + | $valueWrite + |} + |$targetVec.endValue($idx, $nVar);""".stripMargin + OutputEmit(setup, perRow) + case other => + throw new UnsupportedOperationException( + s"CometBatchKernelCodegen.emitWrite: unsupported output type $other") + } + + /** + * Java expression that reads a typed value out of a `SpecializedGetters` (both `ArrayData` and + * `InternalRow` implement it). Used by [[emitWrite]] to source each element/field for its + * recursive inner write. + */ + private def emitSpecializedGetterExpr(target: String, idx: String, elemType: DataType): String = + elemType match { + case BooleanType => s"$target.getBoolean($idx)" + case ByteType => s"$target.getByte($idx)" + case ShortType => s"$target.getShort($idx)" + case IntegerType | DateType => s"$target.getInt($idx)" + case LongType | TimestampType | TimestampNTZType => s"$target.getLong($idx)" + case FloatType => s"$target.getFloat($idx)" + case DoubleType => s"$target.getDouble($idx)" + case dt: DecimalType => s"$target.getDecimal($idx, ${dt.precision}, ${dt.scale})" + case _: StringType => s"$target.getUTF8String($idx)" + case BinaryType => s"$target.getBinary($idx)" + case ArrayType(_, _) => s"$target.getArray($idx)" + case _: MapType => s"$target.getMap($idx)" + case _: StructType => + val numFields = elemType.asInstanceOf[StructType].fields.length + s"$target.getStruct($idx, $numFields)" + case other => + throw new UnsupportedOperationException( + s"CometBatchKernelCodegen.emitSpecializedGetterExpr: unsupported type $other") + } + + /** `setup` is once-per-batch (typed child-vector casts); `perRow` runs per row. */ + private case class OutputEmit(setup: String, perRow: String) +} diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometInternalRow.scala b/spark/src/main/scala/org/apache/comet/codegen/CometInternalRow.scala new file mode 100644 index 0000000000..77321fed9c --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometInternalRow.scala @@ -0,0 +1,84 @@ +/* + * 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.codegen + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} + +import org.apache.comet.shims.CometInternalRowShim + +/** + * Throwing-default `InternalRow` base for the codegen kernel. Subclasses override only the + * getters their input shape needs. Centralizing the throws absorbs forward-compat breakage when + * Spark adds abstract methods. + * + * Two consumers: the compiled kernel (`ctx.INPUT_ROW = "row"` aliases `this`) and per-column + * `InputStruct_${path}` nested classes that back `getStruct(ord, n)`. + */ +abstract class CometInternalRow extends InternalRow with CometInternalRowShim { + + override def numFields: Int = unsupported("numFields") + + override def getInterval(ordinal: Int): CalendarInterval = unsupported("getInterval") + + override def get(ordinal: Int, dataType: DataType): AnyRef = + CometSpecializedGettersDispatch.get(this, ordinal, dataType) + + override def isNullAt(ordinal: Int): Boolean = unsupported("isNullAt") + + override def getBoolean(ordinal: Int): Boolean = unsupported("getBoolean") + + override def getByte(ordinal: Int): Byte = unsupported("getByte") + + override def getShort(ordinal: Int): Short = unsupported("getShort") + + override def getInt(ordinal: Int): Int = unsupported("getInt") + + override def getLong(ordinal: Int): Long = unsupported("getLong") + + override def getFloat(ordinal: Int): Float = unsupported("getFloat") + + override def getDouble(ordinal: Int): Double = unsupported("getDouble") + + override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = + unsupported("getDecimal") + + override def getUTF8String(ordinal: Int): UTF8String = unsupported("getUTF8String") + + override def getBinary(ordinal: Int): Array[Byte] = unsupported("getBinary") + + override def getStruct(ordinal: Int, numFields: Int): InternalRow = unsupported("getStruct") + + override def getArray(ordinal: Int): ArrayData = unsupported("getArray") + + override def getMap(ordinal: Int): MapData = unsupported("getMap") + + override def setNullAt(i: Int): Unit = unsupported("setNullAt") + + override def update(i: Int, value: Any): Unit = unsupported("update") + + override def copy(): InternalRow = unsupported("copy") + + protected def unsupported(method: String): Nothing = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: $method not implemented for this row shape") +} diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometMapData.scala b/spark/src/main/scala/org/apache/comet/codegen/CometMapData.scala new file mode 100644 index 0000000000..ac8254e72d --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometMapData.scala @@ -0,0 +1,54 @@ +/* + * 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.codegen + +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} + +/** + * Throwing-default `MapData` base for the codegen kernel. Per-column `InputMap_${path}` + * subclasses override `numElements`, `keyArray`, and `valueArray` (the latter two return + * `InputArray_*` views over the same backing key/value vectors). + * + * `MapData` does not extend `SpecializedGetters`, so this base does not mix in the row/array shim + * or delegate to [[CometSpecializedGettersDispatch]]. + */ +abstract class CometMapData extends MapData { + + override def keyArray(): ArrayData = unsupported("keyArray") + + override def valueArray(): ArrayData = unsupported("valueArray") + + override def copy(): MapData = unsupported("copy") + + protected def unsupported(method: String): Nothing = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: $method not implemented for this map shape") + + override def toString(): String = { + val n = + try numElements().toString + catch { + case _: Throwable => "?" + } + s"${getClass.getSimpleName}(numElements=$n)" + } + + override def numElements(): Int = unsupported("numElements") +} diff --git a/spark/src/main/scala/org/apache/comet/codegen/CometSpecializedGettersDispatch.scala b/spark/src/main/scala/org/apache/comet/codegen/CometSpecializedGettersDispatch.scala new file mode 100644 index 0000000000..2f81c58c06 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/codegen/CometSpecializedGettersDispatch.scala @@ -0,0 +1,59 @@ +/* + * 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.codegen + +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.types._ + +/** + * Shared `SpecializedGetters.get(ordinal, dataType)` dispatch used by [[CometInternalRow]] and + * [[CometArrayData]]. Spark codegen paths (notably `SafeProjection` for ScalaUDF struct args) and + * interpreted-eval fallbacks (`ArrayDistinct.nullSafeEval` etc.) call the generic `get` instead + * of the typed getter, so both kernel-side bases need a non-throwing implementation. + * + * For complex types, the typed getter allocates a fresh `InputStruct_*` / `InputArray_*` / + * `InputMap_*` per call (`ColumnarRow`-style), so retain-by-reference consumers like + * `OpenHashSet` get distinct identities. + */ +private[codegen] object CometSpecializedGettersDispatch { + + def get(g: SpecializedGetters, ordinal: Int, dataType: DataType): AnyRef = { + if (g.isNullAt(ordinal)) return null + dataType match { + case BooleanType => java.lang.Boolean.valueOf(g.getBoolean(ordinal)) + case ByteType => java.lang.Byte.valueOf(g.getByte(ordinal)) + case ShortType => java.lang.Short.valueOf(g.getShort(ordinal)) + case IntegerType | DateType => java.lang.Integer.valueOf(g.getInt(ordinal)) + case LongType | TimestampType | TimestampNTZType => + java.lang.Long.valueOf(g.getLong(ordinal)) + case FloatType => java.lang.Float.valueOf(g.getFloat(ordinal)) + case DoubleType => java.lang.Double.valueOf(g.getDouble(ordinal)) + case _: StringType => g.getUTF8String(ordinal) + case BinaryType => g.getBinary(ordinal) + case dt: DecimalType => g.getDecimal(ordinal, dt.precision, dt.scale) + case st: StructType => g.getStruct(ordinal, st.size) + case _: ArrayType => g.getArray(ordinal) + case _: MapType => g.getMap(ordinal) + case other => + throw new UnsupportedOperationException( + s"${g.getClass.getSimpleName}: get for dataType $other not implemented") + } + } +} diff --git a/common/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala b/spark/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala similarity index 100% rename from common/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala rename to spark/src/main/scala/org/apache/comet/objectstore/NativeConfig.scala diff --git a/common/src/main/scala/org/apache/comet/package.scala b/spark/src/main/scala/org/apache/comet/package.scala similarity index 100% rename from common/src/main/scala/org/apache/comet/package.scala rename to spark/src/main/scala/org/apache/comet/package.scala diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala deleted file mode 100644 index 0d178dbf81..0000000000 --- a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala +++ /dev/null @@ -1,260 +0,0 @@ -/* - * 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.parquet - -import scala.jdk.CollectionConverters._ - -import org.apache.hadoop.conf.Configuration -import org.apache.parquet.filter2.predicate.FilterApi -import org.apache.parquet.hadoop.ParquetInputFormat -import org.apache.parquet.hadoop.metadata.FileMetaData -import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.catalyst.util.RebaseDateTime.RebaseSpec -import org.apache.spark.sql.comet.CometMetricNode -import org.apache.spark.sql.execution.datasources.DataSourceUtils -import org.apache.spark.sql.execution.datasources.PartitionedFile -import org.apache.spark.sql.execution.datasources.RecordReaderIterator -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions -import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.{DateType, StructType, TimestampType} -import org.apache.spark.util.SerializableConfiguration - -import org.apache.comet.CometConf -import org.apache.comet.CometSparkSessionExtensions.isSpark41Plus -import org.apache.comet.MetricsSupport -import org.apache.comet.shims.ShimSQLConf -import org.apache.comet.vector.CometVector - -/** - * A Comet specific Parquet format. This mostly reuse the functionalities from Spark's - * [[ParquetFileFormat]], but overrides: - * - * - `vectorTypes`, so Spark allocates [[CometVector]] instead of it's own on-heap or off-heap - * column vector in the whole-stage codegen path. - * - `supportBatch`, which simply returns true since data types should have already been checked - * in [[org.apache.comet.CometSparkSessionExtensions]] - * - `buildReaderWithPartitionValues`, so Spark calls Comet's Parquet reader to read values. - */ -class CometParquetFileFormat(session: SparkSession) - extends ParquetFileFormat - with MetricsSupport - with ShimSQLConf { - metrics = - CometMetricNode.nativeScanMetrics(session.sparkContext) ++ CometMetricNode.parquetScanMetrics( - session.sparkContext) - - override def shortName(): String = "parquet" - override def toString: String = "CometParquet" - override def hashCode(): Int = getClass.hashCode() - override def equals(other: Any): Boolean = other.isInstanceOf[CometParquetFileFormat] - - override def vectorTypes( - requiredSchema: StructType, - partitionSchema: StructType, - sqlConf: SQLConf): Option[Seq[String]] = { - val length = requiredSchema.fields.length + partitionSchema.fields.length - Option(Seq.fill(length)(classOf[CometVector].getName)) - } - - override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = true - - override def buildReaderWithPartitionValues( - sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { - val sqlConf = sparkSession.sessionState.conf - CometParquetFileFormat.populateConf(sqlConf, hadoopConf) - val broadcastedHadoopConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - - val isCaseSensitive = sqlConf.caseSensitiveAnalysis - val useFieldId = CometParquetUtils.readFieldId(sqlConf) - val ignoreMissingIds = CometParquetUtils.ignoreMissingIds(sqlConf) - // SPARK-53535 (Spark 4.1+): when reading a struct whose requested fields are all - // missing in the Parquet file, the new default preserves the parent struct's - // nullness from the file. Pre-4.1 Spark hardcodes the legacy behavior, so we - // default to "true" there for backwards compatibility. - val returnNullStructIfAllFieldsMissing = sqlConf - .getConfString( - "spark.sql.legacy.parquet.returnNullStructIfAllFieldsMissing", - if (isSpark41Plus) "false" else "true") - .toBoolean - val pushDownDate = sqlConf.parquetFilterPushDownDate - val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp - val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal - val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate - val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold - val optionsMap = CaseInsensitiveMap[String](options) - val parquetOptions = new ParquetOptions(optionsMap, sqlConf) - val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead - val parquetFilterPushDown = sqlConf.parquetFilterPushDown && - CometConf.COMET_RESPECT_PARQUET_FILTER_PUSHDOWN.get(sqlConf) - - // Comet specific configurations - val capacity = CometConf.COMET_BATCH_SIZE.get(sqlConf) - - (file: PartitionedFile) => { - val sharedConf = broadcastedHadoopConf.value.value - val footer = FooterReader.readFooter(sharedConf, file) - val footerFileMetaData = footer.getFileMetaData - val datetimeRebaseSpec = CometParquetFileFormat.getDatetimeRebaseSpec( - file, - requiredSchema, - sharedConf, - footerFileMetaData, - datetimeRebaseModeInRead) - - val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters( - parquetSchema, - dataSchema, - pushDownDate, - pushDownTimestamp, - pushDownDecimal, - pushDownStringPredicate, - pushDownInFilterThreshold, - isCaseSensitive, - datetimeRebaseSpec) - - val pushed = if (parquetFilterPushDown) { - filters - .flatMap(parquetFilters.createFilter) - .reduceOption(FilterApi.and) - } else { - None - } - pushed.foreach(p => ParquetInputFormat.setFilterPredicate(sharedConf, p)) - val pushedNative = if (parquetFilterPushDown) { - parquetFilters.createNativeFilters(filters) - } else { - None - } - val recordBatchReader = new NativeBatchReader( - sharedConf, - file, - footer, - pushedNative.orNull, - capacity, - requiredSchema, - dataSchema, - isCaseSensitive, - useFieldId, - ignoreMissingIds, - datetimeRebaseSpec.mode == CORRECTED, - returnNullStructIfAllFieldsMissing, - partitionSchema, - file.partitionValues, - metrics.asJava, - CometMetricNode(metrics)) - try { - recordBatchReader.init() - } catch { - case e: Throwable => - recordBatchReader.close() - throw e - } - val iter = new RecordReaderIterator(recordBatchReader) - try { - iter.asInstanceOf[Iterator[InternalRow]] - } catch { - case e: Throwable => - iter.close() - throw e - } - } - } -} - -object CometParquetFileFormat extends Logging with ShimSQLConf { - - /** - * Populates Parquet related configurations from the input `sqlConf` to the `hadoopConf` - */ - def populateConf(sqlConf: SQLConf, hadoopConf: Configuration): Unit = { - hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) - hadoopConf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, sqlConf.sessionLocalTimeZone) - hadoopConf.setBoolean( - SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, - sqlConf.nestedSchemaPruningEnabled) - hadoopConf.setBoolean(SQLConf.CASE_SENSITIVE.key, sqlConf.caseSensitiveAnalysis) - - // Sets flags for `ParquetToSparkSchemaConverter` - hadoopConf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, sqlConf.isParquetBinaryAsString) - hadoopConf.setBoolean( - SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, - sqlConf.isParquetINT96AsTimestamp) - - // Comet specific configs - hadoopConf.setBoolean( - CometConf.COMET_USE_DECIMAL_128.key, - CometConf.COMET_USE_DECIMAL_128.get()) - hadoopConf.setBoolean( - CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key, - CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.get()) - hadoopConf.setInt(CometConf.COMET_BATCH_SIZE.key, CometConf.COMET_BATCH_SIZE.get()) - } - - def getDatetimeRebaseSpec( - file: PartitionedFile, - sparkSchema: StructType, - sharedConf: Configuration, - footerFileMetaData: FileMetaData, - datetimeRebaseModeInRead: String): RebaseSpec = { - val exceptionOnRebase = sharedConf.getBoolean( - CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.key, - CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP.defaultValue.get) - var datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( - footerFileMetaData.getKeyValueMetaData.get, - datetimeRebaseModeInRead) - val hasDateOrTimestamp = sparkSchema.exists(f => - f.dataType match { - case DateType | TimestampType => true - case _ => false - }) - - if (hasDateOrTimestamp && datetimeRebaseSpec.mode == LEGACY) { - if (exceptionOnRebase) { - logWarning( - s"""Found Parquet file $file that could potentially contain dates/timestamps that were - written in legacy hybrid Julian/Gregorian calendar. Unlike Spark 3+, which will rebase - and return these according to the new Proleptic Gregorian calendar, Comet will throw - exception when reading them. If you want to read them as it is according to the hybrid - Julian/Gregorian calendar, please set `spark.comet.exceptionOnDatetimeRebase` to - false. Otherwise, if you want to read them according to the new Proleptic Gregorian - calendar, please disable Comet for this query.""") - } else { - // do not throw exception on rebase - read as it is - datetimeRebaseSpec = datetimeRebaseSpec.copy(CORRECTED) - } - } - - datetimeRebaseSpec - } -} diff --git a/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala similarity index 100% rename from common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala rename to spark/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala diff --git a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala deleted file mode 100644 index f8da68d59f..0000000000 --- a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala +++ /dev/null @@ -1,1044 +0,0 @@ -/* - * 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.parquet - -import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Long => JLong, Short => JShort} -import java.math.{BigDecimal => JBigDecimal} -import java.sql.{Date, Timestamp} -import java.time.{Duration, Instant, LocalDate, Period} -import java.util.Locale - -import scala.jdk.CollectionConverters._ - -import org.apache.parquet.column.statistics.{Statistics => ParquetStatistics} -import org.apache.parquet.filter2.predicate._ -import org.apache.parquet.filter2.predicate.SparkFilterApi._ -import org.apache.parquet.io.api.Binary -import org.apache.parquet.schema.{GroupType, LogicalTypeAnnotation, MessageType, PrimitiveComparator, PrimitiveType, Type} -import org.apache.parquet.schema.LogicalTypeAnnotation.{DecimalLogicalTypeAnnotation, TimeUnit} -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ -import org.apache.parquet.schema.Type.Repetition -import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, CaseInsensitiveMap, DateTimeUtils, IntervalUtils} -import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseGregorianToJulianMicros, RebaseSpec} -import org.apache.spark.sql.sources -import org.apache.spark.sql.types.StructType -import org.apache.spark.unsafe.types.UTF8String - -import com.google.protobuf.CodedOutputStream - -import org.apache.comet.parquet.SourceFilterSerde.{createBinaryExpr, createNameExpr, createUnaryExpr, createValueExpr} -import org.apache.comet.serde.ExprOuterClass -import org.apache.comet.serde.QueryPlanSerde.scalarFunctionExprToProto -import org.apache.comet.shims.ShimSQLConf - -/** - * Copied from Spark 3.4, in order to fix Parquet shading issue. TODO: find a way to remove this - * duplication - * - * Some utility function to convert Spark data source filters to Parquet filters. - */ -class ParquetFilters( - schema: MessageType, - dataSchema: StructType, - pushDownDate: Boolean, - pushDownTimestamp: Boolean, - pushDownDecimal: Boolean, - pushDownStringPredicate: Boolean, - pushDownInFilterThreshold: Int, - caseSensitive: Boolean, - datetimeRebaseSpec: RebaseSpec) - extends ShimSQLConf { - // A map which contains parquet field name and data type, if predicate push down applies. - // - // Each key in `nameToParquetField` represents a column; `dots` are used as separators for - // nested columns. If any part of the names contains `dots`, it is quoted to avoid confusion. - // See `org.apache.spark.sql.connector.catalog.quote` for implementation details. - private val nameToParquetField: Map[String, ParquetPrimitiveField] = { - // Recursively traverse the parquet schema to get primitive fields that can be pushed-down. - // `parentFieldNames` is used to keep track of the current nested level when traversing. - def getPrimitiveFields( - fields: Seq[Type], - parentFieldNames: Array[String] = Array.empty): Seq[ParquetPrimitiveField] = { - fields.flatMap { - // Parquet only supports predicate push-down for non-repeated primitive types. - // TODO(SPARK-39393): Remove extra condition when parquet added filter predicate support for - // repeated columns (https://issues.apache.org/jira/browse/PARQUET-34) - case p: PrimitiveType if p.getRepetition != Repetition.REPEATED => - Some( - ParquetPrimitiveField( - fieldNames = parentFieldNames :+ p.getName, - fieldType = ParquetSchemaType( - p.getLogicalTypeAnnotation, - p.getPrimitiveTypeName, - p.getTypeLength))) - // Note that when g is a `Struct`, `g.getOriginalType` is `null`. - // When g is a `Map`, `g.getOriginalType` is `MAP`. - // When g is a `List`, `g.getOriginalType` is `LIST`. - case g: GroupType if g.getOriginalType == null => - getPrimitiveFields(g.getFields.asScala.toSeq, parentFieldNames :+ g.getName) - // Parquet only supports push-down for primitive types; as a result, Map and List types - // are removed. - case _ => None - } - } - - val primitiveFields = getPrimitiveFields(schema.getFields.asScala.toSeq).map { field => - (field.fieldNames.toSeq.map(quoteIfNeeded).mkString("."), field) - } - if (caseSensitive) { - primitiveFields.toMap - } else { - // Don't consider ambiguity here, i.e. more than one field is matched in case insensitive - // mode, just skip pushdown for these fields, they will trigger Exception when reading, - // See: SPARK-25132. - val dedupPrimitiveFields = - primitiveFields - .groupBy(_._1.toLowerCase(Locale.ROOT)) - .filter(_._2.size == 1) - .mapValues(_.head._2) - CaseInsensitiveMap(dedupPrimitiveFields.toMap) - } - } - - /** - * Holds a single primitive field information stored in the underlying parquet file. - * - * @param fieldNames - * a field name as an array of string multi-identifier in parquet file - * @param fieldType - * field type related info in parquet file - */ - private case class ParquetPrimitiveField( - fieldNames: Array[String], - fieldType: ParquetSchemaType) - - private case class ParquetSchemaType( - logicalTypeAnnotation: LogicalTypeAnnotation, - primitiveTypeName: PrimitiveTypeName, - length: Int) - - private val ParquetBooleanType = ParquetSchemaType(null, BOOLEAN, 0) - private val ParquetByteType = - ParquetSchemaType(LogicalTypeAnnotation.intType(8, true), INT32, 0) - private val ParquetShortType = - ParquetSchemaType(LogicalTypeAnnotation.intType(16, true), INT32, 0) - private val ParquetIntegerType = ParquetSchemaType(null, INT32, 0) - private val ParquetLongType = ParquetSchemaType(null, INT64, 0) - private val ParquetFloatType = ParquetSchemaType(null, FLOAT, 0) - private val ParquetDoubleType = ParquetSchemaType(null, DOUBLE, 0) - private val ParquetStringType = - ParquetSchemaType(LogicalTypeAnnotation.stringType(), BINARY, 0) - private val ParquetBinaryType = ParquetSchemaType(null, BINARY, 0) - private val ParquetDateType = - ParquetSchemaType(LogicalTypeAnnotation.dateType(), INT32, 0) - private val ParquetTimestampMicrosType = - ParquetSchemaType(LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), INT64, 0) - private val ParquetTimestampMillisType = - ParquetSchemaType(LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), INT64, 0) - - private def dateToDays(date: Any): Int = { - val gregorianDays = date match { - case d: Date => DateTimeUtils.fromJavaDate(d) - case ld: LocalDate => DateTimeUtils.localDateToDays(ld) - } - datetimeRebaseSpec.mode match { - case LEGACY => rebaseGregorianToJulianDays(gregorianDays) - case _ => gregorianDays - } - } - - private def timestampToMicros(v: Any): JLong = { - val gregorianMicros = v match { - case i: Instant => DateTimeUtils.instantToMicros(i) - case t: Timestamp => DateTimeUtils.fromJavaTimestamp(t) - } - datetimeRebaseSpec.mode match { - case LEGACY => - rebaseGregorianToJulianMicros(datetimeRebaseSpec.timeZone, gregorianMicros) - case _ => gregorianMicros - } - } - - private def decimalToInt32(decimal: JBigDecimal): Integer = decimal.unscaledValue().intValue() - - private def decimalToInt64(decimal: JBigDecimal): JLong = decimal.unscaledValue().longValue() - - private def decimalToByteArray(decimal: JBigDecimal, numBytes: Int): Binary = { - val decimalBuffer = new Array[Byte](numBytes) - val bytes = decimal.unscaledValue().toByteArray - - val fixedLengthBytes = if (bytes.length == numBytes) { - bytes - } else { - val signByte = if (bytes.head < 0) -1: Byte else 0: Byte - java.util.Arrays.fill(decimalBuffer, 0, numBytes - bytes.length, signByte) - System.arraycopy(bytes, 0, decimalBuffer, numBytes - bytes.length, bytes.length) - decimalBuffer - } - Binary.fromConstantByteArray(fixedLengthBytes, 0, numBytes) - } - - private def timestampToMillis(v: Any): JLong = { - val micros = timestampToMicros(v) - val millis = DateTimeUtils.microsToMillis(micros) - millis.asInstanceOf[JLong] - } - - private def toIntValue(v: Any): Integer = { - Option(v) - .map { - case p: Period => IntervalUtils.periodToMonths(p) - case n => n.asInstanceOf[Number].intValue - } - .map(_.asInstanceOf[Integer]) - .orNull - } - - private def toLongValue(v: Any): JLong = v match { - case d: Duration => IntervalUtils.durationToMicros(d) - case l => l.asInstanceOf[JLong] - } - - private val makeEq - : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { - case ParquetBooleanType => - (n: Array[String], v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[JBoolean]) - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Any) => - FilterApi.eq( - intColumn(n), - Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) - case ParquetLongType => - (n: Array[String], v: Any) => FilterApi.eq(longColumn(n), v.asInstanceOf[JLong]) - case ParquetFloatType => - (n: Array[String], v: Any) => FilterApi.eq(floatColumn(n), v.asInstanceOf[JFloat]) - case ParquetDoubleType => - (n: Array[String], v: Any) => FilterApi.eq(doubleColumn(n), v.asInstanceOf[JDouble]) - - // Binary.fromString and Binary.fromByteArray don't accept null values - case ParquetStringType => - (n: Array[String], v: Any) => - FilterApi.eq( - binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) - case ParquetBinaryType => - (n: Array[String], v: Any) => - FilterApi.eq( - binaryColumn(n), - Option(v).map(_ => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) - case ParquetDateType if pushDownDate => - (n: Array[String], v: Any) => - FilterApi.eq( - intColumn(n), - Option(v).map(date => dateToDays(date).asInstanceOf[Integer]).orNull) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Any) => - FilterApi.eq(longColumn(n), Option(v).map(timestampToMicros).orNull) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Any) => - FilterApi.eq(longColumn(n), Option(v).map(timestampToMillis).orNull) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.eq( - intColumn(n), - Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.eq( - longColumn(n), - Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.eq( - binaryColumn(n), - Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) - } - - private val makeNotEq - : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { - case ParquetBooleanType => - (n: Array[String], v: Any) => FilterApi.notEq(booleanColumn(n), v.asInstanceOf[JBoolean]) - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Any) => - FilterApi.notEq( - intColumn(n), - Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) - case ParquetLongType => - (n: Array[String], v: Any) => FilterApi.notEq(longColumn(n), v.asInstanceOf[JLong]) - case ParquetFloatType => - (n: Array[String], v: Any) => FilterApi.notEq(floatColumn(n), v.asInstanceOf[JFloat]) - case ParquetDoubleType => - (n: Array[String], v: Any) => FilterApi.notEq(doubleColumn(n), v.asInstanceOf[JDouble]) - - case ParquetStringType => - (n: Array[String], v: Any) => - FilterApi.notEq( - binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) - case ParquetBinaryType => - (n: Array[String], v: Any) => - FilterApi.notEq( - binaryColumn(n), - Option(v).map(_ => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) - case ParquetDateType if pushDownDate => - (n: Array[String], v: Any) => - FilterApi.notEq( - intColumn(n), - Option(v).map(date => dateToDays(date).asInstanceOf[Integer]).orNull) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Any) => - FilterApi.notEq(longColumn(n), Option(v).map(timestampToMicros).orNull) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Any) => - FilterApi.notEq(longColumn(n), Option(v).map(timestampToMillis).orNull) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.notEq( - intColumn(n), - Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.notEq( - longColumn(n), - Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.notEq( - binaryColumn(n), - Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) - } - - private val makeLt - : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Any) => - FilterApi.lt(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) - case ParquetLongType => - (n: Array[String], v: Any) => FilterApi.lt(longColumn(n), v.asInstanceOf[JLong]) - case ParquetFloatType => - (n: Array[String], v: Any) => FilterApi.lt(floatColumn(n), v.asInstanceOf[JFloat]) - case ParquetDoubleType => - (n: Array[String], v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[JDouble]) - - case ParquetStringType => - (n: Array[String], v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) - case ParquetBinaryType => - (n: Array[String], v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) - case ParquetDateType if pushDownDate => - (n: Array[String], v: Any) => - FilterApi.lt(intColumn(n), dateToDays(v).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.lt(longColumn(n), timestampToMicros(v)) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.lt(longColumn(n), timestampToMillis(v)) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.lt(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.lt(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.lt(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) - } - - private val makeLtEq - : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Any) => - FilterApi.ltEq(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) - case ParquetLongType => - (n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), v.asInstanceOf[JLong]) - case ParquetFloatType => - (n: Array[String], v: Any) => FilterApi.ltEq(floatColumn(n), v.asInstanceOf[JFloat]) - case ParquetDoubleType => - (n: Array[String], v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[JDouble]) - - case ParquetStringType => - (n: Array[String], v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) - case ParquetBinaryType => - (n: Array[String], v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) - case ParquetDateType if pushDownDate => - (n: Array[String], v: Any) => - FilterApi.ltEq(intColumn(n), dateToDays(v).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), timestampToMicros(v)) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.ltEq(longColumn(n), timestampToMillis(v)) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.ltEq(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.ltEq(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.ltEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) - } - - private val makeGt - : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Any) => - FilterApi.gt(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) - case ParquetLongType => - (n: Array[String], v: Any) => FilterApi.gt(longColumn(n), v.asInstanceOf[JLong]) - case ParquetFloatType => - (n: Array[String], v: Any) => FilterApi.gt(floatColumn(n), v.asInstanceOf[JFloat]) - case ParquetDoubleType => - (n: Array[String], v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[JDouble]) - - case ParquetStringType => - (n: Array[String], v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) - case ParquetBinaryType => - (n: Array[String], v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) - case ParquetDateType if pushDownDate => - (n: Array[String], v: Any) => - FilterApi.gt(intColumn(n), dateToDays(v).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.gt(longColumn(n), timestampToMicros(v)) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.gt(longColumn(n), timestampToMillis(v)) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.gt(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.gt(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.gt(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) - } - - private val makeGtEq - : PartialFunction[ParquetSchemaType, (Array[String], Any) => FilterPredicate] = { - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Any) => - FilterApi.gtEq(intColumn(n), v.asInstanceOf[Number].intValue.asInstanceOf[Integer]) - case ParquetLongType => - (n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), v.asInstanceOf[JLong]) - case ParquetFloatType => - (n: Array[String], v: Any) => FilterApi.gtEq(floatColumn(n), v.asInstanceOf[JFloat]) - case ParquetDoubleType => - (n: Array[String], v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[JDouble]) - - case ParquetStringType => - (n: Array[String], v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) - case ParquetBinaryType => - (n: Array[String], v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) - case ParquetDateType if pushDownDate => - (n: Array[String], v: Any) => - FilterApi.gtEq(intColumn(n), dateToDays(v).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), timestampToMicros(v)) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Any) => FilterApi.gtEq(longColumn(n), timestampToMillis(v)) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.gtEq(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.gtEq(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (n: Array[String], v: Any) => - FilterApi.gtEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) - } - - private val makeInPredicate: PartialFunction[ - ParquetSchemaType, - (Array[String], Array[Any], ParquetStatistics[_]) => FilterPredicate] = { - case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(toIntValue(_).toInt).foreach(statistics.updateStats) - FilterApi.and( - FilterApi.gtEq(intColumn(n), statistics.genericGetMin().asInstanceOf[Integer]), - FilterApi.ltEq(intColumn(n), statistics.genericGetMax().asInstanceOf[Integer])) - - case ParquetLongType => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(toLongValue).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), - FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) - - case ParquetFloatType => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(_.asInstanceOf[JFloat]).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(floatColumn(n), statistics.genericGetMin().asInstanceOf[JFloat]), - FilterApi.ltEq(floatColumn(n), statistics.genericGetMax().asInstanceOf[JFloat])) - - case ParquetDoubleType => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(_.asInstanceOf[JDouble]).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(doubleColumn(n), statistics.genericGetMin().asInstanceOf[JDouble]), - FilterApi.ltEq(doubleColumn(n), statistics.genericGetMax().asInstanceOf[JDouble])) - - case ParquetStringType => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(s => Binary.fromString(s.asInstanceOf[String])).foreach(statistics.updateStats) - FilterApi.and( - FilterApi.gtEq(binaryColumn(n), statistics.genericGetMin().asInstanceOf[Binary]), - FilterApi.ltEq(binaryColumn(n), statistics.genericGetMax().asInstanceOf[Binary])) - - case ParquetBinaryType => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(b => Binary.fromReusedByteArray(b.asInstanceOf[Array[Byte]])) - .foreach(statistics.updateStats) - FilterApi.and( - FilterApi.gtEq(binaryColumn(n), statistics.genericGetMin().asInstanceOf[Binary]), - FilterApi.ltEq(binaryColumn(n), statistics.genericGetMax().asInstanceOf[Binary])) - - case ParquetDateType if pushDownDate => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(dateToDays).map(_.asInstanceOf[Integer]).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(intColumn(n), statistics.genericGetMin().asInstanceOf[Integer]), - FilterApi.ltEq(intColumn(n), statistics.genericGetMax().asInstanceOf[Integer])) - - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(timestampToMicros).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), - FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) - - case ParquetTimestampMillisType if pushDownTimestamp => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(timestampToMillis).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), - FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT32, _) if pushDownDecimal => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(_.asInstanceOf[JBigDecimal]).map(decimalToInt32).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(intColumn(n), statistics.genericGetMin().asInstanceOf[Integer]), - FilterApi.ltEq(intColumn(n), statistics.genericGetMax().asInstanceOf[Integer])) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, INT64, _) if pushDownDecimal => - (n: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(_.asInstanceOf[JBigDecimal]).map(decimalToInt64).foreach(statistics.updateStats(_)) - FilterApi.and( - FilterApi.gtEq(longColumn(n), statistics.genericGetMin().asInstanceOf[JLong]), - FilterApi.ltEq(longColumn(n), statistics.genericGetMax().asInstanceOf[JLong])) - - case ParquetSchemaType(_: DecimalLogicalTypeAnnotation, FIXED_LEN_BYTE_ARRAY, length) - if pushDownDecimal => - (path: Array[String], v: Array[Any], statistics: ParquetStatistics[_]) => - v.map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)) - .foreach(statistics.updateStats) - FilterApi.and( - FilterApi.gtEq(binaryColumn(path), statistics.genericGetMin().asInstanceOf[Binary]), - FilterApi.ltEq(binaryColumn(path), statistics.genericGetMax().asInstanceOf[Binary])) - } - - // Returns filters that can be pushed down when reading Parquet files. - def convertibleFilters(filters: Seq[sources.Filter]): Seq[sources.Filter] = { - filters.flatMap(convertibleFiltersHelper(_, canPartialPushDown = true)) - } - - private def convertibleFiltersHelper( - predicate: sources.Filter, - canPartialPushDown: Boolean): Option[sources.Filter] = { - predicate match { - case sources.And(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - (leftResultOptional, rightResultOptional) match { - case (Some(leftResult), Some(rightResult)) => Some(sources.And(leftResult, rightResult)) - case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) - case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) - case _ => None - } - - case sources.Or(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - if (leftResultOptional.isEmpty || rightResultOptional.isEmpty) { - None - } else { - Some(sources.Or(leftResultOptional.get, rightResultOptional.get)) - } - case sources.Not(pred) => - val resultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) - resultOptional.map(sources.Not) - - case other => - if (createFilter(other).isDefined) { - Some(other) - } else { - None - } - } - } - - /** - * Converts data sources filters to Parquet filter predicates. - */ - def createFilter(predicate: sources.Filter): Option[FilterPredicate] = { - createFilterHelper(predicate, canPartialPushDownConjuncts = true) - } - - // Parquet's type in the given file should be matched to the value's type - // in the pushed filter in order to push down the filter to Parquet. - private def valueCanMakeFilterOn(name: String, value: Any): Boolean = { - value == null || (nameToParquetField(name).fieldType match { - case ParquetBooleanType => value.isInstanceOf[JBoolean] - case ParquetByteType | ParquetShortType | ParquetIntegerType => - value match { - // Byte/Short/Int are all stored as INT32 in Parquet so filters are built using type - // Int. We don't create a filter if the value would overflow. - case _: JByte | _: JShort | _: Integer => true - case v: JLong => v.longValue() >= Int.MinValue && v.longValue() <= Int.MaxValue - case _ => false - } - case ParquetLongType => value.isInstanceOf[JLong] - case ParquetFloatType => value.isInstanceOf[JFloat] - case ParquetDoubleType => value.isInstanceOf[JDouble] - case ParquetStringType => value.isInstanceOf[String] - case ParquetBinaryType => value.isInstanceOf[Array[Byte]] - case ParquetDateType => - value.isInstanceOf[Date] || value.isInstanceOf[LocalDate] - case ParquetTimestampMicrosType | ParquetTimestampMillisType => - value.isInstanceOf[Timestamp] || value.isInstanceOf[Instant] - case ParquetSchemaType(decimalType: DecimalLogicalTypeAnnotation, INT32, _) => - isDecimalMatched(value, decimalType) - case ParquetSchemaType(decimalType: DecimalLogicalTypeAnnotation, INT64, _) => - isDecimalMatched(value, decimalType) - case ParquetSchemaType( - decimalType: DecimalLogicalTypeAnnotation, - FIXED_LEN_BYTE_ARRAY, - _) => - isDecimalMatched(value, decimalType) - case _ => false - }) - } - - // Decimal type must make sure that filter value's scale matched the file. - // If doesn't matched, which would cause data corruption. - private def isDecimalMatched( - value: Any, - decimalLogicalType: DecimalLogicalTypeAnnotation): Boolean = value match { - case decimal: JBigDecimal => - decimal.scale == decimalLogicalType.getScale - case _ => false - } - - private def canMakeFilterOn(name: String, value: Any): Boolean = { - nameToParquetField.contains(name) && valueCanMakeFilterOn(name, value) - } - - /** - * @param predicate - * the input filter predicates. Not all the predicates can be pushed down. - * @param canPartialPushDownConjuncts - * whether a subset of conjuncts of predicates can be pushed down safely. Pushing ONLY one - * side of AND down is safe to do at the top level or none of its ancestors is NOT and OR. - * @return - * the Parquet-native filter predicates that are eligible for pushdown. - */ - private def createFilterHelper( - predicate: sources.Filter, - canPartialPushDownConjuncts: Boolean): Option[FilterPredicate] = { - // NOTE: - // - // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, - // which can be casted to `false` implicitly. Please refer to the `eval` method of these - // operators and the `PruneFilters` rule for details. - - // Hyukjin: - // I added [[EqualNullSafe]] with [[org.apache.parquet.filter2.predicate.Operators.Eq]]. - // So, it performs equality comparison identically when given [[sources.Filter]] is [[EqualTo]]. - // The reason why I did this is, that the actual Parquet filter checks null-safe equality - // comparison. - // So I added this and maybe [[EqualTo]] should be changed. It still seems fine though, because - // physical planning does not set `NULL` to [[EqualTo]] but changes it to [[IsNull]] and etc. - // Probably I missed something and obviously this should be changed. - - predicate match { - case sources.IsNull(name) if canMakeFilterOn(name, null) => - makeEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, null)) - case sources.IsNotNull(name) if canMakeFilterOn(name, null) => - makeNotEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, null)) - - case sources.EqualTo(name, value) if canMakeFilterOn(name, value) => - makeEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - case sources.Not(sources.EqualTo(name, value)) if canMakeFilterOn(name, value) => - makeNotEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - - case sources.EqualNullSafe(name, value) if canMakeFilterOn(name, value) => - makeEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - case sources.Not(sources.EqualNullSafe(name, value)) if canMakeFilterOn(name, value) => - makeNotEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - - case sources.LessThan(name, value) if (value != null) && canMakeFilterOn(name, value) => - makeLt - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - case sources.LessThanOrEqual(name, value) - if (value != null) && canMakeFilterOn(name, value) => - makeLtEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - - case sources.GreaterThan(name, value) if (value != null) && canMakeFilterOn(name, value) => - makeGt - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - case sources.GreaterThanOrEqual(name, value) - if (value != null) && canMakeFilterOn(name, value) => - makeGtEq - .lift(nameToParquetField(name).fieldType) - .map(_(nameToParquetField(name).fieldNames, value)) - - case sources.And(lhs, rhs) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val lhsFilterOption = - createFilterHelper(lhs, canPartialPushDownConjuncts) - val rhsFilterOption = - createFilterHelper(rhs, canPartialPushDownConjuncts) - - (lhsFilterOption, rhsFilterOption) match { - case (Some(lhsFilter), Some(rhsFilter)) => Some(FilterApi.and(lhsFilter, rhsFilter)) - case (Some(lhsFilter), None) if canPartialPushDownConjuncts => Some(lhsFilter) - case (None, Some(rhsFilter)) if canPartialPushDownConjuncts => Some(rhsFilter) - case _ => None - } - - case sources.Or(lhs, rhs) => - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - for { - lhsFilter <- createFilterHelper(lhs, canPartialPushDownConjuncts) - rhsFilter <- createFilterHelper(rhs, canPartialPushDownConjuncts) - } yield FilterApi.or(lhsFilter, rhsFilter) - - case sources.Not(pred) => - createFilterHelper(pred, canPartialPushDownConjuncts = false) - .map(FilterApi.not) - - case sources.In(name, values) - if pushDownInFilterThreshold > 0 && values.nonEmpty && - canMakeFilterOn(name, values.head) => - val fieldType = nameToParquetField(name).fieldType - val fieldNames = nameToParquetField(name).fieldNames - if (values.length <= pushDownInFilterThreshold) { - values.distinct - .flatMap { v => - makeEq.lift(fieldType).map(_(fieldNames, v)) - } - .reduceLeftOption(FilterApi.or) - } else if (canPartialPushDownConjuncts) { - val primitiveType = schema.getColumnDescription(fieldNames).getPrimitiveType - val statistics: ParquetStatistics[_] = ParquetStatistics.createStats(primitiveType) - if (values.contains(null)) { - Seq( - makeEq.lift(fieldType).map(_(fieldNames, null)), - makeInPredicate - .lift(fieldType) - .map(_(fieldNames, values.filter(_ != null), statistics))).flatten - .reduceLeftOption(FilterApi.or) - } else { - makeInPredicate.lift(fieldType).map(_(fieldNames, values, statistics)) - } - } else { - None - } - - case sources.StringStartsWith(name, prefix) - if pushDownStringPredicate && canMakeFilterOn(name, prefix) => - Option(prefix).map { v => - FilterApi.userDefined( - binaryColumn(nameToParquetField(name).fieldNames), - new UserDefinedPredicate[Binary] with Serializable { - private val strToBinary = Binary.fromReusedByteArray(v.getBytes) - private val size = strToBinary.length - - override def canDrop(statistics: Statistics[Binary]): Boolean = { - val comparator = PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR - val max = statistics.getMax - val min = statistics.getMin - comparator.compare(max.slice(0, math.min(size, max.length)), strToBinary) < 0 || - comparator.compare(min.slice(0, math.min(size, min.length)), strToBinary) > 0 - } - - override def inverseCanDrop(statistics: Statistics[Binary]): Boolean = { - val comparator = PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR - val max = statistics.getMax - val min = statistics.getMin - comparator.compare(max.slice(0, math.min(size, max.length)), strToBinary) == 0 && - comparator.compare(min.slice(0, math.min(size, min.length)), strToBinary) == 0 - } - - override def keep(value: Binary): Boolean = { - value != null && UTF8String - .fromBytes(value.getBytes) - .startsWith(UTF8String.fromBytes(strToBinary.getBytes)) - } - }) - } - - case sources.StringEndsWith(name, suffix) - if pushDownStringPredicate && canMakeFilterOn(name, suffix) => - Option(suffix).map { v => - FilterApi.userDefined( - binaryColumn(nameToParquetField(name).fieldNames), - new UserDefinedPredicate[Binary] with Serializable { - private val suffixStr = UTF8String.fromString(v) - override def canDrop(statistics: Statistics[Binary]): Boolean = false - override def inverseCanDrop(statistics: Statistics[Binary]): Boolean = false - override def keep(value: Binary): Boolean = { - value != null && UTF8String.fromBytes(value.getBytes).endsWith(suffixStr) - } - }) - } - - case sources.StringContains(name, value) - if pushDownStringPredicate && canMakeFilterOn(name, value) => - Option(value).map { v => - FilterApi.userDefined( - binaryColumn(nameToParquetField(name).fieldNames), - new UserDefinedPredicate[Binary] with Serializable { - private val subStr = UTF8String.fromString(v) - override def canDrop(statistics: Statistics[Binary]): Boolean = false - override def inverseCanDrop(statistics: Statistics[Binary]): Boolean = false - override def keep(value: Binary): Boolean = { - value != null && UTF8String.fromBytes(value.getBytes).contains(subStr) - } - }) - } - - case _ => None - } - } - - def createNativeFilters(predicates: Seq[sources.Filter]): Option[Array[Byte]] = { - predicates.reduceOption(sources.And).flatMap(createNativeFilter).map { expr => - val size = expr.getSerializedSize - val bytes = new Array[Byte](size) - val codedOutput = CodedOutputStream.newInstance(bytes) - expr.writeTo(codedOutput) - codedOutput.checkNoSpaceLeft() - bytes - } - } - - private def createNativeFilter(predicate: sources.Filter): Option[ExprOuterClass.Expr] = { - def nameUnaryExpr(name: String)( - f: (ExprOuterClass.Expr.Builder, ExprOuterClass.UnaryExpr) => ExprOuterClass.Expr.Builder) - : Option[ExprOuterClass.Expr] = { - createNameExpr(name, dataSchema).map { case (_, childExpr) => - createUnaryExpr(childExpr, f) - } - } - - def nameValueBinaryExpr(name: String, value: Any)( - f: ( - ExprOuterClass.Expr.Builder, - ExprOuterClass.BinaryExpr) => ExprOuterClass.Expr.Builder) - : Option[ExprOuterClass.Expr] = { - createNameExpr(name, dataSchema).flatMap { case (dataType, childExpr) => - createValueExpr(value, dataType).map(createBinaryExpr(childExpr, _, f)) - } - } - - predicate match { - case sources.IsNull(name) if canMakeFilterOn(name, null) => - nameUnaryExpr(name) { (builder, unaryExpr) => - builder.setIsNull(unaryExpr) - } - case sources.IsNotNull(name) if canMakeFilterOn(name, null) => - nameUnaryExpr(name) { (builder, unaryExpr) => - builder.setIsNotNull(unaryExpr) - } - - case sources.EqualTo(name, value) if canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setEq(binaryExpr) - } - - case sources.Not(sources.EqualTo(name, value)) if canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setNeq(binaryExpr) - } - - case sources.EqualNullSafe(name, value) if canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setEqNullSafe(binaryExpr) - } - - case sources.Not(sources.EqualNullSafe(name, value)) if canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setNeqNullSafe(binaryExpr) - } - - case sources.LessThan(name, value) if (value != null) && canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setLt(binaryExpr) - } - - case sources.LessThanOrEqual(name, value) - if (value != null) && canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setLtEq(binaryExpr) - } - - case sources.GreaterThan(name, value) if (value != null) && canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setGt(binaryExpr) - } - - case sources.GreaterThanOrEqual(name, value) - if (value != null) && canMakeFilterOn(name, value) => - nameValueBinaryExpr(name, value) { (builder, binaryExpr) => - builder.setGtEq(binaryExpr) - } - - case sources.And(lhs, rhs) => - (createNativeFilter(lhs), createNativeFilter(rhs)) match { - case (Some(leftExpr), Some(rightExpr)) => - Some( - createBinaryExpr( - leftExpr, - rightExpr, - (builder, binaryExpr) => builder.setAnd(binaryExpr))) - case _ => None - } - - case sources.Or(lhs, rhs) => - (createNativeFilter(lhs), createNativeFilter(rhs)) match { - case (Some(leftExpr), Some(rightExpr)) => - Some( - createBinaryExpr( - leftExpr, - rightExpr, - (builder, binaryExpr) => builder.setOr(binaryExpr))) - case _ => None - } - - case sources.Not(pred) => - val childExpr = createNativeFilter(pred) - childExpr.map { expr => - createUnaryExpr(expr, (builder, unaryExpr) => builder.setNot(unaryExpr)) - } - - case sources.In(name, values) - if pushDownInFilterThreshold > 0 && values.nonEmpty && - canMakeFilterOn(name, values.head) => - createNameExpr(name, dataSchema).flatMap { case (dataType, nameExpr) => - val valueExprs = values.flatMap(createValueExpr(_, dataType)) - if (valueExprs.length != values.length) { - None - } else { - val builder = ExprOuterClass.In.newBuilder() - builder.setInValue(nameExpr) - builder.addAllLists(valueExprs.toSeq.asJava) - builder.setNegated(false) - Some( - ExprOuterClass.Expr - .newBuilder() - .setIn(builder) - .build()) - } - } - - case sources.StringStartsWith(attribute, prefix) - if pushDownStringPredicate && canMakeFilterOn(attribute, prefix) => - val attributeExpr = createNameExpr(attribute, dataSchema) - val prefixExpr = attributeExpr.flatMap { case (dataType, _) => - createValueExpr(prefix, dataType) - } - scalarFunctionExprToProto("starts_with", Some(attributeExpr.get._2), prefixExpr) - - case sources.StringEndsWith(attribute, suffix) - if pushDownStringPredicate && canMakeFilterOn(attribute, suffix) => - val attributeExpr = createNameExpr(attribute, dataSchema) - val suffixExpr = attributeExpr.flatMap { case (dataType, _) => - createValueExpr(suffix, dataType) - } - scalarFunctionExprToProto("ends_with", Some(attributeExpr.get._2), suffixExpr) - - case sources.StringContains(attribute, value) - if pushDownStringPredicate && canMakeFilterOn(attribute, value) => - val attributeExpr = createNameExpr(attribute, dataSchema) - val valueExpr = attributeExpr.flatMap { case (dataType, _) => - createValueExpr(value, dataType) - } - scalarFunctionExprToProto("contains", Some(attributeExpr.get._2), valueExpr) - - case _ => None - } - } -} diff --git a/spark/src/main/scala/org/apache/comet/parquet/SourceFilterSerde.scala b/spark/src/main/scala/org/apache/comet/parquet/SourceFilterSerde.scala deleted file mode 100644 index 9fdf38dc89..0000000000 --- a/spark/src/main/scala/org/apache/comet/parquet/SourceFilterSerde.scala +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.parquet - -import java.math.{BigDecimal => JavaBigDecimal} -import java.sql.{Date, Timestamp} -import java.time.{Instant, LocalDate, LocalDateTime} - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.types._ - -import org.apache.comet.serde.ExprOuterClass -import org.apache.comet.serde.ExprOuterClass.Expr -import org.apache.comet.serde.LiteralOuterClass -import org.apache.comet.serde.QueryPlanSerde.serializeDataType - -object SourceFilterSerde extends Logging { - - def createNameExpr( - name: String, - schema: StructType): Option[(org.apache.spark.sql.types.DataType, ExprOuterClass.Expr)] = { - val filedWithIndex = schema.fields.zipWithIndex.find { case (field, _) => - field.name == name - } - if (filedWithIndex.isDefined) { - val (field, index) = filedWithIndex.get - val dataType = serializeDataType(field.dataType) - if (dataType.isDefined) { - val boundExpr = ExprOuterClass.BoundReference - .newBuilder() - .setIndex(index) - .setDatatype(dataType.get) - .build() - Some( - ( - field.dataType, - ExprOuterClass.Expr - .newBuilder() - .setBound(boundExpr) - .build())) - } else { - None - } - } else { - None - } - - } - - /** - * create a literal value native expression for source filter value, the value is a scala value - */ - def createValueExpr( - value: Any, - dataType: org.apache.spark.sql.types.DataType): Option[ExprOuterClass.Expr] = { - val exprBuilder = LiteralOuterClass.Literal.newBuilder() - var valueIsSet = true - if (value == null) { - exprBuilder.setIsNull(true) - } else { - exprBuilder.setIsNull(false) - // value is a scala value, not a catalyst value - // refer to org.apache.spark.sql.catalyst.CatalystTypeConverters.CatalystTypeConverter#toScala - dataType match { - case _: BooleanType => exprBuilder.setBoolVal(value.asInstanceOf[Boolean]) - case _: ByteType => exprBuilder.setByteVal(value.asInstanceOf[Byte]) - case _: ShortType => exprBuilder.setShortVal(value.asInstanceOf[Short]) - case _: IntegerType => exprBuilder.setIntVal(value.asInstanceOf[Int]) - case _: LongType => exprBuilder.setLongVal(value.asInstanceOf[Long]) - case _: FloatType => exprBuilder.setFloatVal(value.asInstanceOf[Float]) - case _: DoubleType => exprBuilder.setDoubleVal(value.asInstanceOf[Double]) - case _: StringType => exprBuilder.setStringVal(value.asInstanceOf[String]) - case _: TimestampType => - value match { - case v: Timestamp => exprBuilder.setLongVal(DateTimeUtils.fromJavaTimestamp(v)) - case v: Instant => exprBuilder.setLongVal(DateTimeUtils.instantToMicros(v)) - case v: Long => exprBuilder.setLongVal(v) - case _ => - valueIsSet = false - logWarning(s"Unexpected timestamp type '${value.getClass}' for value '$value'") - } - case _: TimestampNTZType => - value match { - case v: LocalDateTime => - exprBuilder.setLongVal(DateTimeUtils.localDateTimeToMicros(v)) - case v: Long => exprBuilder.setLongVal(v) - case _ => - valueIsSet = false - logWarning(s"Unexpected timestamp type '${value.getClass}' for value' $value'") - } - case _: DecimalType => - // Pass decimal literal as bytes. - val unscaled = value.asInstanceOf[JavaBigDecimal].unscaledValue - exprBuilder.setDecimalVal(com.google.protobuf.ByteString.copyFrom(unscaled.toByteArray)) - case _: BinaryType => - val byteStr = - com.google.protobuf.ByteString.copyFrom(value.asInstanceOf[Array[Byte]]) - exprBuilder.setBytesVal(byteStr) - case _: DateType => - value match { - case v: LocalDate => exprBuilder.setIntVal(DateTimeUtils.localDateToDays(v)) - case v: Date => exprBuilder.setIntVal(DateTimeUtils.fromJavaDate(v)) - case v: Int => exprBuilder.setIntVal(v) - case _ => - valueIsSet = false - logWarning(s"Unexpected date type '${value.getClass}' for value '$value'") - } - case dt => - valueIsSet = false - logWarning(s"Unexpected data type '$dt' for literal value '$value'") - } - } - - val dt = serializeDataType(dataType) - - if (valueIsSet && dt.isDefined) { - exprBuilder.setDatatype(dt.get) - - Some( - ExprOuterClass.Expr - .newBuilder() - .setLiteral(exprBuilder) - .build()) - } else { - None - } - } - - def createUnaryExpr( - childExpr: Expr, - f: (ExprOuterClass.Expr.Builder, ExprOuterClass.UnaryExpr) => ExprOuterClass.Expr.Builder) - : ExprOuterClass.Expr = { - // create the generic UnaryExpr message - val inner = ExprOuterClass.UnaryExpr - .newBuilder() - .setChild(childExpr) - .build() - f( - ExprOuterClass.Expr - .newBuilder(), - inner).build() - } - - def createBinaryExpr( - leftExpr: Expr, - rightExpr: Expr, - f: (ExprOuterClass.Expr.Builder, ExprOuterClass.BinaryExpr) => ExprOuterClass.Expr.Builder) - : ExprOuterClass.Expr = { - // create the generic BinaryExpr message - val inner = ExprOuterClass.BinaryExpr - .newBuilder() - .setLeft(leftExpr) - .setRight(rightExpr) - .build() - f( - ExprOuterClass.Expr - .newBuilder(), - inner).build() - } - -} 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..e7c92ccd4f 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -23,15 +23,17 @@ import scala.collection.mutable.ListBuffer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Divide, DoubleLiteral, EqualNullSafe, EqualTo, Expression, FloatLiteral, GreaterThan, GreaterThanOrEqual, KnownFloatingPointNormalized, LessThan, LessThanOrEqual, NamedExpression, Remainder} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateMode, Final, Partial} import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.catalyst.util.sideBySide import org.apache.spark.sql.comet._ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometNativeShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec} +import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec} import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat @@ -57,6 +59,14 @@ import org.apache.comet.shims.{ShimCometStreaming, ShimSubqueryBroadcast} object CometExecRule { + /** + * Tag applied to Partial-mode aggregate operators that must NOT be converted to Comet because + * the corresponding Final-mode aggregate cannot be converted, and the aggregate functions have + * incompatible intermediate buffer formats between Spark and Comet. + */ + val COMET_UNSAFE_PARTIAL: TreeNodeTag[String] = + TreeNodeTag[String]("comet.unsafePartialAgg") + /** * Fully native operators. */ @@ -257,9 +267,10 @@ case class CometExecRule(session: SparkSession) // spotless:on private def transform(plan: SparkPlan): SparkPlan = { def convertNode(op: SparkPlan): SparkPlan = op match { - // Fully native scan for V1 - case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => - convertToComet(scan, CometNativeScan).getOrElse(scan) + // Fully native scan for V1. CometScanExec must always convert to a native scan; the JVM + // fallback path has been removed. If conversion fails, fall back to the original Spark scan. + case scan: CometScanExec => + convertToComet(scan, CometNativeScan).getOrElse(scan.wrapped) // Fully native Iceberg scan for V2 (iceberg-rust path) // Only handle scans with native metadata; other scans fall through to isCometScan @@ -567,6 +578,12 @@ case class CometExecRule(session: SparkSession) normalizedPlan } + // Tag Partial aggregates that must not be converted to Comet because the + // corresponding Final aggregate cannot be converted and the intermediate buffer + // formats are incompatible. This runs before transform() so the tags are checked + // during the bottom-up conversion. Tags persist through AQE stage creation. + tagUnsafePartialAggregates(planWithJoinRewritten) + var newPlan = transform(planWithJoinRewritten) // if the plan cannot be run fully natively then explain why (when appropriate @@ -787,4 +804,129 @@ case class CometExecRule(session: SparkSession) } } + /** + * Walk the plan to find Final-mode aggregates that cannot be converted to Comet. For each such + * Final, if the aggregate functions have incompatible intermediate buffer formats, tag the + * corresponding Partial-mode aggregate so it will also be skipped during conversion. + * + * This prevents the crash described in issue #1389 where a Comet Partial produces intermediate + * data in a format that the Spark Final cannot interpret. + */ + private def tagUnsafePartialAggregates(plan: SparkPlan): Unit = { + plan.foreach { + case agg: BaseAggregateExec => + // Only consider single-mode Final aggregates. Multi-mode Finals come from Spark's + // distinct-aggregate rewrite, where the Comet partial (if any) feeds into a Spark + // PartialMerge rather than directly into a Final, which is a different code path + // than the Comet-Partial → Spark-Final crash scenario from issue #1389. + val modes = agg.aggregateExpressions.map(_.mode).distinct + if (modes == Seq(Final) && + !QueryPlanSerde.allAggsSupportMixedExecution(agg.aggregateExpressions) && + !canAggregateBeConverted(agg, Final)) { + findPartialAggInPlan(agg.child).foreach { partial => + // Only tag if the Partial would otherwise have been converted. If the Partial + // itself cannot be converted (e.g. the aggregate function is incompatible for the + // input type), there is no buffer-format mismatch to guard against, and tagging + // would mask the natural, more specific fallback reason. + if (canAggregateBeConverted(partial, Partial)) { + partial.setTagValue( + CometExecRule.COMET_UNSAFE_PARTIAL, + "Partial aggregate disabled: corresponding final aggregate " + + "cannot be converted to Comet and intermediate buffer formats are incompatible") + } + } + } + case _ => + } + } + + /** + * Conservative check for whether an aggregate could be converted to Comet. Checks operator + * enablement, grouping expressions, aggregate expressions, and result expressions. + * Intentionally skips the sparkFinalMode / child-native checks since those depend on + * transformation state. + * + * WARNING: this intentionally mirrors the predicate checks in `CometBaseAggregate.doConvert` + * (operators.scala). Any change to the convertibility rules there must be reflected here or + * this tagging pass will drift and either crash (missed tag) or over-disable (spurious tag). A + * shared predicate helper would be preferable. + */ + private def canAggregateBeConverted( + agg: BaseAggregateExec, + expectedMode: AggregateMode): Boolean = { + val handler = allExecs.get(agg.getClass) + if (handler.isEmpty) return false + val serde = handler.get.asInstanceOf[CometOperatorSerde[SparkPlan]] + if (!isOperatorEnabled(serde, agg.asInstanceOf[SparkPlan])) return false + + // ObjectHashAggregate has an extra shuffle-enabled guard in its convert method + agg match { + case _: ObjectHashAggregateExec if !isCometShuffleEnabled(agg.conf) => return false + case _ => + } + + val aggregateExpressions = agg.aggregateExpressions + val groupingExpressions = agg.groupingExpressions + + if (groupingExpressions.isEmpty && aggregateExpressions.isEmpty) return false + + if (groupingExpressions.exists(e => QueryPlanSerde.containsMapType(e.dataType))) return false + + if (!groupingExpressions.forall(e => + QueryPlanSerde.exprToProto(e, agg.child.output).isDefined)) { + return false + } + + if (aggregateExpressions.isEmpty) { + // Result expressions always checked when there are no aggregate expressions + val attributes = + groupingExpressions.map(_.toAttribute) ++ agg.aggregateAttributes + return agg.resultExpressions.forall(e => + QueryPlanSerde.exprToProto(e, attributes).isDefined) + } + + val modes = aggregateExpressions.map(_.mode).distinct + if (modes.size != 1 || modes.head != expectedMode) return false + + // In Final mode, exprToProto resolves against the child's output; in Partial/non-Final mode + // it must bind to input attributes. This mirrors the `binding` calculation in + // `CometBaseAggregate.doConvert`. + val binding = expectedMode != Final + if (!aggregateExpressions.forall(e => + QueryPlanSerde.aggExprToProto(e, agg.child.output, binding, agg.conf).isDefined)) { + return false + } + + // doConvert only checks resultExpressions in Final mode when aggregate expressions exist + // (Partial emits the buffer directly). Mirror that here to avoid false negatives. + if (expectedMode == Final) { + val attributes = + groupingExpressions.map(_.toAttribute) ++ agg.aggregateAttributes + agg.resultExpressions.forall(e => QueryPlanSerde.exprToProto(e, attributes).isDefined) + } else { + true + } + } + + /** + * Look for a Partial-mode aggregate that feeds directly into the given plan (the child of a + * Final). Walks through exchanges and AQE stages only, stopping at anything else including + * other aggregate stages. This avoids tagging unrelated Partials found deeper in the plan (e.g. + * the non-distinct Partial in a distinct-aggregate rewrite, which is separated from the Final + * by intermediate PartialMerge stages). Requires `aggregateExpressions.nonEmpty` so that + * group-by-only dedup stages are not mistaken for the partial we want to tag. + */ + private def findPartialAggInPlan(plan: SparkPlan): Option[BaseAggregateExec] = plan match { + case agg: BaseAggregateExec + if agg.aggregateExpressions.nonEmpty && + agg.aggregateExpressions.forall(e => e.mode == Partial) => + Some(agg) + case a: AQEShuffleReadExec => findPartialAggInPlan(a.child) + case s: ShuffleQueryStageExec => findPartialAggInPlan(s.plan) + case e: ShuffleExchangeExec => findPartialAggInPlan(e.child) + case other => + logDebug(s"findPartialAggInPlan: stopping at ${other.nodeName}; not a known passthrough") + None + } + } 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..fb950db861 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -40,14 +40,13 @@ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} +import org.apache.comet.{CometConf, DataTypeSupport} import org.apache.comet.CometConf._ import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, isSpark35Plus, withInfo, withInfos} import org.apache.comet.DataTypeSupport.isComplexType import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} -import org.apache.comet.parquet.Native import org.apache.comet.serde.operator.{CometIcebergNativeScan, CometNativeScan} import org.apache.comet.shims.{CometTypeShim, ShimCometStreaming, ShimFileFormat, ShimSubqueryBroadcast} @@ -183,12 +182,7 @@ case class CometScanRule(session: SparkSession) return scanExec } - COMET_NATIVE_SCAN_IMPL.get() match { - case SCAN_AUTO | SCAN_NATIVE_DATAFUSION => - nativeDataFusionScan(plan, session, scanExec, r, hadoopConf).getOrElse(scanExec) - case SCAN_NATIVE_ICEBERG_COMPAT => - nativeIcebergCompatScan(session, scanExec, r, hadoopConf).getOrElse(scanExec) - } + nativeDataFusionScan(plan, session, scanExec, r, hadoopConf).getOrElse(scanExec) case _ => withInfo(scanExec, s"Unsupported relation ${scanExec.relation}") @@ -202,16 +196,27 @@ case class CometScanRule(session: SparkSession) r: HadoopFsRelation, hadoopConf: Configuration): Option[SparkPlan] = { if (!COMET_EXEC_ENABLED.get()) { + withInfo(scanExec, s"Native Parquet scan requires ${COMET_EXEC_ENABLED.key} to be enabled") + return None + } + // Disabling the vectorized reader opts into parquet-mr's permissive behavior + // (silent overflow / null-on-narrowing). Comet has no parquet-mr-equivalent + // backend, so by default fall back to Spark. Users can opt in to letting Comet + // replace the scan via COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER. + if (!conf.parquetVectorizedReaderEnabled && + !COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER.get()) { withInfo( scanExec, - s"$SCAN_NATIVE_DATAFUSION scan requires ${COMET_EXEC_ENABLED.key} to be enabled") + "Native Parquet scan is incompatible with " + + s"${SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key}=false; set " + + s"${COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER.key}=true to opt in") return None } if (!CometNativeScan.isSupported(scanExec)) { return None } if (encryptionEnabled(hadoopConf) && !isEncryptionConfigSupported(hadoopConf)) { - withInfo(scanExec, s"$SCAN_NATIVE_DATAFUSION does not support encryption") + withInfo(scanExec, "Native Parquet scan does not support encryption") return None } if (scanExec.fileConstantMetadataColumns.nonEmpty) { @@ -236,25 +241,10 @@ case class CometScanRule(session: SparkSession) withInfo(scanExec, "Native DataFusion scan does not support row index generation") return None } - if (!isSchemaSupported(scanExec, SCAN_NATIVE_DATAFUSION, r)) { + if (!isSchemaSupported(scanExec, r)) { return None } - Some(CometScanExec(scanExec, session, SCAN_NATIVE_DATAFUSION)) - } - - private def nativeIcebergCompatScan( - session: SparkSession, - scanExec: FileSourceScanExec, - r: HadoopFsRelation, - hadoopConf: Configuration): Option[SparkPlan] = { - if (encryptionEnabled(hadoopConf) && !isEncryptionConfigSupported(hadoopConf)) { - withInfo(scanExec, s"$SCAN_NATIVE_ICEBERG_COMPAT does not support encryption") - return None - } - if (!isSchemaSupported(scanExec, SCAN_NATIVE_ICEBERG_COMPAT, r)) { - return None - } - Some(CometScanExec(scanExec, session, SCAN_NATIVE_ICEBERG_COMPAT)) + Some(CometScanExec(scanExec, session)) } private def transformV2Scan(scanExec: BatchScanExec): SparkPlan = { @@ -320,7 +310,7 @@ case class CometScanRule(session: SparkSession) return withInfos(scanExec, fallbackReasons.toSet) } - val typeChecker = CometScanTypeChecker(SCAN_NATIVE_DATAFUSION) + val typeChecker = CometScanTypeChecker() val schemaSupported = typeChecker.isSchemaSupported(scanExec.scan.readSchema(), fallbackReasons) @@ -677,19 +667,15 @@ case class CometScanRule(session: SparkSession) case _ => false } - private def isSchemaSupported( - scanExec: FileSourceScanExec, - scanImpl: String, - r: HadoopFsRelation): Boolean = { + private def isSchemaSupported(scanExec: FileSourceScanExec, r: HadoopFsRelation): Boolean = { val fallbackReasons = new ListBuffer[String]() - val typeChecker = CometScanTypeChecker(scanImpl) + val typeChecker = CometScanTypeChecker() val schemaSupported = typeChecker.isSchemaSupported(scanExec.requiredSchema, fallbackReasons) if (!schemaSupported) { withInfo( scanExec, - s"Unsupported schema ${scanExec.requiredSchema} " + - s"for $scanImpl: ${fallbackReasons.mkString(", ")}") + s"Unsupported schema ${scanExec.requiredSchema}: ${fallbackReasons.mkString(", ")}") return false } val partitionSchemaSupported = @@ -697,19 +683,15 @@ case class CometScanRule(session: SparkSession) if (!partitionSchemaSupported) { withInfo( scanExec, - s"Unsupported partitioning schema ${scanExec.requiredSchema} " + - s"for $scanImpl: ${fallbackReasons - .mkString(", ")}") + s"Unsupported partitioning schema ${scanExec.requiredSchema}: " + + fallbackReasons.mkString(", ")) return false } true } } -case class CometScanTypeChecker(scanImpl: String) extends DataTypeSupport with CometTypeShim { - - // this class is intended to be used with a specific scan impl - assert(scanImpl != CometConf.SCAN_AUTO) +case class CometScanTypeChecker() extends DataTypeSupport with CometTypeShim { override def isTypeSupported( dt: DataType, @@ -717,8 +699,8 @@ case class CometScanTypeChecker(scanImpl: String) extends DataTypeSupport with C fallbackReasons: ListBuffer[String]): Boolean = { dt match { case ShortType if CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.get() => - fallbackReasons += s"$scanImpl scan may not handle unsigned UINT_8 correctly for $dt. " + - s"Set ${CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key}=false to allow " + + fallbackReasons += "Native Parquet scan may not handle unsigned UINT_8 correctly for " + + s"$dt. Set ${CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key}=false to allow " + "native execution if your data does not contain unsigned small integers. " + CometConf.COMPAT_GUIDE false @@ -729,9 +711,9 @@ case class CometScanTypeChecker(scanImpl: String) extends DataTypeSupport with C case s: StructType if isVariantStruct(s) => // Spark 4.0's PushVariantIntoScan rewrites a VariantType column into a struct of typed // fields plus per-field VariantMetadata, expecting the scan to honor Parquet variant - // shredding semantics. Comet's native scans don't, so fall back to Spark. + // shredding semantics. Comet's native scan does not, so fall back to Spark. fallbackReasons += - s"Unsupported $name of type VariantType (shredded; not supported by $scanImpl scan)" + s"Unsupported $name of type VariantType (shredded; not supported by native scan)" false case s: StructType if s.fields.isEmpty => false @@ -752,62 +734,6 @@ object CometScanRule extends Logging { val SKIP_COMET_SCAN_TAG: org.apache.spark.sql.catalyst.trees.TreeNodeTag[Unit] = org.apache.spark.sql.catalyst.trees.TreeNodeTag[Unit]("comet.skipCometScan") - /** - * Validating object store configs can cause requests to be made to S3 APIs (such as when - * resolving the region for a bucket). We use a cache to reduce the number of S3 calls. - * - * The key is the config map converted to a string. The value is the reason that the config is - * not valid, or None if the config is valid. - */ - val configValidityMap = new mutable.HashMap[String, Option[String]]() - - /** - * We do not expect to see a large number of unique configs within the lifetime of a Spark - * session, but we reset the cache once it reaches a fixed size to prevent it growing - * indefinitely. - */ - val configValidityMapMaxSize = 1024 - - def validateObjectStoreConfig( - filePath: String, - hadoopConf: Configuration, - fallbackReasons: mutable.ListBuffer[String]): Unit = { - val objectStoreConfigMap = - NativeConfig.extractObjectStoreOptions(hadoopConf, URI.create(filePath)) - - val cacheKey = objectStoreConfigMap - .map { case (k, v) => - s"$k=$v" - } - .toList - .sorted - .mkString("\n") - - if (configValidityMap.size >= configValidityMapMaxSize) { - logWarning("Resetting S3 object store validity cache") - configValidityMap.clear() - } - - configValidityMap.get(cacheKey) match { - case Some(Some(reason)) => - fallbackReasons += reason - case Some(None) => - // previously validated - case _ => - try { - val objectStoreOptions = objectStoreConfigMap.asJava - Native.validateObjectStoreConfig(filePath, objectStoreOptions) - } catch { - case e: CometNativeException => - val reason = "Object store config not supported by " + - s"$SCAN_NATIVE_ICEBERG_COMPAT: ${e.getMessage}" - fallbackReasons += reason - configValidityMap.put(cacheKey, Some(reason)) - } - } - - } - /** * Single-pass validation of Iceberg FileScanTasks. * diff --git a/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala b/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala index 316510400b..9a83152168 100644 --- a/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/CometAggregateExpressionSerde.scala @@ -81,6 +81,17 @@ trait CometAggregateExpressionSerde[T <: AggregateFunction] { */ def getSupportLevel(expr: T): SupportLevel = Compatible(None) + /** + * Whether this aggregate's intermediate buffer format is compatible between Spark and Comet, + * making it safe to run the Partial in one engine and the Final in the other. Aggregates with + * simple single-value buffers (MIN, MAX, bitwise) are safe; those with complex or + * differently-encoded buffers (AVG, SUM with decimals, CollectSet, Variance) are not. COUNT is + * intentionally excluded: mixed COUNT partial/final regressed AQE's + * PropagateEmptyRelationAfterAQE pattern (which matches BaseAggregateExec only) and the Spark + * 4.0 count-bug decorrelation for correlated IN subqueries. + */ + def supportsMixedPartialFinal: Boolean = false + /** * Convert a Spark expression into a protocol buffer representation that can be passed into * native code. diff --git a/spark/src/main/scala/org/apache/comet/serde/CometScalaUDF.scala b/spark/src/main/scala/org/apache/comet/serde/CometScalaUDF.scala new file mode 100644 index 0000000000..bf636f7221 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/CometScalaUDF.scala @@ -0,0 +1,102 @@ +/* + * 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.serde + +import org.apache.spark.SparkEnv +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSeq, BindReferences, Literal, ScalaUDF} +import org.apache.spark.sql.types.BinaryType + +import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.codegen.CometBatchKernelCodegen +import org.apache.comet.serde.ExprOuterClass.Expr +import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, serializeDataType} +import org.apache.comet.udf.codegen.CometScalaUDFCodegen + +/** + * Routes scalar `ScalaUDF` (Scala and Java UDFs) through the codegen dispatcher. + * `ScalaUDF.doGenCode` emits compilable Java that invokes the user function via + * `ctx.addReferenceObj`; the dispatcher serializes the bound tree, the closure serializer carries + * the function reference across the wire, and the Janino-compiled kernel invokes it in a tight + * batch loop. + * + * Not covered: + * - Aggregate UDFs (`ScalaAggregator`, `TypedImperativeAggregate`, legacy UDAF). + * - Table UDFs and generators. + * - Python / Pandas UDFs. + * - Hive `GenericUDF` / `SimpleUDF`. + * + * Gated by [[CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED]]. When disabled, plans containing a + * `ScalaUDF` fall back to Spark for the enclosing operator. + */ +object CometScalaUDF extends CometExpressionSerde[ScalaUDF] { + + override def convert(expr: ScalaUDF, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { + if (!CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.get()) { + withInfo( + expr, + s"${CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.key}=false; ScalaUDF has no native path " + + "so the plan falls back to Spark") + return None + } + + // Bind against only the AttributeReferences the tree actually reads, so ordinals align with + // the data args we ship. + val attrs = expr.collect { case a: AttributeReference => a }.distinct + val boundExpr = BindReferences.bindReference(expr, AttributeSeq(attrs)) + + // Gate at plan time. Surface the reason via withInfo rather than crashing Janino at execute. + CometBatchKernelCodegen.canHandle(boundExpr) match { + case Some(reason) => + withInfo(expr, reason) + return None + case None => + } + + // Serialize via Spark's closure serializer: respects the task context classloader (so user + // UDF jars are visible) and matches Spark's wire format. The bytes become arg 0 of the + // JvmScalarUdf proto and self-describe the expression so this works in cluster mode without + // executor-side driver registry state. + val serializer = SparkEnv.get.closureSerializer.newInstance() + val buffer = serializer.serialize(boundExpr) + val bytes = new Array[Byte](buffer.remaining()) + buffer.get(bytes) + val exprArg = exprToProtoInternal(Literal(bytes, BinaryType), inputs, binding) + .getOrElse(return None) + + val dataArgs = + attrs.map(a => exprToProtoInternal(a, inputs, binding).getOrElse(return None)) + val returnTypeProto = serializeDataType(expr.dataType).getOrElse(return None) + + val udfBuilder = ExprOuterClass.JvmScalarUdf + .newBuilder() + .setClassName(classOf[CometScalaUDFCodegen].getName) + .addArgs(exprArg) + dataArgs.foreach(udfBuilder.addArgs) + udfBuilder + .setReturnType(returnTypeProto) + .setReturnNullable(expr.nullable) + Some( + ExprOuterClass.Expr + .newBuilder() + .setJvmScalarUdf(udfBuilder.build()) + .build()) + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index dced2e4da8..8d48239e76 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -104,9 +104,11 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[Ceil] -> CometCeil, classOf[Cos] -> CometScalarFunction("cos"), classOf[Cosh] -> CometScalarFunction("cosh"), + classOf[Csc] -> CometScalarFunction("csc"), classOf[Divide] -> CometDivide, classOf[Exp] -> CometScalarFunction("exp"), classOf[Expm1] -> CometScalarFunction("expm1"), + classOf[Factorial] -> CometScalarFunction("factorial"), classOf[Floor] -> CometFloor, classOf[Greatest] -> CometScalarFunction("greatest"), classOf[Hex] -> CometHex, @@ -123,7 +125,9 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[Rand] -> CometRand, classOf[Randn] -> CometRandn, classOf[Remainder] -> CometRemainder, + classOf[Rint] -> CometScalarFunction("rint"), classOf[Round] -> CometRound, + classOf[Sec] -> CometScalarFunction("sec"), classOf[Signum] -> CometScalarFunction("signum"), classOf[Sin] -> CometScalarFunction("sin"), classOf[Sinh] -> CometScalarFunction("sinh"), @@ -215,6 +219,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { private[comet] val temporalExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( + classOf[ConvertTimezone] -> CometConvertTimezone, classOf[DateAdd] -> CometDateAdd, classOf[DateDiff] -> CometDateDiff, classOf[DateFormatClass] -> CometDateFormat, @@ -224,6 +229,8 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[DateSub] -> CometDateSub, classOf[UnixDate] -> CometUnixDate, classOf[FromUnixTime] -> CometFromUnixTime, + classOf[FromUTCTimestamp] -> CometFromUTCTimestamp, + classOf[ToUTCTimestamp] -> CometToUTCTimestamp, classOf[LastDay] -> CometLastDay, classOf[Hour] -> CometHour, classOf[MakeDate] -> CometMakeDate, @@ -243,6 +250,9 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[WeekOfYear] -> CometWeekOfYear, classOf[Quarter] -> CometQuarter) + private[comet] val urlExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( + classOf[ParseUrl] -> CometParseUrl) + private val conversionExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[Cast] -> CometCast) @@ -258,6 +268,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[MakeDecimal] -> CometMakeDecimal, classOf[MonotonicallyIncreasingID] -> CometMonotonicallyIncreasingId, classOf[ScalarSubquery] -> CometScalarSubquery, + classOf[ScalaUDF] -> CometScalaUDF, classOf[SparkPartitionID] -> CometSparkPartitionId, classOf[SortOrder] -> CometSortOrder, classOf[StaticInvoke] -> CometStaticInvoke, @@ -270,7 +281,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { mathExpressions ++ hashExpressions ++ stringExpressions ++ conditionalExpressions ++ mapExpressions ++ predicateExpressions ++ structExpressions ++ bitwiseExpressions ++ miscExpressions ++ arrayExpressions ++ - temporalExpressions ++ conversionExpressions + temporalExpressions ++ conversionExpressions ++ urlExpressions /** * Mapping of Spark aggregate expression class to Comet expression handler. @@ -296,6 +307,24 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { classOf[VariancePop] -> CometVariancePop, classOf[VarianceSamp] -> CometVarianceSamp) + /** + * Returns true if all aggregate expressions in the list have intermediate buffer formats that + * are compatible between Spark and Comet, making it safe to run Partial in one engine and Final + * in the other. + */ + def allAggsSupportMixedExecution(aggExprs: Seq[AggregateExpression]): Boolean = { + aggExprs.forall { aggExpr => + val fn = aggExpr.aggregateFunction + aggrSerdeMap.get(fn.getClass) match { + case Some(handler) => + handler + .asInstanceOf[CometAggregateExpressionSerde[AggregateFunction]] + .supportsMixedPartialFinal + case None => false + } + } + } + // A unique id for each expression. ~used to look up QueryContext during error creation. private val exprIdCounter = new AtomicLong(0) @@ -363,6 +392,8 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | _: DecimalType | _: DateType | _: BooleanType | _: NullType => true + case dt if isTimeType(dt) => + true case s: StructType if allowComplex => s.fields.nonEmpty && s.fields.map(_.dataType).forall(supportedDataType(_, allowComplex)) case a: ArrayType if allowComplex => @@ -373,6 +404,19 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { false } + /** + * Returns true if the given data type is or contains a `MapType` at any nesting level. Arrow's + * row format (used by DataFusion's grouped hash aggregate for composite group keys) does not + * support `Map`, so grouping on any type that transitively contains a map would crash in native + * execution. + */ + def containsMapType(dt: DataType): Boolean = dt match { + case _: MapType => true + case a: ArrayType => containsMapType(a.elementType) + case s: StructType => s.fields.exists(f => containsMapType(f.dataType)) + case _ => false + } + /** * Serializes Spark datatype to protobuf. Note that, a datatype can be serialized by this method * doesn't mean it is supported by Comet native execution, i.e., `supportedDataType` may return @@ -397,6 +441,7 @@ object QueryPlanSerde extends Logging with CometExprShim with CometTypeShim { case _: ArrayType => 14 case _: MapType => 15 case _: StructType => 16 + case dt if isTimeType(dt) => 17 case dt => logWarning(s"Cannot serialize Spark data type: $dt") return None diff --git a/spark/src/main/scala/org/apache/comet/serde/aggregates.scala b/spark/src/main/scala/org/apache/comet/serde/aggregates.scala index bb0466b4eb..e7227ccca9 100644 --- a/spark/src/main/scala/org/apache/comet/serde/aggregates.scala +++ b/spark/src/main/scala/org/apache/comet/serde/aggregates.scala @@ -34,6 +34,8 @@ import org.apache.comet.shims.CometEvalModeUtil object CometMin extends CometAggregateExpressionSerde[Min] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, expr: Min, @@ -81,6 +83,8 @@ object CometMin extends CometAggregateExpressionSerde[Min] { object CometMax extends CometAggregateExpressionSerde[Max] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, expr: Max, @@ -319,6 +323,8 @@ object CometLast extends CometAggregateExpressionSerde[Last] { } object CometBitAndAgg extends CometAggregateExpressionSerde[BitAndAgg] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, bitAnd: BitAndAgg, @@ -353,6 +359,8 @@ object CometBitAndAgg extends CometAggregateExpressionSerde[BitAndAgg] { } object CometBitOrAgg extends CometAggregateExpressionSerde[BitOrAgg] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, bitOr: BitOrAgg, @@ -387,6 +395,8 @@ object CometBitOrAgg extends CometAggregateExpressionSerde[BitOrAgg] { } object CometBitXOrAgg extends CometAggregateExpressionSerde[BitXorAgg] { + override def supportsMixedPartialFinal: Boolean = true + override def convert( aggExpr: AggregateExpression, bitXor: BitXorAgg, diff --git a/spark/src/main/scala/org/apache/comet/serde/datetime.scala b/spark/src/main/scala/org/apache/comet/serde/datetime.scala index cb3be75717..b57b1e4e56 100644 --- a/spark/src/main/scala/org/apache/comet/serde/datetime.scala +++ b/spark/src/main/scala/org/apache/comet/serde/datetime.scala @@ -21,7 +21,7 @@ package org.apache.comet.serde import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Attribute, DateAdd, DateDiff, DateFormatClass, DateFromUnixDate, DateSub, DayOfMonth, DayOfWeek, DayOfYear, Days, GetDateField, Hour, Hours, LastDay, Literal, MakeDate, Minute, Month, NextDay, Quarter, Second, SecondsToTimestamp, TruncDate, TruncTimestamp, UnixDate, UnixTimestamp, WeekDay, WeekOfYear, Year} +import org.apache.spark.sql.catalyst.expressions.{Attribute, ConvertTimezone, DateAdd, DateDiff, DateFormatClass, DateFromUnixDate, DateSub, DayOfMonth, DayOfWeek, DayOfYear, Days, FromUTCTimestamp, GetDateField, Hour, Hours, LastDay, Literal, MakeDate, Minute, Month, NextDay, Quarter, Second, SecondsToTimestamp, ToUTCTimestamp, TruncDate, TruncTimestamp, UnixDate, UnixTimestamp, WeekDay, WeekOfYear, Year} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DateType, DoubleType, FloatType, IntegerType, LongType, StringType, TimestampNTZType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -364,6 +364,72 @@ object CometDateAdd extends CometScalarFunction[DateAdd]("date_add") object CometDateSub extends CometScalarFunction[DateSub]("date_sub") +private object UTCTimestampSerde { + val tzParseIncompatReason: String = + "Comet's native timezone parser only accepts IANA zone IDs (e.g." + + " `America/Los_Angeles`) and fixed offsets in `+HH:MM` form. Spark also" + + " accepts forms such as `GMT+1`, `UTC+1`, or three-letter abbreviations like" + + " `PST`; queries using those forms will throw a native parse error at" + + " execution time. See https://github.com/apache/datafusion-comet/issues/2013." +} + +object CometFromUTCTimestamp extends CometExpressionSerde[FromUTCTimestamp] { + + override def getSupportLevel(expr: FromUTCTimestamp): SupportLevel = + Incompatible(Some(UTCTimestampSerde.tzParseIncompatReason)) + + override def getIncompatibleReasons(): Seq[String] = + Seq(UTCTimestampSerde.tzParseIncompatReason) + + override def convert( + expr: FromUTCTimestamp, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + val childExprs = expr.children.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = scalarFunctionExprToProto("from_utc_timestamp", childExprs: _*) + optExprWithInfo(optExpr, expr, expr.children: _*) + } +} + +object CometToUTCTimestamp extends CometExpressionSerde[ToUTCTimestamp] { + + override def getSupportLevel(expr: ToUTCTimestamp): SupportLevel = + Incompatible(Some(UTCTimestampSerde.tzParseIncompatReason)) + + override def getIncompatibleReasons(): Seq[String] = + Seq(UTCTimestampSerde.tzParseIncompatReason) + + override def convert( + expr: ToUTCTimestamp, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + val childExprs = expr.children.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = scalarFunctionExprToProto("to_utc_timestamp", childExprs: _*) + optExprWithInfo(optExpr, expr, expr.children: _*) + } +} + +object CometConvertTimezone extends CometExpressionSerde[ConvertTimezone] { + + override def getSupportLevel(expr: ConvertTimezone): SupportLevel = + Incompatible(Some(UTCTimestampSerde.tzParseIncompatReason)) + + override def getIncompatibleReasons(): Seq[String] = + Seq(UTCTimestampSerde.tzParseIncompatReason) + + override def convert( + expr: ConvertTimezone, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + val srcTz = exprToProtoInternal(expr.sourceTz, inputs, binding) + val tgtTz = exprToProtoInternal(expr.targetTz, inputs, binding) + val ts = exprToProtoInternal(expr.sourceTs, inputs, binding) + val toUtc = scalarFunctionExprToProto("to_utc_timestamp", ts, srcTz) + val fromUtc = scalarFunctionExprToProto("from_utc_timestamp", toUtc, tgtTz) + optExprWithInfo(fromUtc, expr, expr.children: _*) + } +} + object CometNextDay extends CometScalarFunction[NextDay]("next_day") object CometMakeDate extends CometScalarFunction[MakeDate]("make_date") diff --git a/spark/src/main/scala/org/apache/comet/serde/hash.scala b/spark/src/main/scala/org/apache/comet/serde/hash.scala index b059199735..a58e81b02d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/hash.scala +++ b/spark/src/main/scala/org/apache/comet/serde/hash.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Murmur3 import org.apache.spark.sql.types.{ArrayType, DataType, DecimalType, IntegerType, LongType, MapType, StringType, StructType} import org.apache.comet.CometSparkSessionExtensions.withInfo -import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, scalarFunctionExprToProtoWithReturnType, serializeDataType, supportedDataType} +import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, isTimeType, scalarFunctionExprToProtoWithReturnType, serializeDataType, supportedDataType} object CometXxHash64 extends CometExpressionSerde[XxHash64] { override def convert( @@ -126,6 +126,9 @@ private object HashUtils { isSupportedDataType(expr, a.elementType) case m: MapType => isSupportedDataType(expr, m.keyType) && isSupportedDataType(expr, m.valueType) + case dt if isTimeType(dt) => + withInfo(expr, s"Unsupported datatype $dt") + false case _ if !supportedDataType(dt, allowComplex = true) => withInfo(expr, s"Unsupported datatype $dt") false diff --git a/spark/src/main/scala/org/apache/comet/serde/literals.scala b/spark/src/main/scala/org/apache/comet/serde/literals.scala index c81b146a89..5b03985c09 100644 --- a/spark/src/main/scala/org/apache/comet/serde/literals.scala +++ b/spark/src/main/scala/org/apache/comet/serde/literals.scala @@ -32,7 +32,7 @@ import com.google.protobuf.ByteString import org.apache.comet.CometSparkSessionExtensions.withInfo import org.apache.comet.DataTypeSupport.isComplexType import org.apache.comet.serde.{CometExpressionSerde, Compatible, ExprOuterClass, LiteralOuterClass, SupportLevel, Unsupported} -import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType} +import org.apache.comet.serde.QueryPlanSerde.{isTimeType, serializeDataType, supportedDataType} import org.apache.comet.serde.Types.ListLiteral object CometLiteral extends CometExpressionSerde[Literal] with Logging { @@ -80,6 +80,8 @@ object CometLiteral extends CometExpressionSerde[Literal] with Logging { case _: IntegerType | _: DateType => exprBuilder.setIntVal(value.asInstanceOf[Int]) case _: LongType | _: TimestampType | _: TimestampNTZType => exprBuilder.setLongVal(value.asInstanceOf[Long]) + case dt if isTimeType(dt) => + exprBuilder.setLongVal(value.asInstanceOf[Long]) case _: FloatType => exprBuilder.setFloatVal(value.asInstanceOf[Float]) case _: DoubleType => exprBuilder.setDoubleVal(value.asInstanceOf[Double]) case _: StringType => diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index 4c5c8e8fcc..5009232a46 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -211,6 +211,8 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { commonBuilder.setIgnoreMissingFieldId( scan.conf.getConf(SQLConf.IGNORE_MISSING_PARQUET_FIELD_ID)) + commonBuilder.setAllowTypePromotion(CometConf.COMET_SCHEMA_EVOLUTION_ENABLED) + // Collect S3/cloud storage configurations val hadoopConf = scan.relation.sparkSession.sessionState .newHadoopConfWithOptions(scan.relation.options) diff --git a/spark/src/main/scala/org/apache/comet/serde/url.scala b/spark/src/main/scala/org/apache/comet/serde/url.scala new file mode 100644 index 0000000000..fa5742860c --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/url.scala @@ -0,0 +1,47 @@ +/* + * 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.serde + +import org.apache.spark.sql.catalyst.expressions.{Attribute, ParseUrl} + +import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto} + +// On Spark 4.x ParseUrl is RuntimeReplaceable and handled via CometExprShim (ParseUrlEvaluator). +object CometParseUrl extends CometExpressionSerde[ParseUrl] { + + private val incompatibleReason = + "Native parse_url diverges from Spark on several edge cases " + + "(https://github.com/apache/datafusion/issues/21943)" + + override def getIncompatibleReasons(): Seq[String] = Seq(incompatibleReason) + + override def getSupportLevel(expr: ParseUrl): SupportLevel = + Incompatible(Some(incompatibleReason)) + + override def convert( + expr: ParseUrl, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + val funcName = if (expr.failOnError) "parse_url" else "try_parse_url" + val childExprs = expr.children.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = scalarFunctionExprToProto(funcName, childExprs: _*) + optExprWithInfo(optExpr, expr, expr.children: _*) + } +} diff --git a/common/src/main/scala/org/apache/comet/udf/CometUDF.scala b/spark/src/main/scala/org/apache/comet/udf/CometUDF.scala similarity index 61% rename from common/src/main/scala/org/apache/comet/udf/CometUDF.scala rename to spark/src/main/scala/org/apache/comet/udf/CometUDF.scala index 5b6652d90a..6b435c4064 100644 --- a/common/src/main/scala/org/apache/comet/udf/CometUDF.scala +++ b/spark/src/main/scala/org/apache/comet/udf/CometUDF.scala @@ -30,12 +30,18 @@ import org.apache.arrow.vector.ValueVector * - The returned vector's length must match `numRows`. * * `numRows` mirrors DataFusion's `ScalarFunctionArgs.number_rows` and is the batch row count. - * UDFs that always have at least one batch-length input can derive length from the inputs and - * ignore `numRows`; UDFs that may be called with zero data columns (e.g. a zero-arg ScalaUDF) - * need `numRows` to know how many rows to produce. + * UDFs that always have at least one batch-length input can read length from it and ignore + * `numRows`; UDFs that may be called with zero data columns (e.g. a zero-arg ScalaUDF through the + * codegen dispatcher) need `numRows` to know how many rows to produce. * - * Implementations must have a public no-arg constructor and must be stateless: a single instance - * per class is cached and shared across native worker threads for the lifetime of the JVM. + * Implementations must have a public no-arg constructor. A fresh instance is created per Spark + * task attempt per class and reused for every call within that task. Instances may hold per-task + * state in fields (counters, compiled patterns, scratch buffers); instances are dropped at task + * completion. Do not hold state that must persist across tasks. + * + * At most one thread calls `evaluate` on a given instance at a time: Spark runs one native future + * per partition and Tokio polls one future per worker, so the per-task instance is never touched + * concurrently even if the task's future migrates between Tokio workers across batches. */ trait CometUDF { def evaluate(inputs: Array[ValueVector], numRows: Int): ValueVector diff --git a/spark/src/main/scala/org/apache/comet/udf/codegen/CometScalaUDFCodegen.scala b/spark/src/main/scala/org/apache/comet/udf/codegen/CometScalaUDFCodegen.scala new file mode 100644 index 0000000000..f575dd5b53 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/udf/codegen/CometScalaUDFCodegen.scala @@ -0,0 +1,326 @@ +/* + * 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.udf.codegen + +import java.nio.ByteBuffer +import java.util.Collections +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable +import scala.util.control.NonFatal + +import org.apache.arrow.vector._ +import org.apache.arrow.vector.complex.{ListVector, MapVector, StructVector} +import org.apache.arrow.vector.types.pojo.Field +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.comet.util.Utils +import org.apache.spark.sql.types.{BinaryType, DataType, StringType} + +import org.apache.comet.codegen.{CometBatchKernel, CometBatchKernelCodegen} +import org.apache.comet.codegen.CometBatchKernelCodegen.{ArrayColumnSpec, ArrowColumnSpec, MapColumnSpec, ScalarColumnSpec, StructColumnSpec, StructFieldSpec} +import org.apache.comet.udf.CometUDF + +/** + * Arrow-direct codegen dispatcher. For each `(bound expression, input Arrow schema)` pair, + * compiles a specialized [[CometBatchKernel]] on first encounter, initializes it with the task's + * partition index, and caches the live instance. + * + * Arg 0 is a `VarBinaryVector` scalar carrying the closure-serialized bound `Expression` bytes; + * args 1..N are the data columns the `BoundReference`s read in ordinal order. + * + * Caching hierarchy, broadest scope on the left: + * {{{ + * +----------------------------+ +----------------------------+ +----------------------------+ + * | 1. JVM bytecode cache | | 2. Per-task dispatcher | | 3. Per-task kernel cache | + * | (Spark's CodeGenerator) | | (CometUdfBridge. | | (kernelCache field) | + * | | | INSTANCES) | | | + * +----------------------------+ +----------------------------+ +----------------------------+ + * | Key: generated Java | | Key: task + UDF class | | Key: bound expression + | + * | source | | | | input column shapes | + * | Value: compiled Java class | | Value: dispatcher object | | Value: ready-to-run kernel | + * | Scope: JVM, all queries | | Scope: one Spark task | | with state primed | + * | share it | | | | Scope: one Spark task | + * | Owner: Spark | | Owner: Comet | | (lives inside 2) | + * | | | | | Owner: Comet | + * +----------------------------+ +----------------------------+ +----------------------------+ + * }}} + * + * Stateful expressions (`Rand`, `MonotonicallyIncreasingID`) advance inside the per-task kernel + * across batches. + * + * `evaluate` runs under `this.synchronized` because DataFusion operators like `HashJoinExec` + * pipeline build/probe via `OnceAsync` (`tokio::spawn`), so multiple Tokio worker threads can + * call back into one task's dispatcher. The kernel's per-batch instance fields would race + * otherwise. + * + * TODO(udf-codegen-pool): if intra-task UDF parallelism shows up as a bottleneck, replace the + * per-key kernel instance with a pool and externalize per-partition counters. + */ +class CometScalaUDFCodegen extends CometUDF with Logging { + + /** + * Per-task cache keyed on serialized expression bytes plus per-column specs. The deserialized + * `boundExpr` carries mutable state (`NamedLambdaVariable.value` for HOFs, `Rand`'s + * `XORShiftRandom`) that must not be shared across concurrent tasks running the same query; + * keeping the cache per-task gives each task its own copy. Guarded by `this.synchronized`. + */ + private val kernelCache + : mutable.Map[CometScalaUDFCodegen.CacheKey, CometScalaUDFCodegen.CacheEntry] = + mutable.HashMap.empty + + override def evaluate(inputs: Array[ValueVector], numRows: Int): ValueVector = { + require( + inputs.length >= 1, + "CometScalaUDFCodegen requires at least 1 input (serialized expression), " + + s"got ${inputs.length}") + val exprVec = inputs(0).asInstanceOf[VarBinaryVector] + require( + exprVec.getValueCount >= 1 && !exprVec.isNull(0), + "CometScalaUDFCodegen requires non-null serialized expression bytes at arg 0") + val bytes = exprVec.get(0) + + // TODO(dict-encoded): kernels assume materialized inputs. Dict-encoded vectors would fail the + // cast in `specFor` below. Fix is to materialize at the dispatcher (via + // `CDataDictionaryProvider`) or widen `emitTypedGetters` with a dict-index + lookup path. + + val numDataCols = inputs.length - 1 + val dataCols = new Array[ValueVector](numDataCols) + val specs = new Array[ArrowColumnSpec](numDataCols) + var di = 0 + while (di < numDataCols) { + val v = inputs(di + 1) + dataCols(di) = v + specs(di) = specFor(v) + di += 1 + } + val n = numRows + val specsSeq = specs.toIndexedSeq + + val key = CometScalaUDFCodegen.CacheKey(ByteBuffer.wrap(bytes), specsSeq) + + // Cache lookup and `process` run under one lock to serialize concurrent Tokio callers that + // would otherwise race on the kernel's per-batch instance fields. + this.synchronized { + val entry = lookupOrCompile(key, bytes, specsSeq) + + val out = CometBatchKernelCodegen.allocateOutput( + entry.outputField, + n, + estimatedOutputBytes(entry.outputType, dataCols)) + try { + entry.kernel.process(dataCols, out, n) + out.setValueCount(n) + out + } catch { + case t: Throwable => + try out.close() + catch { + case NonFatal(_) => () + } + throw t + } + } + } + + private def lookupOrCompile( + key: CometScalaUDFCodegen.CacheKey, + bytes: Array[Byte], + specs: IndexedSeq[ArrowColumnSpec]): CometScalaUDFCodegen.CacheEntry = { + assert(Thread.holdsLock(this), "lookupOrCompile must run under this.synchronized") + kernelCache.get(key) match { + case Some(entry) => + CometScalaUDFCodegen.cacheHitCount.incrementAndGet() + entry + case None => + val loader = Option(Thread.currentThread().getContextClassLoader) + .getOrElse(classOf[Expression].getClassLoader) + val boundExpr = + try { + SparkEnv.get.closureSerializer + .newInstance() + .deserialize[Expression](ByteBuffer.wrap(bytes), loader) + } catch { + case NonFatal(t) => + logError( + "CometScalaUDFCodegen: closure-deserialize failed " + + s"(bytes=${bytes.length}, specs=$specs)", + t) + throw t + } + val compiled = CometBatchKernelCodegen.compile(boundExpr, specs) + val kernel = compiled.newInstance() + kernel.init(CometScalaUDFCodegen.currentPartitionIndex()) + val outputField = CometBatchKernelCodegen.toFfiArrowField( + "codegen_result", + boundExpr.dataType, + boundExpr.nullable) + val entry = + CometScalaUDFCodegen.CacheEntry(compiled, kernel, boundExpr.dataType, outputField) + kernelCache.put(key, entry) + CometScalaUDFCodegen.compileCount.incrementAndGet() + CometScalaUDFCodegen.recordCompiledSignature(specs, boundExpr.dataType) + entry + } + } + + /** + * Build the compile-time spec for one input Arrow vector. Recurses on complex types. + * + * Top-level `nullable=true` is hardcoded: the cache key does not specialize on per-batch null + * density. Schema-declared nullability still reaches the kernel via `BoundReference.nullable` + * embedded in `bytesKey`, so `BoundReference.doGenCode` elides its own `isNullAt` probe on + * non-null columns. `StructFieldSpec.nullable` reads `field.isNullable` from Arrow metadata, + * which is a schema property and therefore stable across batches. + */ + private def specFor(v: ValueVector): ArrowColumnSpec = v match { + case map: MapVector => + // MapVector extends ListVector, match it first. + val struct = map.getDataVector.asInstanceOf[StructVector] + val keyVec = struct.getChildByOrdinal(0).asInstanceOf[ValueVector] + val valueVec = struct.getChildByOrdinal(1).asInstanceOf[ValueVector] + MapColumnSpec( + nullable = true, + keySparkType = Utils.fromArrowField(keyVec.getField), + valueSparkType = Utils.fromArrowField(valueVec.getField), + key = specFor(keyVec), + value = specFor(valueVec)) + case list: ListVector => + val child = list.getDataVector + ArrayColumnSpec(nullable = true, Utils.fromArrowField(child.getField), specFor(child)) + case struct: StructVector => + val fieldSpecs = (0 until struct.size()).map { fi => + val childVec = struct.getChildByOrdinal(fi).asInstanceOf[ValueVector] + val field = struct.getField.getChildren.get(fi) + StructFieldSpec( + name = field.getName, + sparkType = Utils.fromArrowField(field), + nullable = field.isNullable, + child = specFor(childVec)) + } + StructColumnSpec(nullable = true, fieldSpecs) + case _: BitVector | _: TinyIntVector | _: SmallIntVector | _: IntVector | _: BigIntVector | + _: Float4Vector | _: Float8Vector | _: DecimalVector | _: VarCharVector | + _: VarBinaryVector | _: DateDayVector | _: TimeStampMicroVector | + _: TimeStampMicroTZVector => + ScalarColumnSpec(v.getClass.asInstanceOf[Class[_ <: ValueVector]], nullable = true) + case other => + throw new UnsupportedOperationException( + s"CometScalaUDFCodegen: unsupported Arrow vector ${other.getClass.getSimpleName}") + } + + /** + * Sum of variable-width input data buffer sizes as an upper bound for typical transform outputs + * (replace, upper, lower, substring, concat). Underestimates are still corrected by `setSafe`; + * this just reduces the odds of mid-loop reallocation. + */ + private def estimatedOutputBytes(outputType: DataType, dataCols: Array[ValueVector]): Int = { + outputType match { + case _: StringType | _: BinaryType => + var sum = 0 + var i = 0 + while (i < dataCols.length) { + dataCols(i) match { + case v: BaseVariableWidthVector => sum += v.getDataBuffer.writerIndex().toInt + case _ => // no size hint for fixed-width vector types + } + i += 1 + } + sum + case _ => -1 + } + } +} + +object CometScalaUDFCodegen { + + // JVM-wide counters across all per-task instances. Compile work is deduped JVM-wide via + // `CodeGenerator.compile`'s source cache. These track this dispatcher's per-task cache activity. + private val compileCount = new AtomicLong(0) + private val cacheHitCount = new AtomicLong(0) + + // Append-only set of distinct compiled-kernel signatures. Lets tests assert specialization + // shape (vector-class / dataType combinations the dispatcher emitted) and that composed + // subtrees fuse into one kernel. Per-task caches are dropped on completion, leaving no other + // place to observe the set across runs. + private val compiledSignatures = + Collections.synchronizedSet( + new java.util.HashSet[(IndexedSeq[Class[_ <: ValueVector]], DataType)]()) + + /** Snapshot of JVM-wide counters and distinct-signature count. */ + def stats(): DispatcherStats = + DispatcherStats(compileCount.get(), cacheHitCount.get(), compiledSignatures.size()) + + /** Reset counters; leaves the signature set intact. Tests only. */ + def resetStats(): Unit = { + compileCount.set(0) + cacheHitCount.set(0) + } + + /** + * Distinct compiled-kernel signatures: `(input vector classes in ordinal order, output Spark + * DataType)`. `ArrowColumnSpec.nullable` is intentionally omitted so the signature reflects + * what would specialize the kernel regardless of any future per-batch nullability variants. + */ + def snapshotCompiledSignatures(): Set[(IndexedSeq[Class[_ <: ValueVector]], DataType)] = { + import scala.jdk.CollectionConverters._ + compiledSignatures.synchronized { + compiledSignatures.iterator().asScala.toSet + } + } + + private[codegen] def recordCompiledSignature( + specs: IndexedSeq[ArrowColumnSpec], + outputType: DataType): Unit = { + compiledSignatures.add((specs.map(_.vectorClass), outputType)) + } + + /** + * Partition index for the kernel's `init`. Expressions whose `doGenCode` calls + * `addPartitionInitializationStatement` (`Rand`, `Randn`, `Uuid`) reseed mutable state from + * this. Falls back to 0 when the dispatcher is exercised outside a Spark task (unit tests). + */ + private def currentPartitionIndex(): Int = + Option(TaskContext.get()).map(_.partitionId()).getOrElse(0) + + /** + * Cache key: serialized expression bytes plus per-column compile-time invariants. `hashCode` + * walks `bytesKey` per lookup, so for large ScalaUDF closures it scales with closure size. + * + * TODO(perf-cache-key): if hot, options are a driver-precomputed hash piggybacked through the + * proto, per-instance last-key memoization, or a two-tier cache keyed on the generated source. + */ + final case class CacheKey(bytesKey: ByteBuffer, specs: IndexedSeq[ArrowColumnSpec]) + + /** Snapshot of dispatcher cache counters and current size. */ + final case class DispatcherStats(compileCount: Long, cacheHitCount: Long, cacheSize: Int) { + def hitRate: Double = + if (totalLookups == 0) 0.0 else cacheHitCount.toDouble / totalLookups.toDouble + + def totalLookups: Long = compileCount + cacheHitCount + } + + private case class CacheEntry( + compiled: CometBatchKernelCodegen.CompiledKernel, + kernel: CometBatchKernel, + outputType: DataType, + outputField: Field) +} diff --git a/common/src/main/scala/org/apache/comet/vector/NativeUtil.scala b/spark/src/main/scala/org/apache/comet/vector/NativeUtil.scala similarity index 100% rename from common/src/main/scala/org/apache/comet/vector/NativeUtil.scala rename to spark/src/main/scala/org/apache/comet/vector/NativeUtil.scala diff --git a/common/src/main/scala/org/apache/comet/vector/StreamReader.scala b/spark/src/main/scala/org/apache/comet/vector/StreamReader.scala similarity index 100% rename from common/src/main/scala/org/apache/comet/vector/StreamReader.scala rename to spark/src/main/scala/org/apache/comet/vector/StreamReader.scala diff --git a/common/src/main/scala/org/apache/spark/comet/CometTaskContextShim.scala b/spark/src/main/scala/org/apache/spark/comet/CometTaskContextShim.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/comet/CometTaskContextShim.scala rename to spark/src/main/scala/org/apache/spark/comet/CometTaskContextShim.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/CastOverflowException.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CastOverflowException.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/CastOverflowException.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/CastOverflowException.scala 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..b9fc47c5c8 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 @@ -19,8 +19,6 @@ package org.apache.spark.sql.comet -import scala.reflect.ClassTag - import org.apache.spark.{Partition, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -39,7 +37,7 @@ import org.apache.spark.util.collection._ import com.google.common.base.Objects -import org.apache.comet.parquet.{CometParquetFileFormat, CometParquetUtils} +import org.apache.comet.parquet.CometParquetUtils import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.exprToProto @@ -355,17 +353,6 @@ object CometNativeScanExec { scanExec: FileSourceScanExec, session: SparkSession, scan: CometScanExec): CometNativeScanExec = { - // TreeNode.mapProductIterator is protected method. - def mapProductIterator[B: ClassTag](product: Product, f: Any => B): Array[B] = { - val arr = Array.ofDim[B](product.productArity) - var i = 0 - while (i < arr.length) { - arr(i) = f(product.productElement(i)) - i += 1 - } - arr - } - // Generate unique key for this scan so PlanDataInjector can match common+partition data. // Multiple scans of same table with different projections/filters get different keys. val common = nativeOp.getNativeScan.getCommon @@ -378,31 +365,18 @@ object CometNativeScanExec { val hashCode = keyComponents.mkString("|").hashCode val sourceKey = s"${source}_${hashCode}" - // Replacing the relation in FileSourceScanExec by `copy` seems causing some issues - // on other Spark distributions if FileSourceScanExec constructor is changed. - // Using `makeCopy` to avoid the issue. - // https://github.com/apache/arrow-datafusion-comet/issues/190 - def transform(arg: Any): AnyRef = arg match { - case _: HadoopFsRelation => - scanExec.relation.copy(fileFormat = new CometParquetFileFormat(session))(session) - case other: AnyRef => other - case null => null - } - - val newArgs = mapProductIterator(scanExec, transform) - val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec] val batchScanExec = CometNativeScanExec( nativeOp, - wrapped.relation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.optionalNumCoalescedBuckets, - wrapped.dataFilters, - wrapped.tableIdentifier, - wrapped.disableBucketedScan, - wrapped, + scanExec.relation, + scanExec.output, + scanExec.requiredSchema, + scanExec.partitionFilters, + scanExec.optionalBucketSet, + scanExec.optionalNumCoalescedBuckets, + scanExec.dataFilters, + scanExec.tableIdentifier, + scanExec.disableBucketedScan, + scanExec, SerializedPlan(None), scan, sourceKey) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala index 652fdfc96d..8dd092fc74 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -21,7 +21,6 @@ package org.apache.spark.sql.comet import scala.collection.mutable.HashMap import scala.concurrent.duration.NANOSECONDS -import scala.reflect.ClassTag import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD @@ -31,33 +30,23 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.comet.shims.ShimCometScanExec import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric._ import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection._ -import org.apache.comet.{CometConf, MetricsSupport} -import org.apache.comet.parquet.CometParquetFileFormat - /** - * Comet physical scan node for DataSource V1. Most of the code here follow Spark's - * [[FileSourceScanExec]]. - * - * This is a hybrid scan where the native plan will contain a `ScanExec` that reads batches of - * data from the JVM via JNI. The ultimate source of data may be a JVM implementation such as - * Spark readers, or could be the `native_iceberg_compat` native scan. - * - * Note that scanImpl can only be `native_datafusion` after CometScanRule runs and before - * CometExecRule runs. It will never be set to `native_datafusion` at execution time + * Comet physical scan node for DataSource V1. This node is created by CometScanRule as a planning + * intermediate and is always replaced before execution: CometExecRule converts it to a + * [[CometNativeScanExec]], or falls back to the wrapped [[FileSourceScanExec]] on failure. It is + * not a runtime exec node and its `doExecute` / `doExecuteColumnar` will throw. */ case class CometScanExec( - scanImpl: String, @transient relation: HadoopFsRelation, output: Seq[Attribute], requiredSchema: StructType, @@ -72,10 +61,8 @@ case class CometScanExec( with ShimCometScanExec with CometPlan { - assert(scanImpl != CometConf.SCAN_AUTO) - override val nodeName: String = - s"CometScan [$scanImpl] $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" + s"CometScan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" // FIXME: ideally we should reuse wrapped.supportsColumnar, however that fails many tests override lazy val supportsColumnar: Boolean = @@ -154,18 +141,13 @@ case class CometScanExec( } /** - * Returns the data filters that are supported for this scan implementation. For - * native_datafusion scans, this excludes dynamic pruning filters (subqueries) and null checks - * on array columns (see [[isNullCheckOnArrayColumn]]). + * Returns the data filters that are supported for this scan. Excludes dynamic pruning filters + * (subqueries) and null checks on array columns (see [[isNullCheckOnArrayColumn]]). */ lazy val supportedDataFilters: Seq[Expression] = { - if (scanImpl == CometConf.SCAN_NATIVE_DATAFUSION) { - dataFilters - .filterNot(isDynamicPruningFilter) - .filterNot(isNullCheckOnArrayColumn) - } else { - dataFilters - } + dataFilters + .filterNot(isDynamicPruningFilter) + .filterNot(isNullCheckOnArrayColumn) } /** @@ -196,11 +178,6 @@ case class CometScanExec( case _ => false } - @transient - private lazy val pushedDownFilters = { - getPushedDownFilters(relation, supportedDataFilters) - } - override lazy val metadata: Map[String, String] = if (wrapped == null) Map.empty else wrapped.metadata @@ -231,36 +208,9 @@ case class CometScanExec( |""".stripMargin } - lazy val inputRDD: RDD[InternalRow] = { - val options = relation.options + - (FileFormat.OPTION_RETURNING_BATCH -> supportsColumnar.toString) - val readFile: (PartitionedFile) => Iterator[InternalRow] = - relation.fileFormat.buildReaderWithPartitionValues( - sparkSession = relation.sparkSession, - dataSchema = relation.dataSchema, - partitionSchema = relation.partitionSchema, - requiredSchema = requiredSchema, - filters = pushedDownFilters, - options = options, - hadoopConf = - relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options)) - - val readRDD = if (bucketedScan) { - createBucketedReadRDD( - relation.bucketSpec.get, - readFile, - dynamicallySelectedPartitions, - relation) - } else { - createReadRDD(readFile, dynamicallySelectedPartitions, relation) - } - sendDriverMetrics() - readRDD - } - - override def inputRDDs(): Seq[RDD[InternalRow]] = { - inputRDD :: Nil - } + override def inputRDDs(): Seq[RDD[InternalRow]] = + throw new UnsupportedOperationException( + "CometScanExec is a planning intermediate and should never reach execution") /** Helper for computing total number and size of files in selected partitions. */ private def setFilesNumAndSizeMetric( @@ -281,46 +231,19 @@ case class CometScanExec( } override lazy val metrics: Map[String, SQLMetric] = - wrapped.driverMetrics ++ CometMetricNode.baseScanMetrics( - session.sparkContext) ++ (relation.fileFormat match { - case m: MetricsSupport => m.getMetrics - case _ => Map.empty - }) - - protected override def doExecute(): RDD[InternalRow] = { - ColumnarToRowExec(this).doExecute() - } + wrapped.driverMetrics ++ CometMetricNode.baseScanMetrics(session.sparkContext) - protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { - val rdd = inputRDD.asInstanceOf[RDD[ColumnarBatch]] - - // These metrics are important for streaming solutions. - // despite there being similar metrics published by the native reader. - val numOutputRows = longMetric("numOutputRows") - val scanTime = longMetric("scanTime") - rdd.mapPartitionsInternal { batches => - new Iterator[ColumnarBatch] { - - override def hasNext: Boolean = { - // The `FileScanRDD` returns an iterator which scans the file during the `hasNext` call. - val startNs = System.nanoTime() - val res = batches.hasNext - scanTime += System.nanoTime() - startNs - res - } + protected override def doExecute(): RDD[InternalRow] = + throw new UnsupportedOperationException( + "CometScanExec is a planning intermediate and should never reach execution") - override def next(): ColumnarBatch = { - val batch = batches.next() - numOutputRows += batch.numRows() - batch - } - } - } - } + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = + throw new UnsupportedOperationException( + "CometScanExec is a planning intermediate and should never reach execution") - override def executeCollect(): Array[InternalRow] = { - ColumnarToRowExec(this).executeCollect() - } + override def executeCollect(): Array[InternalRow] = + throw new UnsupportedOperationException( + "CometScanExec is a planning intermediate and should never reach execution") /** * Get the file partitions for this scan without instantiating readers or RDD. This is useful @@ -456,67 +379,8 @@ case class CometScanExec( FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) } - /** - * Create an RDD for bucketed reads. The non-bucketed variant of this function is - * [[createReadRDD]]. - * - * Each RDD partition being returned should include all the files with the same bucket id from - * all the given Hive partitions. - * - * @param bucketSpec - * the bucketing spec. - * @param readFile - * a function to read each (part of a) file. - * @param selectedPartitions - * Hive-style partition that are part of the read. - * @param fsRelation - * [[HadoopFsRelation]] associated with the read. - */ - private def createBucketedReadRDD( - bucketSpec: BucketSpec, - readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Array[PartitionDirectory], - fsRelation: HadoopFsRelation): RDD[InternalRow] = { - val filePartitions = - createFilePartitionsForBucketedScan(bucketSpec, selectedPartitions, fsRelation) - prepareRDD(fsRelation, readFile, filePartitions) - } - - /** - * Create an RDD for non-bucketed reads. The bucketed variant of this function is - * [[createBucketedReadRDD]]. - * - * @param readFile - * a function to read each (part of a) file. - * @param selectedPartitions - * Hive-style partition that are part of the read. - * @param fsRelation - * [[HadoopFsRelation]] associated with the read. - */ - private def createReadRDD( - readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Array[PartitionDirectory], - fsRelation: HadoopFsRelation): RDD[InternalRow] = { - val filePartitions = createFilePartitionsForNonBucketedScan(selectedPartitions, fsRelation) - prepareRDD(fsRelation, readFile, filePartitions) - } - - private def prepareRDD( - fsRelation: HadoopFsRelation, - readFile: (PartitionedFile) => Iterator[InternalRow], - partitions: Seq[FilePartition]): RDD[InternalRow] = { - val sqlConf = fsRelation.sparkSession.sessionState.conf - newFileScanRDD( - fsRelation, - readFile, - partitions, - new StructType(requiredSchema.fields ++ fsRelation.partitionSchema.fields), - new ParquetOptions(CaseInsensitiveMap(relation.options), sqlConf)) - } - override def doCanonicalize(): CometScanExec = { CometScanExec( - scanImpl, relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, @@ -534,46 +398,18 @@ case class CometScanExec( object CometScanExec { - def apply( - scanExec: FileSourceScanExec, - session: SparkSession, - scanImpl: String): CometScanExec = { - // TreeNode.mapProductIterator is protected method. - def mapProductIterator[B: ClassTag](product: Product, f: Any => B): Array[B] = { - val arr = Array.ofDim[B](product.productArity) - var i = 0 - while (i < arr.length) { - arr(i) = f(product.productElement(i)) - i += 1 - } - arr - } - - // Replacing the relation in FileSourceScanExec by `copy` seems causing some issues - // on other Spark distributions if FileSourceScanExec constructor is changed. - // Using `makeCopy` to avoid the issue. - // https://github.com/apache/arrow-datafusion-comet/issues/190 - def transform(arg: Any): AnyRef = arg match { - case _: HadoopFsRelation => - scanExec.relation.copy(fileFormat = new CometParquetFileFormat(session))(session) - case other: AnyRef => other - case null => null - } - - val newArgs = mapProductIterator(scanExec, transform) - val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec] + def apply(scanExec: FileSourceScanExec, session: SparkSession): CometScanExec = { val batchScanExec = CometScanExec( - scanImpl, - wrapped.relation, - wrapped.output, - wrapped.requiredSchema, - wrapped.partitionFilters, - wrapped.optionalBucketSet, - wrapped.optionalNumCoalescedBuckets, - wrapped.dataFilters, - wrapped.tableIdentifier, - wrapped.disableBucketedScan, - wrapped) + scanExec.relation, + scanExec.output, + scanExec.requiredSchema, + scanExec.partitionFilters, + scanExec.optionalBucketSet, + scanExec.optionalNumCoalescedBuckets, + scanExec.dataFilters, + scanExec.tableIdentifier, + scanExec.disableBucketedScan, + scanExec) scanExec.logicalLink.foreach(batchScanExec.setLogicalLink) batchScanExec } diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowReaderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowReaderIterator.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowReaderIterator.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowReaderIterator.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/arrow/ArrowWriters.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/execution/arrow/CometArrowConverters.scala 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..b24b96abd1 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 @@ -43,7 +43,7 @@ import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, ShortType, StringType, StructType, TimestampNTZType, TimestampType} +import org.apache.spark.sql.types.{ArrayType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, ShortType, StringType, TimestampNTZType, TimestampType} import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration import org.apache.spark.util.io.ChunkedByteBuffer @@ -54,8 +54,10 @@ import com.google.protobuf.CodedOutputStream import org.apache.comet.{CometConf, CometExecIterator, CometRuntimeException, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.{isCometShuffleEnabled, withInfo} import org.apache.comet.parquet.CometParquetUtils +import org.apache.comet.rules.CometExecRule import org.apache.comet.serde.{CometOperatorSerde, Compatible, Incompatible, OperatorOuterClass, SupportLevel, Unsupported} import org.apache.comet.serde.OperatorOuterClass.{AggregateMode => CometAggregateMode, Operator} +import org.apache.comet.serde.QueryPlanSerde import org.apache.comet.serde.QueryPlanSerde.{aggExprToProto, exprToProto, isStringCollationType, supportedSortType} import org.apache.comet.serde.operator.CometSink @@ -1218,7 +1220,8 @@ object CometExplodeExec extends CometOperatorSerde[GenerateExec] { if (op.generator.children.length != 1) { return Unsupported(Some("generators with multiple inputs are not supported")) } - if (op.generator.nodeName.toLowerCase(Locale.ROOT) != "explode") { + val nodeName = op.generator.nodeName.toLowerCase(Locale.ROOT) + if (nodeName != "explode" && nodeName != "posexplode") { return Unsupported(Some(s"Unsupported generator: ${op.generator.nodeName}")) } if (op.outer) { @@ -1262,10 +1265,13 @@ object CometExplodeExec extends CometOperatorSerde[GenerateExec] { return None } + val isPosExplode = op.generator.nodeName.toLowerCase(Locale.ROOT) == "posexplode" + val explodeBuilder = OperatorOuterClass.Explode .newBuilder() .setChild(childExprProto.get) .setOuter(op.outer) + .setPosition(isPosExplode) .addAllProjectList(projectExprs.map(_.get).asJava) Some(builder.setExplode(explodeBuilder).build()) @@ -1404,13 +1410,6 @@ case class CometUnionExec( trait CometBaseAggregate { - private def containsMapType(dt: DataType): Boolean = dt match { - case _: MapType => true - case StructType(fields) => fields.exists(f => containsMapType(f.dataType)) - case ArrayType(elementType, _) => containsMapType(elementType) - case _ => false - } - def doConvert( aggregate: BaseAggregateExec, builder: Operator.Builder, @@ -1423,10 +1422,29 @@ trait CometBaseAggregate { // We support {Partial, PartialMerge} mix; other combinations are rejected. val multiMode = modes.size > 1 && modeSet != Set(Partial, PartialMerge) // For a final mode HashAggregate, we only need to transform the HashAggregate - // if there is Comet partial aggregation. + // if there is Comet partial aggregation, unless all aggregates have compatible + // intermediate buffer formats (safe for mixed Spark/Comet execution). val sparkFinalMode = modes.contains(Final) && findCometPartialAgg(aggregate.child).isEmpty - if (multiMode || sparkFinalMode) { + if (multiMode) { + withInfo(aggregate, s"Unsupported mixed aggregation modes: ${modes.mkString(", ")}") + return None + } + + if (sparkFinalMode && + !QueryPlanSerde.allAggsSupportMixedExecution(aggregate.aggregateExpressions)) { + withInfo( + aggregate, + "Spark Final aggregate without Comet Partial requires compatible " + + "intermediate buffer formats") + return None + } + + // Check if this aggregate has been tagged as unsafe for mixed execution + // (Comet partial + Spark final with incompatible intermediate buffers) + val unsafeReason = aggregate.getTagValue(CometExecRule.COMET_UNSAFE_PARTIAL) + if (unsafeReason.isDefined) { + withInfo(aggregate, unsafeReason.get) return None } @@ -1441,7 +1459,7 @@ trait CometBaseAggregate { return None } - if (groupingExpressions.exists(expr => containsMapType(expr.dataType))) { + if (groupingExpressions.exists(expr => QueryPlanSerde.containsMapType(expr.dataType))) { withInfo(aggregate, "Grouping on map-containing types is not supported") return None } diff --git a/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala b/spark/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/parquet/CometParquetReadSupport.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala similarity index 100% rename from common/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/parquet/CometSparkToParquetSchemaConverter.scala diff --git a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala b/spark/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala similarity index 97% rename from common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala rename to spark/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala index 78f2e81c7c..783367c054 100644 --- a/common/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/util/Utils.scala @@ -108,6 +108,12 @@ object Utils extends CometTypeShim with Logging { case yi: ArrowType.Interval if yi.getUnit == IntervalUnit.YEAR_MONTH => YearMonthIntervalType() case di: ArrowType.Interval if di.getUnit == IntervalUnit.DAY_TIME => DayTimeIntervalType() + case t: ArrowType.Time if t.getUnit == TimeUnit.NANOSECOND && t.getBitWidth == 64 => + // scalastyle:off classforname + val clazz = Class.forName("org.apache.spark.sql.types.TimeType$") + // scalastyle:on classforname + val module = clazz.getField("MODULE$").get(null) + clazz.getMethod("apply").invoke(module).asInstanceOf[DataType] case _ => throw new UnsupportedOperationException(s"Unsupported data type: ${dt.toString}") } @@ -142,6 +148,8 @@ object Utils extends CometTypeShim with Logging { } case TimestampNTZType => new ArrowType.Timestamp(TimeUnit.MICROSECOND, null) + case dt if isTimeType(dt) => + new ArrowType.Time(TimeUnit.NANOSECOND, 64) case _ => throw new UnsupportedOperationException( s"Unsupported data type: [${dt.getClass.getName}] ${dt.catalogString}") @@ -393,7 +401,7 @@ object Utils extends CometTypeShim with Logging { _: BigIntVector | _: Float4Vector | _: Float8Vector | _: VarCharVector | _: DecimalVector | _: DateDayVector | _: TimeStampMicroTZVector | _: VarBinaryVector | _: FixedSizeBinaryVector | _: TimeStampMicroVector | _: StructVector | _: ListVector | - _: MapVector | _: NullVector) => + _: MapVector | _: NullVector | _: TimeNanoVector) => v.asInstanceOf[FieldVector] case _ => throw new SparkException(s"Unsupported Arrow Vector for $reason: ${valueVector.getClass}") diff --git a/common/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala similarity index 100% rename from common/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala rename to spark/src/main/spark-3.4/org/apache/comet/shims/ShimBatchReader.scala diff --git a/common/src/main/spark-3.4/org/apache/comet/shims/ShimFileFormat.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/ShimFileFormat.scala similarity index 100% rename from common/src/main/spark-3.4/org/apache/comet/shims/ShimFileFormat.scala rename to spark/src/main/spark-3.4/org/apache/comet/shims/ShimFileFormat.scala diff --git a/common/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala b/spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala similarity index 100% rename from common/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala rename to spark/src/main/spark-3.4/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala diff --git a/common/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala similarity index 100% rename from common/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala rename to spark/src/main/spark-3.5/org/apache/comet/shims/ShimBatchReader.scala diff --git a/common/src/main/spark-3.5/org/apache/comet/shims/ShimFileFormat.scala b/spark/src/main/spark-3.5/org/apache/comet/shims/ShimFileFormat.scala similarity index 100% rename from common/src/main/spark-3.5/org/apache/comet/shims/ShimFileFormat.scala rename to spark/src/main/spark-3.5/org/apache/comet/shims/ShimFileFormat.scala diff --git a/common/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala b/spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala similarity index 100% rename from common/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala rename to spark/src/main/spark-3.5/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/CometExprTraitShim.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/CometExprTraitShim.scala new file mode 100644 index 0000000000..2ae589a996 --- /dev/null +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/CometExprTraitShim.scala @@ -0,0 +1,39 @@ +/* + * 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.shims + +import org.apache.spark.sql.catalyst.expressions.{Expression, NullIntolerant} + +/** + * Per-profile view of expression traits that shifted shape across Spark versions. Spark 3.x has a + * `NullIntolerant` marker trait and no scalar-expression `Stateful` concept (added in 4.x as a + * boolean method on `Expression`). Routing checks through one shim avoids version pattern matches + * in the codegen dispatcher. + */ +trait CometExprTraitShim { + def isNullIntolerant(expr: Expression): Boolean = expr.isInstanceOf[NullIntolerant] + + // Aggregate/window/generator stateful cases are rejected elsewhere in `canHandle`, so treating + // all scalar expressions as non-stateful here is conservative-correct on this profile. + def isStateful(expr: Expression): Boolean = false + + // No collation / `ResolvedCollation` concept in 3.x. + def isCodegenInertUnevaluable(expr: Expression): Boolean = false +} diff --git a/common/src/main/spark-4.x/org/apache/comet/shims/ShimCometConf.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/CometInternalRowShim.scala similarity index 73% rename from common/src/main/spark-4.x/org/apache/comet/shims/ShimCometConf.scala rename to spark/src/main/spark-3.x/org/apache/comet/shims/CometInternalRowShim.scala index 0eb57c52b4..18b3a4e6b3 100644 --- a/common/src/main/spark-4.x/org/apache/comet/shims/ShimCometConf.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/CometInternalRowShim.scala @@ -19,6 +19,9 @@ package org.apache.comet.shims -trait ShimCometConf { - protected val COMET_SCHEMA_EVOLUTION_ENABLED_DEFAULT = true -} +/** + * Per-profile shim mixed into `CometInternalRow` and `CometArrayData`. Spark 4.x adds abstract + * `SpecializedGetters` getters (`getVariant` in 4.0, `getGeography` and `getGeometry` in 4.1) + * that subclasses must implement; Spark 3.x has none, so this trait is empty. + */ +trait CometInternalRowShim diff --git a/common/src/main/spark-3.x/org/apache/comet/shims/CometTypeShim.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/CometTypeShim.scala similarity index 92% rename from common/src/main/spark-3.x/org/apache/comet/shims/CometTypeShim.scala rename to spark/src/main/spark-3.x/org/apache/comet/shims/CometTypeShim.scala index 5c47a57a22..041eb3ce67 100644 --- a/common/src/main/spark-3.x/org/apache/comet/shims/CometTypeShim.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/CometTypeShim.scala @@ -32,4 +32,7 @@ trait CometTypeShim { @nowarn // Spark 4 feature; Variant shredding doesn't exist in Spark 3.x. def isVariantStruct(s: StructType): Boolean = false + + @nowarn // Spark 4.1 feature; TimeType doesn't exist in Spark 3.x. + def isTimeType(dt: DataType): Boolean = false } diff --git a/common/src/main/java/org/apache/comet/CometSchemaImporter.java b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala similarity index 62% rename from common/src/main/java/org/apache/comet/CometSchemaImporter.java rename to spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala index 4841f16f19..a113893c84 100644 --- a/common/src/main/java/org/apache/comet/CometSchemaImporter.java +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala @@ -17,16 +17,15 @@ * under the License. */ -package org.apache.comet; +package org.apache.comet.shims -import org.apache.arrow.c.*; -import org.apache.arrow.memory.BufferAllocator; +trait ShimCometConf { -/** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */ -@IcebergApi -public class CometSchemaImporter extends AbstractCometSchemaImporter { - @IcebergApi - public CometSchemaImporter(BufferAllocator allocator) { - super(allocator); - } + /** + * Whether Comet's Parquet scan paths allow widening type promotions (e.g. INT32 → INT64, FLOAT + * → DOUBLE). Spark 3.x's vectorized reader rejects these on read, so Comet matches by + * defaulting to false on 3.x. Reads from the deprecated `spark.comet.schemaEvolution.enabled` + * SQL conf were removed in favor of this per-version constant; see #4298. + */ + val COMET_SCHEMA_EVOLUTION_ENABLED: Boolean = false } diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala index 3d5b34bfd2..c159bde0de 100644 --- a/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala @@ -23,11 +23,12 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Sum import org.apache.spark.sql.catalyst.expressions.json.StructsToJsonEvaluator import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} +import org.apache.spark.sql.catalyst.expressions.url.ParseUrlEvaluator import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DataTypes, MapType, StringType} -import org.apache.comet.CometConf +import org.apache.comet.{CometConf, CometExplainInfo} import org.apache.comet.CometSparkSessionExtensions.withInfo import org.apache.comet.expressions.{CometCast, CometEvalMode} import org.apache.comet.serde.{CommonStringExprs, Compatible, ExprOuterClass, Incompatible, SupportLevel} @@ -133,9 +134,9 @@ trait CometExprShim extends CommonStringExprs { val optExpr = scalarFunctionExprToProto("width_bucket", childExprs: _*) optExprWithInfo(optExpr, wb, wb.children: _*) - // In Spark 4.0, StructsToJson is a RuntimeReplaceable whose replacement is - // Invoke(Literal(StructsToJsonEvaluator), "evaluate", ...). Reconstruct the - // original StructsToJson and recurse so support-level checks apply. + // In Spark 4.x, RuntimeReplaceable expressions (StructsToJson, ParseUrl) become + // Invoke(Literal(Evaluator), "evaluate", ...). Reconstruct the original expression + // and recurse so support-level checks apply. case i: Invoke => (i.targetObject, i.functionName, i.arguments) match { case (Literal(evaluator: StructsToJsonEvaluator, _), "evaluate", Seq(child)) => @@ -143,6 +144,15 @@ trait CometExprShim extends CommonStringExprs { StructsToJson(evaluator.options, child, evaluator.timeZoneId), inputs, binding) + case (Literal(evaluator: ParseUrlEvaluator, _), "evaluate", args) => + val parseUrl = ParseUrl(args, evaluator.failOnError) + val result = exprToProtoInternal(parseUrl, inputs, binding) + if (result.isEmpty) { + parseUrl + .getTagValue(CometExplainInfo.EXTENSION_INFO) + .foreach(reasons => i.setTagValue(CometExplainInfo.EXTENSION_INFO, reasons)) + } + result case _ => None } diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/CometInternalRowShim.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/CometInternalRowShim.scala new file mode 100644 index 0000000000..b855fe3a91 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/CometInternalRowShim.scala @@ -0,0 +1,34 @@ +/* + * 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.shims + +import org.apache.spark.unsafe.types.VariantVal + +/** + * Throwing defaults for Spark 4.0 `SpecializedGetters` additions: `getVariant`. Mixed into + * `CometInternalRow` and `CometArrayData` so the codegen kernel's subclasses satisfy the + * abstract-method check at class-load time. 4.1 also adds `getGeography` / `getGeometry` (see the + * spark-4.1 shim). + */ +trait CometInternalRowShim { + def getVariant(ordinal: Int): VariantVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getVariant not supported") +} diff --git a/spark/src/main/spark-4.1/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-4.1/org/apache/comet/shims/CometExprShim.scala index 5e906a0d83..9d6092aa5c 100644 --- a/spark/src/main/spark-4.1/org/apache/comet/shims/CometExprShim.scala +++ b/spark/src/main/spark-4.1/org/apache/comet/shims/CometExprShim.scala @@ -23,11 +23,13 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Sum import org.apache.spark.sql.catalyst.expressions.json.StructsToJsonEvaluator import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} +import org.apache.spark.sql.catalyst.expressions.url.ParseUrlEvaluator +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation -import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DataTypes, MapType, StringType} +import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DataTypes, MapType, StringType, TimeType} -import org.apache.comet.CometConf +import org.apache.comet.{CometConf, CometExplainInfo} import org.apache.comet.CometSparkSessionExtensions.withInfo import org.apache.comet.expressions.{CometCast, CometEvalMode} import org.apache.comet.serde.{CommonStringExprs, Compatible, ExprOuterClass, Incompatible, SupportLevel} @@ -92,6 +94,16 @@ trait CometExprShim extends CommonStringExprs { val Seq(bin, charset, _, _) = s.arguments stringDecode(expr, charset, bin, inputs, binding) + case s: StaticInvoke + if s.staticObject == classOf[DateTimeUtils.type] && + s.functionName == "makeTime" && + s.arguments.size == 3 && + s.dataType.isInstanceOf[TimeType] => + val childExprs = s.arguments.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = + scalarFunctionExprToProtoWithReturnType("make_time", s.dataType, true, childExprs: _*) + optExprWithInfo(optExpr, expr, s.arguments: _*) + case expr @ ToPrettyString(child, timeZoneId) => val castSupported = CometCast.isSupported( child.dataType, @@ -132,9 +144,9 @@ trait CometExprShim extends CommonStringExprs { val optExpr = scalarFunctionExprToProto("width_bucket", childExprs: _*) optExprWithInfo(optExpr, wb, wb.children: _*) - // In Spark 4.0, StructsToJson is a RuntimeReplaceable whose replacement is - // Invoke(Literal(StructsToJsonEvaluator), "evaluate", ...). Reconstruct the - // original StructsToJson and recurse so support-level checks apply. + // In Spark 4.x, RuntimeReplaceable expressions (StructsToJson, ParseUrl) become + // Invoke(Literal(Evaluator), "evaluate", ...). Reconstruct the original expression + // and recurse so support-level checks apply. case i: Invoke => (i.targetObject, i.functionName, i.arguments) match { case (Literal(evaluator: StructsToJsonEvaluator, _), "evaluate", Seq(child)) => @@ -142,6 +154,36 @@ trait CometExprShim extends CommonStringExprs { StructsToJson(evaluator.options, child, evaluator.timeZoneId), inputs, binding) + case (Literal(evaluator: ParseUrlEvaluator, _), "evaluate", args) => + val parseUrl = ParseUrl(args, evaluator.failOnError) + val result = exprToProtoInternal(parseUrl, inputs, binding) + if (result.isEmpty) { + parseUrl + .getTagValue(CometExplainInfo.EXTENSION_INFO) + .foreach(reasons => i.setTagValue(CometExplainInfo.EXTENSION_INFO, reasons)) + } + result + case (Literal(parser: ToTimeParser, _), "parse", args) + if i.dataType.isInstanceOf[TimeType] && parser.fmt.isEmpty && args.size == 1 => + val childExprs = args.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = + scalarFunctionExprToProtoWithReturnType("to_time", i.dataType, true, childExprs: _*) + optExprWithInfo(optExpr, i, args: _*) + case _ => None + } + + // try_to_time resolves to TryEval(Invoke(Literal(ToTimeParser), "parse", ...)) + case TryEval(i: Invoke) => + (i.targetObject, i.functionName, i.arguments) match { + case (Literal(parser: ToTimeParser, _), "parse", args) + if i.dataType.isInstanceOf[TimeType] && parser.fmt.isEmpty && args.size == 1 => + val childExprs = args.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = scalarFunctionExprToProtoWithReturnType( + "to_time", + i.dataType, + false, + childExprs: _*) + optExprWithInfo(optExpr, expr, args: _*) case _ => None } diff --git a/spark/src/main/spark-4.1/org/apache/comet/shims/CometInternalRowShim.scala b/spark/src/main/spark-4.1/org/apache/comet/shims/CometInternalRowShim.scala new file mode 100644 index 0000000000..ce4cb7c06f --- /dev/null +++ b/spark/src/main/spark-4.1/org/apache/comet/shims/CometInternalRowShim.scala @@ -0,0 +1,40 @@ +/* + * 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.shims + +import org.apache.spark.unsafe.types.{GeographyVal, GeometryVal, VariantVal} + +/** + * Throwing defaults for Spark 4.x `SpecializedGetters` additions: `getVariant` (4.0), + * `getGeography` and `getGeometry` (4.1). Mixed into `CometInternalRow` and `CometArrayData`. + */ +trait CometInternalRowShim { + def getVariant(ordinal: Int): VariantVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getVariant not supported") + + def getGeography(ordinal: Int): GeographyVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getGeography not supported") + + def getGeometry(ordinal: Int): GeometryVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getGeometry not supported") +} diff --git a/spark/src/main/spark-4.2/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-4.2/org/apache/comet/shims/CometExprShim.scala index 5e906a0d83..9d6092aa5c 100644 --- a/spark/src/main/spark-4.2/org/apache/comet/shims/CometExprShim.scala +++ b/spark/src/main/spark-4.2/org/apache/comet/shims/CometExprShim.scala @@ -23,11 +23,13 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.Sum import org.apache.spark.sql.catalyst.expressions.json.StructsToJsonEvaluator import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} +import org.apache.spark.sql.catalyst.expressions.url.ParseUrlEvaluator +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.StringTypeWithCollation -import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DataTypes, MapType, StringType} +import org.apache.spark.sql.types.{ArrayType, BinaryType, BooleanType, DataTypes, MapType, StringType, TimeType} -import org.apache.comet.CometConf +import org.apache.comet.{CometConf, CometExplainInfo} import org.apache.comet.CometSparkSessionExtensions.withInfo import org.apache.comet.expressions.{CometCast, CometEvalMode} import org.apache.comet.serde.{CommonStringExprs, Compatible, ExprOuterClass, Incompatible, SupportLevel} @@ -92,6 +94,16 @@ trait CometExprShim extends CommonStringExprs { val Seq(bin, charset, _, _) = s.arguments stringDecode(expr, charset, bin, inputs, binding) + case s: StaticInvoke + if s.staticObject == classOf[DateTimeUtils.type] && + s.functionName == "makeTime" && + s.arguments.size == 3 && + s.dataType.isInstanceOf[TimeType] => + val childExprs = s.arguments.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = + scalarFunctionExprToProtoWithReturnType("make_time", s.dataType, true, childExprs: _*) + optExprWithInfo(optExpr, expr, s.arguments: _*) + case expr @ ToPrettyString(child, timeZoneId) => val castSupported = CometCast.isSupported( child.dataType, @@ -132,9 +144,9 @@ trait CometExprShim extends CommonStringExprs { val optExpr = scalarFunctionExprToProto("width_bucket", childExprs: _*) optExprWithInfo(optExpr, wb, wb.children: _*) - // In Spark 4.0, StructsToJson is a RuntimeReplaceable whose replacement is - // Invoke(Literal(StructsToJsonEvaluator), "evaluate", ...). Reconstruct the - // original StructsToJson and recurse so support-level checks apply. + // In Spark 4.x, RuntimeReplaceable expressions (StructsToJson, ParseUrl) become + // Invoke(Literal(Evaluator), "evaluate", ...). Reconstruct the original expression + // and recurse so support-level checks apply. case i: Invoke => (i.targetObject, i.functionName, i.arguments) match { case (Literal(evaluator: StructsToJsonEvaluator, _), "evaluate", Seq(child)) => @@ -142,6 +154,36 @@ trait CometExprShim extends CommonStringExprs { StructsToJson(evaluator.options, child, evaluator.timeZoneId), inputs, binding) + case (Literal(evaluator: ParseUrlEvaluator, _), "evaluate", args) => + val parseUrl = ParseUrl(args, evaluator.failOnError) + val result = exprToProtoInternal(parseUrl, inputs, binding) + if (result.isEmpty) { + parseUrl + .getTagValue(CometExplainInfo.EXTENSION_INFO) + .foreach(reasons => i.setTagValue(CometExplainInfo.EXTENSION_INFO, reasons)) + } + result + case (Literal(parser: ToTimeParser, _), "parse", args) + if i.dataType.isInstanceOf[TimeType] && parser.fmt.isEmpty && args.size == 1 => + val childExprs = args.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = + scalarFunctionExprToProtoWithReturnType("to_time", i.dataType, true, childExprs: _*) + optExprWithInfo(optExpr, i, args: _*) + case _ => None + } + + // try_to_time resolves to TryEval(Invoke(Literal(ToTimeParser), "parse", ...)) + case TryEval(i: Invoke) => + (i.targetObject, i.functionName, i.arguments) match { + case (Literal(parser: ToTimeParser, _), "parse", args) + if i.dataType.isInstanceOf[TimeType] && parser.fmt.isEmpty && args.size == 1 => + val childExprs = args.map(exprToProtoInternal(_, inputs, binding)) + val optExpr = scalarFunctionExprToProtoWithReturnType( + "to_time", + i.dataType, + false, + childExprs: _*) + optExprWithInfo(optExpr, expr, args: _*) case _ => None } diff --git a/spark/src/main/spark-4.2/org/apache/comet/shims/CometInternalRowShim.scala b/spark/src/main/spark-4.2/org/apache/comet/shims/CometInternalRowShim.scala new file mode 100644 index 0000000000..ce4cb7c06f --- /dev/null +++ b/spark/src/main/spark-4.2/org/apache/comet/shims/CometInternalRowShim.scala @@ -0,0 +1,40 @@ +/* + * 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.shims + +import org.apache.spark.unsafe.types.{GeographyVal, GeometryVal, VariantVal} + +/** + * Throwing defaults for Spark 4.x `SpecializedGetters` additions: `getVariant` (4.0), + * `getGeography` and `getGeometry` (4.1). Mixed into `CometInternalRow` and `CometArrayData`. + */ +trait CometInternalRowShim { + def getVariant(ordinal: Int): VariantVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getVariant not supported") + + def getGeography(ordinal: Int): GeographyVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getGeography not supported") + + def getGeometry(ordinal: Int): GeometryVal = + throw new UnsupportedOperationException( + s"${getClass.getSimpleName}: getGeometry not supported") +} diff --git a/spark/src/main/spark-4.x/org/apache/comet/shims/CometExprTraitShim.scala b/spark/src/main/spark-4.x/org/apache/comet/shims/CometExprTraitShim.scala new file mode 100644 index 0000000000..a9a3d26bba --- /dev/null +++ b/spark/src/main/spark-4.x/org/apache/comet/shims/CometExprTraitShim.scala @@ -0,0 +1,41 @@ +/* + * 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.shims + +import org.apache.spark.sql.catalyst.expressions.{Expression, ResolvedCollation} + +/** + * Spark 4.x replaced the `NullIntolerant` marker trait with a boolean method on `Expression` and + * added a `stateful` boolean. Neither exists as a trait in 4.x. This shim routes the checks + * through the method form. + */ +trait CometExprTraitShim { + def isNullIntolerant(expr: Expression): Boolean = expr.nullIntolerant + def isStateful(expr: Expression): Boolean = expr.stateful + + // `ResolvedCollation` is an `Unevaluable` leaf living only in `Collate.collation` as a + // type-level marker. `Collate.genCode` passes through to its child and never invokes it. Spark + // 4.1 analyzes it away; 4.0 leaves it in the tree, so the dispatcher's `Unevaluable` guard + // would trip without this exemption. + def isCodegenInertUnevaluable(expr: Expression): Boolean = expr match { + case _: ResolvedCollation => true + case _ => false + } +} diff --git a/common/src/main/spark-4.x/org/apache/comet/shims/CometTypeShim.scala b/spark/src/main/spark-4.x/org/apache/comet/shims/CometTypeShim.scala similarity index 96% rename from common/src/main/spark-4.x/org/apache/comet/shims/CometTypeShim.scala rename to spark/src/main/spark-4.x/org/apache/comet/shims/CometTypeShim.scala index cc287a9b98..17b2738b35 100644 --- a/common/src/main/spark-4.x/org/apache/comet/shims/CometTypeShim.scala +++ b/spark/src/main/spark-4.x/org/apache/comet/shims/CometTypeShim.scala @@ -53,4 +53,7 @@ trait CometTypeShim { // variant shredding layout, so reading such a struct natively returns nulls. Detect the marker // and force scan fallback. def isVariantStruct(s: StructType): Boolean = VariantMetadata.isVariantStruct(s) + + def isTimeType(dt: DataType): Boolean = + dt.getClass.getSimpleName.startsWith("TimeType") } diff --git a/common/src/main/spark-4.x/org/apache/comet/shims/ShimBatchReader.scala b/spark/src/main/spark-4.x/org/apache/comet/shims/ShimBatchReader.scala similarity index 100% rename from common/src/main/spark-4.x/org/apache/comet/shims/ShimBatchReader.scala rename to spark/src/main/spark-4.x/org/apache/comet/shims/ShimBatchReader.scala diff --git a/common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala b/spark/src/main/spark-4.x/org/apache/comet/shims/ShimCometConf.scala similarity index 68% rename from common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala rename to spark/src/main/spark-4.x/org/apache/comet/shims/ShimCometConf.scala index 6147c18ddb..e89b37d604 100644 --- a/common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala +++ b/spark/src/main/spark-4.x/org/apache/comet/shims/ShimCometConf.scala @@ -20,5 +20,12 @@ package org.apache.comet.shims trait ShimCometConf { - protected val COMET_SCHEMA_EVOLUTION_ENABLED_DEFAULT = false + + /** + * Whether Comet's Parquet scan paths allow widening type promotions (e.g. INT32 → INT64, FLOAT + * → DOUBLE, INT32 → DOUBLE). Spark 4.x's vectorized reader accepts these by default. Reads from + * the deprecated `spark.comet.schemaEvolution.enabled` SQL conf were removed in favor of this + * per-version constant; see #4298. + */ + val COMET_SCHEMA_EVOLUTION_ENABLED: Boolean = true } diff --git a/common/src/main/spark-4.x/org/apache/comet/shims/ShimFileFormat.scala b/spark/src/main/spark-4.x/org/apache/comet/shims/ShimFileFormat.scala similarity index 100% rename from common/src/main/spark-4.x/org/apache/comet/shims/ShimFileFormat.scala rename to spark/src/main/spark-4.x/org/apache/comet/shims/ShimFileFormat.scala diff --git a/common/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala b/spark/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala similarity index 100% rename from common/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala rename to spark/src/main/spark-4.x/org/apache/spark/sql/comet/shims/ShimTaskMetrics.scala diff --git a/common/src/test/java/org/apache/comet/parquet/TestColumnReader.java b/spark/src/test/java/org/apache/comet/parquet/TestColumnReader.java similarity index 100% rename from common/src/test/java/org/apache/comet/parquet/TestColumnReader.java rename to spark/src/test/java/org/apache/comet/parquet/TestColumnReader.java diff --git a/spark/src/test/resources/sql-tests/expressions/array/posexplode.sql b/spark/src/test/resources/sql-tests/expressions/array/posexplode.sql new file mode 100644 index 0000000000..20b0547ebe --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/array/posexplode.sql @@ -0,0 +1,100 @@ +-- 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. + +-- posexplode_outer is gated behind allowIncompatible=true (DataFusion #19053). +-- Setting it at file scope does not affect the non-outer cases. +-- Config: spark.comet.operator.GenerateExec.allowIncompatible=true + +statement +CREATE TABLE test_posexplode_int(id int, arr array) USING parquet + +statement +INSERT INTO test_posexplode_int VALUES + (1, array(10, 20, 30)), + (2, array(40, 50)), + (3, array(60)), + (4, array()), + (5, NULL) + +-- basic posexplode over an int array column +query +SELECT id, posexplode(arr) FROM test_posexplode_int + +-- posexplode with explicit pos/value aliasing +query +SELECT id, pos, value FROM test_posexplode_int LATERAL VIEW posexplode(arr) p AS pos, value + +-- posexplode_outer keeps rows whose array is NULL. Empty arrays are excluded by +-- the WHERE clause because DataFusion #19053 drops empty arrays under +-- preserve_nulls=true; that is the documented Incompatible behavior. +query +SELECT id, posexplode_outer(arr) FROM test_posexplode_int WHERE id != 4 + +-- posexplode of a literal array (constant folding is disabled by the test runner) +query +SELECT id, posexplode(array(100, 200, 300)) FROM test_posexplode_int WHERE id = 1 + +statement +CREATE TABLE test_posexplode_str(id int, arr array) USING parquet + +statement +INSERT INTO test_posexplode_str VALUES + (1, array('a', 'b', 'c')), + (2, array('d', 'e')), + (3, array('f')) + +-- posexplode over a string array +query +SELECT id, posexplode(arr) FROM test_posexplode_str + +statement +CREATE TABLE test_posexplode_nullable(id int, arr array) USING parquet + +statement +INSERT INTO test_posexplode_nullable VALUES + (1, array(1, NULL, 3)), + (2, array(NULL, 5)), + (3, array(6)) + +-- posexplode preserves null elements within the array +query +SELECT id, posexplode(arr) FROM test_posexplode_nullable + +statement +CREATE TABLE test_posexplode_struct(id int, arr array>) USING parquet + +statement +INSERT INTO test_posexplode_struct VALUES + (1, array(named_struct('v1', 10, 'v2', 'a'), named_struct('v1', 20, 'v2', 'b'))), + (2, array(named_struct('v1', 30, 'v2', 'c'))), + (3, array()) + +-- posexplode over an array of structs, then project struct fields out of the unnested column +query +SELECT id, pos, value.v1 AS v1, value.v2 AS v2 FROM test_posexplode_struct LATERAL VIEW posexplode(arr) p AS pos, value + +statement +CREATE TABLE test_posexplode_map(id int, m map) USING parquet + +statement +INSERT INTO test_posexplode_map VALUES + (1, map('a', 1, 'b', 2)), + (2, map('c', 3)) + +-- posexplode over a map falls back to Spark (Comet only supports array inputs) +query expect_fallback(size does not support map inputs) +SELECT id, posexplode(m) FROM test_posexplode_map diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/convert_timezone.sql b/spark/src/test/resources/sql-tests/expressions/datetime/convert_timezone.sql new file mode 100644 index 0000000000..6c9a8947b5 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/convert_timezone.sql @@ -0,0 +1,49 @@ +-- 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. + +-- Config: spark.comet.expression.ConvertTimezone.allowIncompatible=true +-- ConfigMatrix: spark.sql.session.timeZone=UTC,America/Los_Angeles + +statement +CREATE TABLE test_convert_timezone(ts timestamp_ntz, src string, tgt string) USING parquet + +statement +INSERT INTO test_convert_timezone VALUES + (timestamp_ntz'2021-12-06 08:00:00', 'UTC', 'America/Los_Angeles'), + (timestamp_ntz'2021-07-01 12:00:00', 'America/New_York', 'Asia/Tokyo'), + (timestamp_ntz'2023-01-15 09:30:00', 'America/Los_Angeles', 'UTC'), + (NULL, 'UTC', 'Asia/Tokyo'), + (timestamp_ntz'2021-12-06 08:00:00', NULL, 'Asia/Tokyo'), + (timestamp_ntz'2021-12-06 08:00:00', 'UTC', NULL) + +query +SELECT convert_timezone('UTC', 'America/Los_Angeles', timestamp_ntz'2021-12-06 08:00:00') + +query +SELECT convert_timezone('Asia/Tokyo', 'Europe/Berlin', timestamp_ntz'2021-12-06 12:00:00') + +query +SELECT convert_timezone('America/Los_Angeles', 'Asia/Tokyo', timestamp_ntz'2023-01-15 20:00:00') + +query +SELECT convert_timezone(CAST(NULL AS STRING), 'Asia/Tokyo', timestamp_ntz'2021-12-06 08:00:00') + +query +SELECT convert_timezone('UTC', CAST(NULL AS STRING), timestamp_ntz'2021-12-06 08:00:00') + +query +SELECT convert_timezone(src, tgt, ts) FROM test_convert_timezone diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/current_timezone.sql b/spark/src/test/resources/sql-tests/expressions/datetime/current_timezone.sql new file mode 100644 index 0000000000..e5ca98f102 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/current_timezone.sql @@ -0,0 +1,40 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +statement +CREATE TABLE test_current_timezone(id INT) USING parquet + +statement +INSERT INTO test_current_timezone VALUES (1), (2), (3), (4), (NULL) + +query +SELECT current_timezone() + +query +SELECT current_timezone() IS NOT NULL + +query +SELECT current_timezone() = current_timezone() + +query +SELECT length(current_timezone()) > 0 + +query +SELECT id, current_timezone() IS NOT NULL FROM test_current_timezone + +query +SELECT id FROM test_current_timezone WHERE current_timezone() = current_timezone() diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/from_utc_timestamp.sql b/spark/src/test/resources/sql-tests/expressions/datetime/from_utc_timestamp.sql new file mode 100644 index 0000000000..55491c9ab7 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/from_utc_timestamp.sql @@ -0,0 +1,72 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +-- The result of from_utc_timestamp is a shift of the underlying microsecond +-- value, so it must not depend on the session timezone. Verify across two +-- representative session zones. +-- ConfigMatrix: spark.sql.session.timeZone=UTC,America/Los_Angeles + +-- CometFromUTCTimestamp is marked Incompatible because Comet's native timezone +-- parser does not accept Spark's legacy timezone forms; enable allowIncompatible +-- to force the native path for the timezones covered here. +-- Config: spark.comet.expression.FromUTCTimestamp.allowIncompatible=true + +statement +CREATE TABLE test_from_utc_timestamp(ts timestamp, tz string) USING parquet + +-- Includes a summer and a winter row for the LA timezone so that both DST +-- branches are exercised. The third row uses a fixed-offset tz to match the +-- form Spark's own DateFunctionsSuite covers via CEST = +02:00. +statement +INSERT INTO test_from_utc_timestamp VALUES + (timestamp('2015-07-24 00:00:00'), 'America/Los_Angeles'), + (timestamp('2015-01-24 00:00:00'), 'America/Los_Angeles'), + (timestamp('2024-06-15 10:30:45'), '+02:00'), + (timestamp('2024-01-01 00:00:00'), 'Asia/Seoul'), + (timestamp('1969-12-31 23:59:59'), 'UTC'), + (NULL, 'UTC'), + (timestamp('2024-06-15 10:30:45'), NULL), + (NULL, NULL) + +-- column timestamp, literal IANA timezone +query +SELECT from_utc_timestamp(ts, 'America/Los_Angeles') FROM test_from_utc_timestamp + +query +SELECT from_utc_timestamp(ts, 'Asia/Seoul') FROM test_from_utc_timestamp + +query +SELECT from_utc_timestamp(ts, 'UTC') FROM test_from_utc_timestamp + +-- column timestamp, literal fixed-offset timezone +query +SELECT from_utc_timestamp(ts, '+02:00') FROM test_from_utc_timestamp + +-- column timestamp, column timezone (mix of IANA and fixed-offset values) +query +SELECT from_utc_timestamp(ts, tz) FROM test_from_utc_timestamp + +-- literal arguments +query +SELECT from_utc_timestamp(timestamp('2017-07-14 02:40:00'), 'Etc/GMT-1') + +query +SELECT from_utc_timestamp(timestamp('2016-08-31 00:00:00'), 'Asia/Seoul') + +-- null handling +query +SELECT from_utc_timestamp(NULL, 'UTC'), from_utc_timestamp(timestamp('2024-01-01 00:00:00'), NULL), from_utc_timestamp(NULL, NULL) diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/local_timestamp.sql b/spark/src/test/resources/sql-tests/expressions/datetime/local_timestamp.sql new file mode 100644 index 0000000000..c8ee42a5e8 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/local_timestamp.sql @@ -0,0 +1,34 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +statement +CREATE TABLE test_local_timestamp(id int) USING parquet + +statement +INSERT INTO test_local_timestamp VALUES (1), (2), (3), (4), (NULL) + +query +SELECT localtimestamp() IS NOT NULL + +query +SELECT id, localtimestamp() IS NOT NULL FROM test_local_timestamp + +query +SELECT id FROM test_local_timestamp WHERE localtimestamp() = localtimestamp() + +query +SELECT year(localtimestamp()) = year(current_date()) diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/make_time.sql b/spark/src/test/resources/sql-tests/expressions/datetime/make_time.sql new file mode 100644 index 0000000000..8270e8f322 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/make_time.sql @@ -0,0 +1,141 @@ +-- 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. + +-- MinSparkVersion: 4.1 +-- Config: spark.sql.timeType.enabled=true + +statement +CREATE TABLE test_make_time(hours int, minutes int, secs decimal(16,6)) USING parquet + +statement +INSERT INTO test_make_time VALUES + (0, 0, 0.000000), + (12, 30, 45.123456), + (23, 59, 59.999999), + (1, 2, 3.500000), + (0, 0, 0.000001), + (NULL, 0, 0.000000), + (12, NULL, 30.000000), + (12, 30, NULL), + (NULL, NULL, NULL) + +-- column arguments (spark_answer_only: shuffle does not support TimeType yet; TODO: promote to +-- full native-verification once SPARK-51779 lands) +query spark_answer_only +SELECT hours, minutes, secs, make_time(hours, minutes, secs) FROM test_make_time ORDER BY hours, minutes, secs + +-- literal hour, column minutes and secs (spark_answer_only: shuffle does not support TimeType yet) +query spark_answer_only +SELECT make_time(10, minutes, secs) FROM test_make_time ORDER BY minutes, secs + +-- column hours, literal minutes and secs (spark_answer_only: shuffle does not support TimeType yet) +query spark_answer_only +SELECT make_time(hours, 15, 30.5) FROM test_make_time ORDER BY hours + +-- all literals +query +SELECT make_time(0, 0, 0) + +query +SELECT make_time(12, 30, 45.123456) + +query +SELECT make_time(23, 59, 59.999999) + +-- midnight +query +SELECT make_time(0, 0, 0.0) + +-- one microsecond after midnight +query +SELECT make_time(0, 0, 0.000001) + +-- end of day +query +SELECT make_time(23, 59, 59.999999) + +-- null handling with literals +query +SELECT make_time(NULL, 0, 0) + +query +SELECT make_time(12, NULL, 0) + +query +SELECT make_time(12, 30, NULL) + +-- integer seconds (implicit cast to decimal) +query +SELECT make_time(10, 20, 30) + +query +SELECT make_time(1, 2, 0) + +-- boundary valid values +query +SELECT make_time(0, 0, 0) + +query +SELECT make_time(23, 0, 0) + +query +SELECT make_time(0, 59, 0) + +query +SELECT make_time(0, 0, 59.999999) + +-- invalid hours - should throw error +query expect_error(HourOfDay) +SELECT make_time(24, 0, 0) + +query expect_error(HourOfDay) +SELECT make_time(25, 2, 23.5) + +query expect_error(HourOfDay) +SELECT make_time(-1, 0, 0) + +-- invalid minutes - should throw error +query expect_error(MinuteOfHour) +SELECT make_time(12, 60, 0) + +query expect_error(MinuteOfHour) +SELECT make_time(23, -1, 23.5) + +-- invalid seconds - should throw error +query expect_error(SecondOfMinute) +SELECT make_time(12, 30, 60.0) + +query expect_error(SecondOfMinute) +SELECT make_time(23, 12, 100.5) + +query expect_error(SecondOfMinute) +SELECT make_time(0, 0, -1.0) + +-- overflow seconds +query expect_error(SecondOfMinute) +SELECT make_time(1, 18, 4294967297.999999) + +-- time literal in comparison (make_time with all literals is constant-folded to a Time literal) +query +SELECT make_time(12, 30, 0) > make_time(11, 0, 0) + +query +SELECT make_time(0, 0, 0) = make_time(0, 0, 0) + +-- current_time() is foldable and produces a Time literal +query +SELECT current_time() IS NOT NULL diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/to_time.sql b/spark/src/test/resources/sql-tests/expressions/datetime/to_time.sql new file mode 100644 index 0000000000..b3ac439fd8 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/to_time.sql @@ -0,0 +1,260 @@ +-- 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. + +-- MinSparkVersion: 4.1 +-- Config: spark.sql.timeType.enabled=true + +statement +CREATE TABLE test_to_time(s STRING) USING parquet + +statement +INSERT INTO test_to_time VALUES + ('00:00'), + ('12:30'), + ('23:59'), + ('12:30:45'), + ('00:00:00'), + ('23:59:59'), + ('00:00:00.1'), + ('00:00:00.001'), + ('00:00:00.000001'), + ('00:00:00.1234567'), + ('23:59:59.999999'), + ('1:2:3'), + ('1:2:3.04'), + ('T12:30:45'), + ('T1:02:3.04'), + ('12:00:00 AM'), + ('1:00:00 AM'), + ('12:00:00 PM'), + ('1:00:00 PM'), + ('11:59:59 PM'), + ('12:59:59.999999 PM'), + ('12:00:00AM'), + ('1:00:00PM'), + (' 12:30:45'), + (' 12:30 PM'), + (NULL) + +-- column argument: basic time formats (spark_answer_only: shuffle does not support TimeType yet; +-- TODO: promote to full native-verification once SPARK-51779 lands) +query spark_answer_only +SELECT s, to_time(s) FROM test_to_time ORDER BY s + +-- literal HH:mm +query +SELECT to_time('00:00') + +query +SELECT to_time('12:30') + +query +SELECT to_time('23:59') + +-- literal HH:mm:ss +query +SELECT to_time('12:30:45') + +query +SELECT to_time('00:00:00') + +query +SELECT to_time('23:59:59') + +-- fractional seconds +query +SELECT to_time('00:00:00.1') + +query +SELECT to_time('00:00:00.001') + +query +SELECT to_time('00:00:00.000001') + +query +SELECT to_time('23:59:59.999999') + +-- more than 6 fractional digits (truncated to microseconds) +query +SELECT to_time('00:00:00.1234567') + +-- single digit hour/min/sec +query +SELECT to_time('1:2:3') + +query +SELECT to_time('1:2:3.04') + +-- T-prefix +query +SELECT to_time('T12:30:45') + +query +SELECT to_time('T1:02:3.04') + +-- AM/PM +query +SELECT to_time('12:00:00 AM') + +query +SELECT to_time('1:00:00 AM') + +query +SELECT to_time('11:59:59 AM') + +query +SELECT to_time('12:00:00 PM') + +query +SELECT to_time('1:00:00 PM') + +query +SELECT to_time('11:59:59 PM') + +-- AM/PM case insensitive +query +SELECT to_time('12:00:00 am') + +query +SELECT to_time('12:00:00 pm') + +-- AM/PM without space +query +SELECT to_time('12:00:00AM') + +query +SELECT to_time('1:00:00PM') + +-- AM/PM with fractional seconds +query +SELECT to_time('12:59:59.999999 PM') + +-- null input +query +SELECT to_time(NULL) + +-- trailing whitespace +query +SELECT to_time('12:30:45 ') + +-- leading whitespace +query +SELECT to_time(' 12:30:45') + +query +SELECT to_time(' 12:30:45') + +query +SELECT to_time(' 12:30:45 ') + +query +SELECT to_time(' 12:30') + +query +SELECT to_time(' 12:30 PM') + +query +SELECT to_time(' 1:00:00 PM') + +-- leading tab and newline +query +SELECT to_time('\t12:30:45') + +query +SELECT to_time('\n12:30:45') + +-- leading whitespace with T-prefix is rejected by Spark +query expect_error(cannot be parsed to a TIME value) +SELECT to_time(' T12:30:45') + +-- invalid inputs - should throw error with to_time +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('XYZ') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('24:00:00') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('23:60:00') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('23:00:60') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('120000') + +-- invalid AM/PM - should throw error +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('0:00:00 AM') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('13:00:00 AM') + +query expect_error(cannot be parsed to a TIME value) +SELECT to_time('13:00:00 PM') + +-- try_to_time: returns null for invalid inputs +query +SELECT try_to_time('12:30:45') + +query +SELECT try_to_time('') + +query +SELECT try_to_time('XYZ') + +query +SELECT try_to_time('24:00:00') + +query +SELECT try_to_time('23:60:00') + +query +SELECT try_to_time(NULL) + +query +SELECT try_to_time('0:00:00 AM') + +query +SELECT try_to_time('13:00:00 PM') + +-- try_to_time: leading whitespace parses successfully +query +SELECT try_to_time(' 12:30:45') + +query +SELECT try_to_time(' 1:00:00 PM') + +-- to_time with format pattern falls back to Spark (not supported natively) +query expect_fallback(invoke is not supported) +SELECT to_time('12:30:45', 'HH:mm:ss') + +statement +CREATE TABLE test_to_time_col_fmt(s STRING, f STRING) USING parquet + +statement +INSERT INTO test_to_time_col_fmt VALUES + ('14.30.00', 'HH.mm.ss'), + ('1230', 'HHmm') + +-- A non-foldable format column should fall back to Spark because Comet does +-- not implement the format-pattern variant of to_time. +query expect_fallback(invoke is not supported) +SELECT to_time(s, f) FROM test_to_time_col_fmt diff --git a/spark/src/test/resources/sql-tests/expressions/datetime/to_utc_timestamp.sql b/spark/src/test/resources/sql-tests/expressions/datetime/to_utc_timestamp.sql new file mode 100644 index 0000000000..044bab2c07 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/datetime/to_utc_timestamp.sql @@ -0,0 +1,72 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +-- The result of to_utc_timestamp is a shift of the underlying microsecond +-- value, so it must not depend on the session timezone. Verify across two +-- representative session zones. +-- ConfigMatrix: spark.sql.session.timeZone=UTC,America/Los_Angeles + +-- CometToUTCTimestamp is marked Incompatible because Comet's native timezone +-- parser does not accept Spark's legacy timezone forms; enable allowIncompatible +-- to force the native path for the timezones covered here. +-- Config: spark.comet.expression.ToUTCTimestamp.allowIncompatible=true + +statement +CREATE TABLE test_to_utc_timestamp(ts timestamp, tz string) USING parquet + +-- Includes a summer and a winter row for the LA timezone so that both DST +-- branches are exercised. The third row uses a fixed-offset tz to match the +-- form Spark's own DateFunctionsSuite covers via CEST = +02:00. +statement +INSERT INTO test_to_utc_timestamp VALUES + (timestamp('2015-07-24 00:00:00'), 'America/Los_Angeles'), + (timestamp('2015-01-24 00:00:00'), 'America/Los_Angeles'), + (timestamp('2024-06-15 10:30:45'), '+02:00'), + (timestamp('2024-01-01 00:00:00'), 'Asia/Seoul'), + (timestamp('1969-12-31 23:59:59'), 'UTC'), + (NULL, 'UTC'), + (timestamp('2024-06-15 10:30:45'), NULL), + (NULL, NULL) + +-- column timestamp, literal IANA timezone +query +SELECT to_utc_timestamp(ts, 'America/Los_Angeles') FROM test_to_utc_timestamp + +query +SELECT to_utc_timestamp(ts, 'Asia/Seoul') FROM test_to_utc_timestamp + +query +SELECT to_utc_timestamp(ts, 'UTC') FROM test_to_utc_timestamp + +-- column timestamp, literal fixed-offset timezone +query +SELECT to_utc_timestamp(ts, '+02:00') FROM test_to_utc_timestamp + +-- column timestamp, column timezone (mix of IANA and fixed-offset values) +query +SELECT to_utc_timestamp(ts, tz) FROM test_to_utc_timestamp + +-- literal arguments +query +SELECT to_utc_timestamp(timestamp('2017-07-14 02:40:00'), 'Etc/GMT-1') + +query +SELECT to_utc_timestamp(timestamp('2016-08-31 00:00:00'), 'Asia/Seoul') + +-- null handling +query +SELECT to_utc_timestamp(NULL, 'UTC'), to_utc_timestamp(timestamp('2024-01-01 00:00:00'), NULL), to_utc_timestamp(NULL, NULL) diff --git a/spark/src/test/resources/sql-tests/expressions/math/csc.sql b/spark/src/test/resources/sql-tests/expressions/math/csc.sql new file mode 100644 index 0000000000..1b786c6c0b --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/math/csc.sql @@ -0,0 +1,25 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +statement +CREATE TABLE test_csc(d double) USING parquet + +statement +INSERT INTO test_csc VALUES (0.0), (-0.0), (1.5707963267948966), (-1.5707963267948966), (3.141592653589793), (NULL), (cast('NaN' as double)), (cast('Infinity' as double)), (cast('-Infinity' as double)) + +query tolerance=1e-6 +SELECT csc(d) FROM test_csc diff --git a/spark/src/test/resources/sql-tests/expressions/math/factorial.sql b/spark/src/test/resources/sql-tests/expressions/math/factorial.sql new file mode 100644 index 0000000000..612a3af4ec --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/math/factorial.sql @@ -0,0 +1,39 @@ +-- 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. + +-- ConfigMatrix: parquet.enable.dictionary=false,true + +statement +CREATE TABLE test_factorial(i int) USING parquet + +statement +INSERT INTO test_factorial VALUES + (0), + (1), + (5), + (12), + (20), + (21), + (-1), + (-5), + (100), + (NULL), + (2147483647), + (-2147483648) + +query +SELECT i, factorial(i) FROM test_factorial diff --git a/spark/src/test/resources/sql-tests/expressions/math/rint.sql b/spark/src/test/resources/sql-tests/expressions/math/rint.sql new file mode 100644 index 0000000000..7af2d1e7e0 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/math/rint.sql @@ -0,0 +1,60 @@ +-- 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. + +-- ConfigMatrix: parquet.enable.dictionary=false,true + +-- Spark's Rint extends UnaryMathExpression with inputTypes = Seq(DoubleType). +-- It returns the double value closest to the argument equal to a mathematical integer +-- (Java's Math.rint, IEEE 754 round-half-to-even / banker's rounding). + +statement +CREATE TABLE test_rint(v double) USING parquet + +statement +INSERT INTO test_rint VALUES + (0.0), + (-0.0), + (1.0), + (-1.0), + (0.4), + (0.5), + (0.6), + (1.5), + (2.5), + (3.5), + (-0.4), + (-0.5), + (-0.6), + (-1.5), + (-2.5), + (-3.5), + (12.3456), + (-12.3456), + (1.7976931348623157E308), + (-1.7976931348623157E308), + (4.9E-324), + (cast('NaN' as double)), + (cast('Infinity' as double)), + (cast('-Infinity' as double)), + (NULL) + +query +SELECT rint(v) FROM test_rint + +-- column with arithmetic +query +SELECT rint(v + 0.5) FROM test_rint diff --git a/spark/src/test/resources/sql-tests/expressions/math/sec.sql b/spark/src/test/resources/sql-tests/expressions/math/sec.sql new file mode 100644 index 0000000000..4a9494fe09 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/math/sec.sql @@ -0,0 +1,25 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +statement +CREATE TABLE test_sec(d double) USING parquet + +statement +INSERT INTO test_sec VALUES (0.0), (-0.0), (1.5707963267948966), (-1.5707963267948966), (3.141592653589793), (NULL), (cast('NaN' as double)), (cast('Infinity' as double)), (cast('-Infinity' as double)) + +query tolerance=1e-6 +SELECT sec(d) FROM test_sec diff --git a/spark/src/test/resources/sql-tests/expressions/string/decode.sql b/spark/src/test/resources/sql-tests/expressions/string/decode.sql new file mode 100644 index 0000000000..45aeaacee4 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/string/decode.sql @@ -0,0 +1,104 @@ +-- 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. + +-- Tests for the SQL `decode` function. +-- +-- Spark's `decode` is overloaded: +-- * decode(bin, charset) -> StringDecode (charset binary->string) +-- * decode(expr, search, result, ..., [default]) -> CaseWhen with EqualNullSafe branches +-- +-- The Oracle-style form is implemented in Spark via the RuntimeReplaceable trait, so by the +-- time Comet sees the plan the wrapper has already been replaced with CaseWhen and Comet +-- handles it through its existing CaseWhen + EqualNullSafe serde. +-- +-- The 2-arg charset form lowers to a cast(binary, string) inside Comet's stringDecode +-- handler, but only when the charset is 'utf-8' (case-insensitive). Other charsets fall +-- back to Spark JVM execution. + +-- =========================================================================== +-- Charset form: decode(bin, charset) for UTF-8 (the supported native path) +-- =========================================================================== + +statement +CREATE TABLE test_decode_utf8(b binary) USING parquet + +statement +INSERT INTO test_decode_utf8 VALUES (CAST('hello' AS BINARY)), (CAST('world' AS BINARY)), (CAST('' AS BINARY)), + (CAST('café' AS BINARY)), (NULL) + +query +SELECT decode(b, 'utf-8') FROM test_decode_utf8 + +query +SELECT decode(b, 'UTF-8') FROM test_decode_utf8 + +query +SELECT decode(CAST('hello' AS BINARY), 'utf-8'), decode(CAST('' AS BINARY), 'utf-8'), decode(NULL, 'utf-8') + +-- Charset form: non-UTF-8 + +statement +CREATE TABLE test_decode_charset_safe(b binary) USING parquet + +statement +INSERT INTO test_decode_charset_safe VALUES (CAST('ab' AS BINARY)), (CAST('abcd' AS BINARY)), (CAST('' AS BINARY)), (NULL) + +query expect_fallback(Comet only supports decoding with 'utf-8'.) +SELECT decode(b, 'UTF-16BE') FROM test_decode_charset_safe + +query expect_fallback(Comet only supports decoding with 'utf-8'.) +SELECT decode(b, 'US-ASCII') FROM test_decode_charset_safe + +query expect_fallback(Comet only supports decoding with 'utf-8'.) +SELECT decode(b, 'ISO-8859-1') FROM test_decode_utf8 + + +statement +CREATE TABLE test_decode_oracle(status string, code int) USING parquet + +statement +INSERT INTO test_decode_oracle VALUES ('A', 1), ('I', 2), ('X', 3), (NULL, 4), ('A', NULL) + +query +SELECT decode(status, 'A', 'Active', 'I', 'Inactive', 'Other') FROM test_decode_oracle + +query +SELECT decode(status, 'A', 'Active', 'I', 'Inactive') FROM test_decode_oracle + +query +SELECT decode(code, 1, 'one', 2, 'two', 3, 'three', 'unknown') FROM test_decode_oracle + +query +SELECT decode(code, 1, 'one', 2, 'two') FROM test_decode_oracle + +query +SELECT decode(status, 'A', 'has-A', NULL, 'is-null', 'other') FROM test_decode_oracle + +query +SELECT decode(status, 'A', 'Active') FROM test_decode_oracle + +query +SELECT decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 'Other') + +query +SELECT decode(6, 1, 'Southlake', 2, 'San Francisco', 'Other') + +query +SELECT decode(6, 1, 'Southlake', 2, 'San Francisco') + +query +SELECT decode(NULL, 6, 'Spark', NULL, 'SQL', 4, 'rocks') diff --git a/spark/src/test/resources/sql-tests/expressions/url/parse_url.sql b/spark/src/test/resources/sql-tests/expressions/url/parse_url.sql new file mode 100644 index 0000000000..b6882ec19b --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/url/parse_url.sql @@ -0,0 +1,37 @@ +-- Licensed to the Apache Software Foundation (ASF) under one +-- or more contributor license agreements. See the NOTICE file +-- distributed with this work for additional information +-- regarding copyright ownership. The ASF licenses this file +-- to you under the Apache License, Version 2.0 (the +-- "License"); you may not use this file except in compliance +-- with the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, +-- software distributed under the License is distributed on an +-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +-- KIND, either express or implied. See the License for the +-- specific language governing permissions and limitations +-- under the License. + +statement +CREATE TABLE test_parse_url(url string) USING parquet + +statement +INSERT INTO test_parse_url VALUES + ('http://spark.apache.org/path?query=1'), + ('https://user:pass@host:8080/path?k=v#ref'), + (NULL) + +query expect_fallback(not fully compatible with Spark) +SELECT parse_url(url, 'HOST') FROM test_parse_url + +query expect_fallback(not fully compatible with Spark) +SELECT parse_url(url, 'PATH') FROM test_parse_url + +query expect_fallback(not fully compatible with Spark) +SELECT parse_url(url, 'QUERY') FROM test_parse_url + +query expect_fallback(not fully compatible with Spark) +SELECT parse_url(url, 'QUERY', 'k') FROM test_parse_url diff --git a/spark/src/test/resources/sql-tests/expressions/url/parse_url_ansi.sql b/spark/src/test/resources/sql-tests/expressions/url/parse_url_ansi.sql new file mode 100644 index 0000000000..1e71031f41 --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/url/parse_url_ansi.sql @@ -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. + +-- Test parse_url() in ANSI mode (failOnError=true -> native "parse_url" path) +-- Config: spark.sql.ansi.enabled=true +-- Config: spark.comet.expression.ParseUrl.allowIncompatible=true + +-- valid URLs should work identically in ANSI mode +query +SELECT parse_url('http://example.com/path?foo=bar', 'HOST') + +query +SELECT parse_url('http://example.com/path?foo=bar', 'PATH') + +query +SELECT parse_url('http://example.com/path?foo=bar', 'QUERY', 'foo') + +query +SELECT parse_url('https://user:pass@host:8080/p?k=v#ref', 'AUTHORITY') + +query +SELECT parse_url('https://user:pass@host:8080/p?k=v#ref', 'USERINFO') + +query +SELECT parse_url('https://user:pass@host:8080/p?k=v#ref', 'REF') + +-- NULL inputs still return NULL in ANSI mode +query +SELECT parse_url(NULL, 'HOST') + +-- invalid URL throws in ANSI mode (native returns NULL instead of throwing) +query ignore(known divergence: native parse_url does not throw INVALID_URL for malformed URLs) +SELECT parse_url('not a url at all', 'HOST') + +query ignore(known divergence: native parse_url does not throw INVALID_URL for malformed URLs) +SELECT parse_url('://missing-scheme', 'HOST') + +query ignore(known divergence: native parse_url does not throw INVALID_URL for malformed URLs) +SELECT parse_url('', 'HOST') diff --git a/spark/src/test/resources/sql-tests/expressions/url/parse_url_enabled.sql b/spark/src/test/resources/sql-tests/expressions/url/parse_url_enabled.sql new file mode 100644 index 0000000000..1bde241b4d --- /dev/null +++ b/spark/src/test/resources/sql-tests/expressions/url/parse_url_enabled.sql @@ -0,0 +1,140 @@ +-- 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. + +-- Test parse_url() with allowIncompatible enabled (native execution) +-- Config: spark.comet.expression.ParseUrl.allowIncompatible=true + +statement +CREATE TABLE test_parse_url_enabled(url string) USING parquet + +statement +INSERT INTO test_parse_url_enabled VALUES + ('http://spark.apache.org/path?query=1'), + ('https://user:pass@host:8080/path?k=v#ref'), + ('http://example.com/path?a=1&b=2&a=3'), + ('ftp://ftp.example.com/dir/file.txt'), + (NULL) + +-- HOST +query +SELECT parse_url(url, 'HOST') FROM test_parse_url_enabled + +-- PATH +query +SELECT parse_url(url, 'PATH') FROM test_parse_url_enabled + +-- QUERY (no key) +query +SELECT parse_url(url, 'QUERY') FROM test_parse_url_enabled + +-- QUERY with key +query +SELECT parse_url(url, 'QUERY', 'k') FROM test_parse_url_enabled + +-- PROTOCOL +query +SELECT parse_url(url, 'PROTOCOL') FROM test_parse_url_enabled + +-- REF (fragment) +query +SELECT parse_url(url, 'REF') FROM test_parse_url_enabled + +-- AUTHORITY +query +SELECT parse_url(url, 'AUTHORITY') FROM test_parse_url_enabled + +-- USERINFO +query +SELECT parse_url(url, 'USERINFO') FROM test_parse_url_enabled + +-- FILE +query +SELECT parse_url(url, 'FILE') FROM test_parse_url_enabled + +-- literal arguments +query +SELECT parse_url('http://example.com/path?foo=bar', 'HOST') + +query +SELECT parse_url('http://example.com/path?foo=bar', 'PATH') + +query +SELECT parse_url('http://example.com/path?foo=bar', 'QUERY') + +query +SELECT parse_url('http://example.com/path?foo=bar', 'QUERY', 'foo') + +query +SELECT parse_url('http://example.com/path?foo=bar', 'PROTOCOL') + +-- NULL handling +query +SELECT parse_url(NULL, 'HOST') + +query +SELECT parse_url('http://example.com', NULL) + +-- invalid part key +query +SELECT parse_url('http://example.com', 'INVALID') + +-- malformed URL returns NULL in non-ANSI mode (#7) +query +SELECT parse_url('not a url at all', 'HOST') + +query +SELECT parse_url('://missing-scheme', 'HOST') + +query +SELECT parse_url('', 'HOST') + +-- column-valued part key (#5) +statement +CREATE TABLE test_parse_url_parts(url string, part string, key string) USING parquet + +statement +INSERT INTO test_parse_url_parts VALUES + ('http://example.com/path?foo=bar', 'HOST', NULL), + ('http://example.com/path?foo=bar', 'PATH', NULL), + ('http://example.com/path?foo=bar', 'QUERY', 'foo'), + ('https://user:pw@host:9090/p?a=1#frag', 'REF', NULL), + ('https://user:pw@host:9090/p?a=1#frag', 'USERINFO', NULL) + +query +SELECT parse_url(url, part) FROM test_parse_url_parts + +query +SELECT parse_url(url, 'QUERY', key) FROM test_parse_url_parts WHERE key IS NOT NULL + +-- edge cases for known divergences (#6) +query +SELECT parse_url('http://example.com//double//slashes', 'PATH') + +query ignore(known divergence: native decodes percent-encoding in QUERY values) +SELECT parse_url('http://example.com/path?key=value%20encoded', 'QUERY', 'key') + +query +SELECT parse_url('http://example.com/path?', 'QUERY') + +query ignore(known divergence: native returns "/" for FILE when URL has no path) +SELECT parse_url('http://example.com#frag', 'FILE') + +query +SELECT parse_url('http://[::1]:8080/path', 'HOST') + +query +SELECT parse_url('http://example.com/path?a=1&a=2', 'QUERY', 'a') diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt deleted file mode 100644 index 3537e14dcf..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 32 out of 55 eligible operators (58%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt deleted file mode 100644 index 013bd790b1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,112 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 96 out of 104 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt deleted file mode 100644 index d7fde7133a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 30 out of 37 eligible operators (81%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt deleted file mode 100644 index 1ce50f8385..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,60 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 33 out of 53 eligible operators (62%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt similarity index 97% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt index e5ddbc5b9a..116913a30a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Expand diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt deleted file mode 100644 index fac88c8bc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt deleted file mode 100644 index 7a4afd3d2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,343 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery - -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt deleted file mode 100644 index f7a30e9f90..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometColumnarExchange - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometSort - : : +- CometColumnarExchange - : : +- Project - : : +- Filter - : : +- Window - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 57 eligible operators (78%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt deleted file mode 100644 index 7df6105d58..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,95 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- ReusedSubquery - +- ReusedSubquery - -Comet accelerated 75 out of 88 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt deleted file mode 100644 index 7190b29a9b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,116 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Subquery - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt deleted file mode 100644 index ee55822981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt deleted file mode 100644 index fac88c8bc7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 101 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33.native_iceberg_compat/extended.txt deleted file mode 100644 index 93698f3e40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,96 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 88 out of 92 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt deleted file mode 100644 index 2044df0e8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 63 out of 86 eligible operators (73%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56.native_iceberg_compat/extended.txt deleted file mode 100644 index a4d34c2e29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60.native_iceberg_compat/extended.txt deleted file mode 100644 index a4d34c2e29..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66.native_iceberg_compat/extended.txt deleted file mode 100644 index bbbbb2874f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,69 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - -Comet accelerated 62 out of 65 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2.native_iceberg_compat/extended.txt deleted file mode 100644 index 1292af6d8b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,60 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 57 out of 57 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5.native_iceberg_compat/extended.txt deleted file mode 100644 index 089c3142a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,108 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 97 out of 104 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54.native_iceberg_compat/extended.txt deleted file mode 100644 index 90a73147f9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,101 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometUnion - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- ReusedSubquery - +- ReusedSubquery - -Comet accelerated 81 out of 94 eligible operators (86%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt deleted file mode 100644 index b7c63811ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt deleted file mode 100644 index 72960fc267..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 34 out of 54 eligible operators (62%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt similarity index 96% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt index 5e8a5364b6..07af300183 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt deleted file mode 100644 index 0e6397bfc2..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt deleted file mode 100644 index eb85a80eb1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt deleted file mode 100644 index 385e0fdaad..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt deleted file mode 100644 index 466892ba9d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,469 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 424 out of 458 eligible operators (92%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt deleted file mode 100644 index 78557e7ffb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,339 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt deleted file mode 100644 index b8907128e8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt deleted file mode 100644 index c4efd6fb10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 39 out of 39 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt deleted file mode 100644 index 52ca742891..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt deleted file mode 100644 index 6ae79a435f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt deleted file mode 100644 index 9c22a76577..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt deleted file mode 100644 index 75511320a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,48 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 45 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt deleted file mode 100644 index 643e47b0c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt deleted file mode 100644 index 8da34310d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt deleted file mode 100644 index 831d892584..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt deleted file mode 100644 index 94af831deb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,146 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 127 out of 138 eligible operators (92%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt deleted file mode 100644 index 2cff47fb04..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,198 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometSort - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 177 out of 190 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt deleted file mode 100644 index fa983a36da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 66 out of 86 eligible operators (76%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt similarity index 92% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt index 8249b86a1e..4fd8252b21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/extended.txt @@ -1,9 +1,9 @@ Filter : +- Subquery -: +- HashAggregate +: +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate -: +- HashAggregate +: +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -44,10 +44,10 @@ Filter : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt deleted file mode 100644 index fa983a36da..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -Filter -: +- Subquery -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- HashAggregate -: +- Exchange -: +- HashAggregate -: +- Project -: +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] -: :- CometNativeColumnarToRow -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometSortMergeJoin -: : : : : :- CometSort -: : : : : : +- CometExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometSort -: : : : : +- CometExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -+- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 66 out of 86 eligible operators (76%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt similarity index 92% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt index 8249b86a1e..4fd8252b21 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/extended.txt @@ -1,9 +1,9 @@ Filter : +- Subquery -: +- HashAggregate +: +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate -: +- HashAggregate +: +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -44,10 +44,10 @@ Filter : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt deleted file mode 100644 index 52ca742891..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt deleted file mode 100644 index d868ceeeb3..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt deleted file mode 100644 index 8b6297f645..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt deleted file mode 100644 index 35dac9a909..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : :- CometNativeColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- BroadcastExchange -: : : : +- CometNativeColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- BroadcastExchange -: : : +- CometNativeColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : +- BroadcastExchange -: : +- CometNativeColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -+- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 54 out of 64 eligible operators (84%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt deleted file mode 100644 index 9ca9f72ab4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 55 out of 61 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt deleted file mode 100644 index 8b2f687037..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt deleted file mode 100644 index 3318935f4d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,126 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt deleted file mode 100644 index 60c49121a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt deleted file mode 100644 index 638a87ab14..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,97 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt deleted file mode 100644 index 18ed3e6335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt deleted file mode 100644 index 72960fc267..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 34 out of 54 eligible operators (62%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt similarity index 96% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt index 5e8a5364b6..07af300183 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt deleted file mode 100644 index e434c5ff76..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt deleted file mode 100644 index a92eb1903d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt deleted file mode 100644 index cb4d06350b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt deleted file mode 100644 index e8f621d32a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt deleted file mode 100644 index e8f621d32a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 56 out of 60 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt deleted file mode 100644 index 570f9bbcc9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 118 out of 126 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt deleted file mode 100644 index f9c807168e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,40 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 34 out of 36 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt deleted file mode 100644 index a3c8dd9568..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt deleted file mode 100644 index 1272f205e4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt deleted file mode 100644 index 259056f9f6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,63 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt deleted file mode 100644 index f8c94605ab..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,46 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- Filter - +- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 31 out of 41 eligible operators (75%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt similarity index 95% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt index 6ed69e641f..f95c69368f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45/extended.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt deleted file mode 100644 index 07093ae00b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt deleted file mode 100644 index 9691120d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt deleted file mode 100644 index c8325b49ca..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt deleted file mode 100644 index 0b4672eabb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt deleted file mode 100644 index 4b840c6a27..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt deleted file mode 100644 index aa0c4c1c38..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt deleted file mode 100644 index ea746c5fee..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt deleted file mode 100644 index 093505a6cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt deleted file mode 100644 index f4318de30f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,91 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- SubqueryBroadcast - : : : : : : : : +- BroadcastExchange - : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : :- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 75 out of 84 eligible operators (89%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt deleted file mode 100644 index e5122dc8d7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,21 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt deleted file mode 100644 index faef2d5263..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt deleted file mode 100644 index 5e8acff889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt deleted file mode 100644 index 81412250ec..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,115 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 98 out of 108 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt deleted file mode 100644 index 3594fb8035..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt deleted file mode 100644 index e3297b4d66..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt deleted file mode 100644 index faef2d5263..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,100 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt deleted file mode 100644 index 783de88195..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,88 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt deleted file mode 100644 index dc09a1507f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt deleted file mode 100644 index 093505a6cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt deleted file mode 100644 index a868670a16..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt deleted file mode 100644 index b0c27d6da7..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt deleted file mode 100644 index 46b1956f4e..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - -Comet accelerated 63 out of 66 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt deleted file mode 100644 index da0bef861d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,38 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt deleted file mode 100644 index 07093ae00b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,49 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt deleted file mode 100644 index e553dcb0a5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 49 out of 53 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt deleted file mode 100644 index 783b9a64bb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt deleted file mode 100644 index 730d37ec0a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 33 out of 52 eligible operators (63%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt similarity index 97% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt index c1d5342c8b..7fd7fa746f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt @@ -4,7 +4,7 @@ TakeOrderedAndProject +- CometNativeColumnarToRow +- CometSort +- CometColumnarExchange - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Expand diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt deleted file mode 100644 index 4de5ab6c3b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,53 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - -Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt deleted file mode 100644 index 22fde3b02c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt deleted file mode 100644 index 18ed3e6335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt deleted file mode 100644 index 922864baa1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt deleted file mode 100644 index ec53af4809..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt deleted file mode 100644 index 685ff36fed..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,47 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt deleted file mode 100644 index b2653c7068..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,116 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 93 out of 109 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt similarity index 98% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt index 72280ab0a2..5bffdf3688 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Expand diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt deleted file mode 100644 index 1b1e6d0cde..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometNativeColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt deleted file mode 100644 index f3e80e01cb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,39 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt deleted file mode 100644 index be5abf4913..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,52 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt deleted file mode 100644 index ed8d0deb6d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,131 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt deleted file mode 100644 index eb965c592c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,65 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt deleted file mode 100644 index 17e0a010c0..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - -Comet accelerated 28 out of 30 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt deleted file mode 100644 index 3e72e3d553..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,119 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 109 out of 114 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt deleted file mode 100644 index fcd8384c22..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,35 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - -Comet accelerated 32 out of 32 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt deleted file mode 100644 index f99808facc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 50 out of 52 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt deleted file mode 100644 index c0056e2382..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,32 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt deleted file mode 100644 index cb4d06350b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,70 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 62 out of 66 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt deleted file mode 100644 index f1e4c8679a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,216 +0,0 @@ -BroadcastNestedLoopJoin -:- BroadcastNestedLoopJoin -: :- BroadcastNestedLoopJoin -: : :- BroadcastNestedLoopJoin -: : : :- BroadcastNestedLoopJoin -: : : : :- BroadcastNestedLoopJoin -: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] -: : : : : : :- CometNativeColumnarToRow -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometExchange -: : : : : : : +- CometHashAggregate -: : : : : : : +- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : : :- CometProject -: : : : : : : : : : +- CometFilter -: : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : : +- CometBroadcastExchange -: : : : : : : : : +- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : : +- BroadcastExchange -: : : : : : +- CometNativeColumnarToRow -: : : : : : +- CometHashAggregate -: : : : : : +- CometExchange -: : : : : : +- CometHashAggregate -: : : : : : +- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometBroadcastHashJoin -: : : : : : : : :- CometProject -: : : : : : : : : +- CometFilter -: : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : : +- CometBroadcastExchange -: : : : : : : : +- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : : +- BroadcastExchange -: : : : : +- CometNativeColumnarToRow -: : : : : +- CometHashAggregate -: : : : : +- CometExchange -: : : : : +- CometHashAggregate -: : : : : +- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometBroadcastHashJoin -: : : : : : : :- CometProject -: : : : : : : : +- CometFilter -: : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : : +- CometBroadcastExchange -: : : : : : : +- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : : +- BroadcastExchange -: : : : +- CometNativeColumnarToRow -: : : : +- CometHashAggregate -: : : : +- CometExchange -: : : : +- CometHashAggregate -: : : : +- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometBroadcastHashJoin -: : : : : : :- CometProject -: : : : : : : +- CometFilter -: : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : : +- CometBroadcastExchange -: : : : : : +- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : : +- BroadcastExchange -: : : +- CometNativeColumnarToRow -: : : +- CometHashAggregate -: : : +- CometExchange -: : : +- CometHashAggregate -: : : +- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometBroadcastHashJoin -: : : : : :- CometProject -: : : : : : +- CometFilter -: : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : : +- CometBroadcastExchange -: : : : : +- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: : +- BroadcastExchange -: : +- CometNativeColumnarToRow -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometBroadcastHashJoin -: : : : :- CometProject -: : : : : +- CometFilter -: : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : : +- CometBroadcastExchange -: : : : +- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -: +- BroadcastExchange -: +- CometNativeColumnarToRow -: +- CometHashAggregate -: +- CometExchange -: +- CometHashAggregate -: +- CometProject -: +- CometBroadcastHashJoin -: :- CometProject -: : +- CometBroadcastHashJoin -: : :- CometProject -: : : +- CometBroadcastHashJoin -: : : :- CometProject -: : : : +- CometFilter -: : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: : : +- CometBroadcastExchange -: : : +- CometProject -: : : +- CometFilter -: : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -: : +- CometBroadcastExchange -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -: +- CometBroadcastExchange -: +- CometProject -: +- CometFilter -: +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -+- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt deleted file mode 100644 index 093505a6cc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt deleted file mode 100644 index e3614131fe..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,61 +0,0 @@ - Project [COMET: ] -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: :- ReusedSubquery -: :- Subquery -: : +- CometNativeColumnarToRow -: : +- CometProject -: : +- CometHashAggregate -: : +- CometExchange -: : +- CometHashAggregate -: : +- CometProject -: : +- CometFilter -: : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -: :- ReusedSubquery -: +- ReusedSubquery -+- CometNativeColumnarToRow - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 37 out of 53 eligible operators (69%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt deleted file mode 100644 index 3b1bd00423..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,55 +0,0 @@ -Project -+- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt deleted file mode 100644 index 961299b3d9..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,51 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - -Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt deleted file mode 100644 index 0fb6e6c158..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt deleted file mode 100644 index 79acf0e4eb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,24 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.reason - -Comet accelerated 21 out of 21 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt deleted file mode 100644 index a89558eb80..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,42 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 39 out of 39 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt deleted file mode 100644 index 9a692023ae..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,64 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometSortMergeJoin - : : : :- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometProject - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 61 out of 61 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt deleted file mode 100644 index 0f07a6473f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,27 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 24 out of 24 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt deleted file mode 100644 index 4aa20a3755..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,37 +0,0 @@ -CometNativeColumnarToRow -+- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt deleted file mode 100644 index a18137ef10..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,34 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt deleted file mode 100644 index 47e040af97..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.ship_mode - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 28 out of 28 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt deleted file mode 100644 index 2afcab203a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,298 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometColumnarExchange - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 262 out of 285 eligible operators (91%). Final plan contains 11 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt deleted file mode 100644 index ff6d42786b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,171 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window - : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window - +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 156 eligible operators (62%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt similarity index 95% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt index 44999fda71..bf78b909ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject +- CometColumnarExchange +- HashAggregate +- Union - :- HashAggregate + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -58,10 +58,10 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -111,10 +111,10 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt deleted file mode 100644 index 7a4afd3d2b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,343 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- ReusedSubquery - -Comet accelerated 298 out of 331 eligible operators (90%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt deleted file mode 100644 index ee55822981..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,59 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 54 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a.native_iceberg_compat/extended.txt deleted file mode 100644 index eaccfdfc70..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,2209 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 2010 out of 2169 eligible operators (92%). Final plan contains 38 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt deleted file mode 100644 index 2044df0e8a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,92 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 63 out of 86 eligible operators (73%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a.native_iceberg_compat/extended.txt deleted file mode 100644 index 2ea3aa5e71..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,323 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 296 out of 317 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt deleted file mode 100644 index bdbfb0d6dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt deleted file mode 100644 index 922864baa1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt deleted file mode 100644 index eb85a80eb1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt deleted file mode 100644 index 78557e7ffb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,339 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometBroadcastHashJoin - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- Subquery - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 298 out of 327 eligible operators (91%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt deleted file mode 100644 index a8adb0443d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,2350 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometUnion - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : :- CometFilter - : : : +- ReusedSubquery - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : +- ReusedSubquery - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometFilter - : : +- Subquery - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometFilter - : : +- ReusedSubquery - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometFilter - : +- ReusedSubquery - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 2127 out of 2302 eligible operators (92%). Final plan contains 46 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt deleted file mode 100644 index 94f7834f85..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,214 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 204 out of 210 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt deleted file mode 100644 index 643e47b0c6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt deleted file mode 100644 index 136315f3db..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 18 out of 28 eligible operators (64%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt similarity index 93% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt index d669223566..7d564dd7a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Expand diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt deleted file mode 100644 index d83496133a..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,159 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - -Comet accelerated 141 out of 151 eligible operators (93%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt deleted file mode 100644 index 4488b9b0ff..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,95 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Filter - : +- Subquery - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin [COMET: Comet is not compatible with Spark for case conversion in locale-specific cases. Set spark.comet.caseConversion.enabled=true to enable it anyway.] - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - -Comet accelerated 68 out of 88 eligible operators (77%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt similarity index 92% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt index 8e67cf3e18..a94d9e453a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/extended.txt @@ -3,10 +3,10 @@ CometNativeColumnarToRow +- CometColumnarExchange +- Filter : +- Subquery - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -47,10 +47,10 @@ CometNativeColumnarToRow : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.customer_address - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt deleted file mode 100644 index e7288e7e47..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,99 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 91 out of 95 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt deleted file mode 100644 index 18ed3e6335..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,41 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - -Comet accelerated 35 out of 37 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt deleted file mode 100644 index 72960fc267..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,62 +0,0 @@ -TakeOrderedAndProject -+- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- BroadcastHashJoin - : : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)] - : : : :- CometNativeColumnarToRow - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- CometNativeColumnarToRow - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 34 out of 54 eligible operators (62%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt similarity index 96% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt index 5e8a5364b6..07af300183 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject -+- HashAggregate ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt deleted file mode 100644 index bdbfb0d6dc..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,56 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometUnion - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt deleted file mode 100644 index e9571a1f12..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt deleted file mode 100644 index 9691120d2f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt deleted file mode 100644 index 0b4672eabb..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,94 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt deleted file mode 100644 index 190830204f..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,212 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Exchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : : +- CometNativeColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometSortMergeJoin - :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 134 out of 196 eligible operators (68%). Final plan contains 14 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt similarity index 97% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt index 36a797bad9..6977b1a25a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt @@ -13,7 +13,7 @@ TakeOrderedAndProject : +- CometSortMergeJoin : :- CometSort : : +- CometColumnarExchange - : : +- HashAggregate + : : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : : +- Exchange : : +- HashAggregate : : +- Project @@ -63,7 +63,7 @@ TakeOrderedAndProject : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometSort : +- CometColumnarExchange - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -114,7 +114,7 @@ TakeOrderedAndProject +- CometSortMergeJoin :- CometSort : +- CometColumnarExchange - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -164,7 +164,7 @@ TakeOrderedAndProject : +- CometNativeScan parquet spark_catalog.default.date_dim +- CometSort +- CometColumnarExchange - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt deleted file mode 100644 index 5e8acff889..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,105 +0,0 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : : +- CometNativeColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt deleted file mode 100644 index e2832782b8..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,269 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometUnion - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- ReusedSubquery - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometBroadcastExchange - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - -Comet accelerated 242 out of 263 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt deleted file mode 100644 index e3297b4d66..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,57 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : : +- Subquery - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 48 out of 52 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt deleted file mode 100644 index a868670a16..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,247 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometExchange - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : :- CometProject - : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : :- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : : : : : : : : :- CometBroadcastExchange - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometHashAggregate - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometSortMergeJoin - : : : : : : : : : : : : : : : :- CometSort - : : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : : : : : : +- CometSort - : : : : : : : : : : : : : : : +- CometExchange - : : : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : : +- CometProject - : : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : : +- CometFilter - : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt deleted file mode 100644 index b610537f40..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,294 +0,0 @@ -TakeOrderedAndProject -+- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt deleted file mode 100644 index a9c5d2e774..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,168 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometNativeColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - : +- CometNativeColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometNativeColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 97 out of 153 eligible operators (63%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt similarity index 95% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt index dab9824951..4f9deee67e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt @@ -8,7 +8,7 @@ TakeOrderedAndProject +- CometColumnarExchange +- HashAggregate +- Union - :- HashAggregate + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -57,10 +57,10 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - :- HashAggregate + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Project @@ -109,10 +109,10 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.date_dim - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Project diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt deleted file mode 100644 index 22fde3b02c..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,72 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : : : : +- SubqueryBroadcast - : : : : : : : : : : +- BroadcastExchange - : : : : : : : : : : +- CometNativeColumnarToRow - : : : : : : : : : : +- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.inventory - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometFilter - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.warehouse - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - -Comet accelerated 66 out of 68 eligible operators (97%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt deleted file mode 100644 index 922864baa1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,90 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 79 out of 85 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt deleted file mode 100644 index ec53af4809..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,172 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - +- CometSort - +- CometExchange - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - -Comet accelerated 159 out of 167 eligible operators (95%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt deleted file mode 100644 index 82e3d521e5..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,350 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - :- HashAggregate - : +- Exchange - : +- HashAggregate - : +- HashAggregate - : +- Exchange - : +- HashAggregate - : +- Union - : :- CometNativeColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- CometNativeColumnarToRow - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : :- Project - : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : : :- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- HashAggregate - +- Exchange - +- HashAggregate - +- HashAggregate - +- Exchange - +- HashAggregate - +- Union - :- CometNativeColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - :- Project - : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] - : :- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometNativeColumnarToRow - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page - -Comet accelerated 282 out of 332 eligible operators (84%). Final plan contains 16 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt similarity index 97% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt index 45a4b3bb5a..7831e37634 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_datafusion/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt @@ -4,7 +4,7 @@ CometNativeColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- HashAggregate + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Union @@ -115,10 +115,10 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page - :- HashAggregate + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate - : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] : +- Exchange : +- HashAggregate : +- Union @@ -229,10 +229,10 @@ CometNativeColumnarToRow : +- CometBroadcastExchange : +- CometFilter : +- CometNativeScan parquet spark_catalog.default.web_page - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate - +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] +- Exchange +- HashAggregate +- Union diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt deleted file mode 100644 index 1b1e6d0cde..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,80 +0,0 @@ -TakeOrderedAndProject -+- Project [COMET: Comet does not support Spark's BigDecimal rounding] - +- CometNativeColumnarToRow - +- CometSortMergeJoin - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometSortMergeJoin - : : : :- CometSort - : : : : +- CometExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometNativeColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometFilter - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometFilter - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- ReusedSubquery - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 70 out of 76 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt deleted file mode 100644 index fbd7cba46d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,392 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : : +- SubqueryBroadcast - : : : : : : : +- BroadcastExchange - : : : : : : : +- CometNativeColumnarToRow - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometSortMergeJoin - : : : : : : :- CometSort - : : : : : : : +- CometExchange - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : : +- SubqueryBroadcast - : : : : : : +- BroadcastExchange - : : : : : : +- CometNativeColumnarToRow - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometSortMergeJoin - : : : : : :- CometSort - : : : : : : +- CometExchange - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : : +- ReusedSubquery - : : : : : +- CometSort - : : : : : +- CometExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometSortMergeJoin - : : : : :- CometSort - : : : : : +- CometExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : +- ReusedSubquery - : : : : +- CometSort - : : : : +- CometExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion - -Comet accelerated 374 out of 386 eligible operators (96%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt deleted file mode 100644 index 58d00a427d..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,87 +0,0 @@ -TakeOrderedAndProject -+- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometNativeColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt deleted file mode 100644 index 1a985ead1b..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ /dev/null @@ -1,33 +0,0 @@ -CometNativeColumnarToRow -+- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] - +- CometNativeColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometNativeColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt similarity index 100% rename from spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_datafusion/extended.txt rename to spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt diff --git a/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala index 63936a94b7..7591279786 100644 --- a/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala @@ -247,7 +247,7 @@ class CometArrayExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelp .withColumn("arrUnsupportedArgs", expr("array_insert(arr, idx, 1)")) checkSparkAnswerAndFallbackReasons( df.select("arrUnsupportedArgs"), - Set("scalaudf is not supported", "unsupported arguments for ArrayInsert")) + Set("ScalaUDF has no native path", "unsupported arguments for ArrayInsert")) } } } @@ -942,61 +942,64 @@ class CometArrayExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelp } test("size with array input") { - withTempDir { dir => - withTempView("t1") { - val path = new Path(dir.toURI.toString, "test.parquet") - makeParquetFileAllPrimitiveTypes(path, dictionaryEnabled = true, 100) - spark.read.parquet(path.toString).createOrReplaceTempView("t1") + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withTempDir { dir => + withTempView("t1") { + val path = new Path(dir.toURI.toString, "test.parquet") + makeParquetFileAllPrimitiveTypes(path, dictionaryEnabled = true, 100) + spark.read.parquet(path.toString).createOrReplaceTempView("t1") - // Test size function with arrays built from columns (ensures native execution) - checkSparkAnswerAndOperator( - sql("SELECT size(array(_2, _3, _4)) from t1 where _2 is not null order by _2, _3, _4")) - checkSparkAnswerAndOperator( - sql("SELECT size(array(_1)) from t1 where _1 is not null order by _1")) - checkSparkAnswerAndOperator( - sql("SELECT size(array(_2, _3)) from t1 where _2 is null order by _2, _3")) + // Test size function with arrays built from columns (ensures native execution) + checkSparkAnswerAndOperator( + sql( + "SELECT size(array(_2, _3, _4)) from t1 where _2 is not null order by _2, _3, _4")) + checkSparkAnswerAndOperator( + sql("SELECT size(array(_1)) from t1 where _1 is not null order by _1")) + checkSparkAnswerAndOperator( + sql("SELECT size(array(_2, _3)) from t1 where _2 is null order by _2, _3")) - // Test with conditional arrays (forces runtime evaluation) - checkSparkAnswerAndOperator(sql( - "SELECT size(case when _2 > 0 then array(_2, _3, _4) else array(_2) end) from t1 order by _2, _3, _4")) - checkSparkAnswerAndOperator(sql( - "SELECT size(case when _1 then array(_8, _9) else array(_8, _9, _10) end) from t1 order by _1, _8, _9, _10")) + // Test with conditional arrays (forces runtime evaluation) + checkSparkAnswerAndOperator(sql( + "SELECT size(case when _2 > 0 then array(_2, _3, _4) else array(_2) end) from t1 order by _2, _3, _4")) + checkSparkAnswerAndOperator(sql( + "SELECT size(case when _1 then array(_8, _9) else array(_8, _9, _10) end) from t1 order by _1, _8, _9, _10")) - // Test empty arrays using conditional logic to avoid constant folding - checkSparkAnswerAndOperator(sql( - "SELECT size(case when _2 < 0 then array(_2, _3) else array() end) from t1 order by _2, _3")) + // Test empty arrays using conditional logic to avoid constant folding + checkSparkAnswerAndOperator(sql( + "SELECT size(case when _2 < 0 then array(_2, _3) else array() end) from t1 order by _2, _3")) - // Test null arrays using conditional logic - checkSparkAnswerAndOperator(sql( - "SELECT size(case when _2 is null then cast(null as array) else array(_2) end) from t1 order by _2")) + // Test null arrays using conditional logic + checkSparkAnswerAndOperator(sql( + "SELECT size(case when _2 is null then cast(null as array) else array(_2) end) from t1 order by _2")) - // Test with different data types using column references - checkSparkAnswerAndOperator( - sql("SELECT size(array(_8, _9, _10)) from t1 where _8 is not null order by _8, _9, _10") - ) // string arrays - checkSparkAnswerAndOperator( - sql( - "SELECT size(array(_2, _3, _4, _5, _6)) from t1 where _2 is not null order by _2, _3, _4, _5, _6" - ) - ) // int arrays + // Test with different data types using column references + checkSparkAnswerAndOperator( + sql( + "SELECT size(array(_8, _9, _10)) from t1 where _8 is not null order by _8, _9, _10" + ) + ) // string arrays + checkSparkAnswerAndOperator( + sql( + "SELECT size(array(_2, _3, _4, _5, _6)) from t1 where _2 is not null order by _2, _3, _4, _5, _6" + ) + ) // int arrays + } } } } test("size - respect to legacySizeOfNull") { val table = "t1" - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT) { - withTable(table) { - sql(s"create table $table(col array) using parquet") - sql(s"insert into $table values(null)") - withSQLConf(SQLConf.LEGACY_SIZE_OF_NULL.key -> "false") { - checkSparkAnswerAndOperator(sql(s"select size(col) from $table")) - } - withSQLConf( - SQLConf.LEGACY_SIZE_OF_NULL.key -> "true", - SQLConf.ANSI_ENABLED.key -> "false") { - checkSparkAnswerAndOperator(sql(s"select size(col) from $table")) - } + withTable(table) { + sql(s"create table $table(col array) using parquet") + sql(s"insert into $table values(null)") + withSQLConf(SQLConf.LEGACY_SIZE_OF_NULL.key -> "false") { + checkSparkAnswerAndOperator(sql(s"select size(col) from $table")) + } + withSQLConf( + SQLConf.LEGACY_SIZE_OF_NULL.key -> "true", + SQLConf.ANSI_ENABLED.key -> "false") { + checkSparkAnswerAndOperator(sql(s"select size(col) from $table")) } } } diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 1e4ea0990a..4968809c29 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ListBuffer import scala.util.Random import org.apache.hadoop.fs.Path +import org.apache.spark.SparkConf import org.apache.spark.sql.{CometTestBase, DataFrame, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.parser.ParseException @@ -41,6 +42,11 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ + // Casts in this suite predominantly test non-ANSI semantics (silent overflow/null on + // invalid input); tests that target ANSI behavior opt in explicitly via withSQLConf. + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.ANSI_ENABLED.key, "false") + /** Create a data generator using a fixed seed so that tests are reproducible */ private val gen = DataGenerator.DEFAULT @@ -1540,13 +1546,8 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("cast ArrayType to StringType") { - val hasIncompatibleType = (dt: DataType) => - if (CometConf.COMET_NATIVE_SCAN_IMPL.get() == "auto") { - true - } else { - !CometScanTypeChecker(CometConf.COMET_NATIVE_SCAN_IMPL.get()) - .isTypeSupported(dt, "a", ListBuffer.empty) - } + val hasIncompatibleType = + (dt: DataType) => !CometScanTypeChecker().isTypeSupported(dt, "a", ListBuffer.empty) Seq( BooleanType, StringType, @@ -1589,7 +1590,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } // https://github.com/apache/datafusion-comet/issues/3906 - ignore("cast nested ArrayType to nested ArrayType") { + test("cast nested ArrayType to nested ArrayType") { val types = Seq( BooleanType, StringType, @@ -1600,14 +1601,12 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { FloatType, DoubleType, DecimalType(10, 2), - DecimalType(38, 18), + // DecimalType(38, 18) is excluded for the same reason as the one-dimensional array + // matrix: decimal-to-float/double casts can differ by ~1 ULP from Spark. DateType, TimestampType, BinaryType) - testArrayCastMatrix( - types, - dt => ArrayType(ArrayType(dt)), - dt => generateArrays(100, ArrayType(dt))) + testArrayCastMatrix(types, dt => ArrayType(ArrayType(dt)), dt => generateNestedArrays(20, dt)) } // CAST from TimestampNTZType @@ -1715,7 +1714,7 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } private def generateArrays(rowNum: Int, elementType: DataType): DataFrame = { - import scala.collection.JavaConverters._ + import scala.jdk.CollectionConverters._ val schema = StructType(Seq(StructField("a", ArrayType(elementType), true))) def buildRows(values: Seq[Any]): Seq[Row] = { Range(0, rowNum).map { i => @@ -1769,6 +1768,37 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + private def generateNestedArrays(rowNum: Int, elementType: DataType): DataFrame = { + import scala.jdk.CollectionConverters._ + val schema = StructType(Seq(StructField("a", ArrayType(ArrayType(elementType)), true))) + val innerArrays = generateArrays(rowNum, elementType) + .collect() + .map { row => + if (row.isNullAt(0)) { + null + } else { + row.getSeq[Any](0) + } + } + .toSeq + + def buildRows(values: Seq[Seq[Any]]): Seq[Row] = { + Range(0, rowNum).map { i => + Row( + Seq[Any]( + values(i % values.length), + // Keep every third row's middle nested-array element null. + if (i % 3 == 0) null else values((i + 1) % values.length), + values((i + 2) % values.length))) + } + } + + val sampleValue = innerArrays.find(_ != null).orNull + val rows = Seq(Row(Seq(sampleValue, null, sampleValue)), Row(Seq.empty[Any]), Row(null)) ++ + buildRows(innerArrays) + spark.createDataFrame(rows.asJava, schema) + } + // https://github.com/apache/datafusion-comet/issues/2038 test("test implicit cast to dictionary with case when and dictionary type") { withSQLConf("parquet.enable.dictionary" -> "true") { diff --git a/spark/src/test/scala/org/apache/comet/CometCodegenAssertions.scala b/spark/src/test/scala/org/apache/comet/CometCodegenAssertions.scala new file mode 100644 index 0000000000..13334a5134 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometCodegenAssertions.scala @@ -0,0 +1,82 @@ +/* + * 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 + +import org.apache.arrow.vector.ValueVector +import org.apache.spark.sql.types.DataType + +import org.apache.comet.udf.codegen.CometScalaUDFCodegen + +/** + * Shared assertions for the codegen-dispatcher test suites. Mix in alongside `CometTestBase`. + */ +trait CometCodegenAssertions { + + /** Asserts the dispatcher actually ran during `f`, guarding against silent serde fallback. */ + protected def assertCodegenRan(f: => Unit): Unit = { + CometScalaUDFCodegen.resetStats() + f + val after = CometScalaUDFCodegen.stats() + assert( + after.compileCount + after.cacheHitCount >= 1, + s"expected codegen dispatcher activity, got $after") + } + + /** + * Asserts the composed subtree fused into one kernel signature, not N (one per sub-expression). + * Uses the JVM-wide signature set rather than `compileCount` because per-task `boundExpr` + * isolation makes multi-partition queries trip `compileCount > 1` even when the bytecode is + * shared. + */ + protected def assertOneKernelForSubtree(f: => Unit): Unit = { + CometScalaUDFCodegen.resetStats() + val sigsBefore = CometScalaUDFCodegen.snapshotCompiledSignatures() + f + val sigsAfter = CometScalaUDFCodegen.snapshotCompiledSignatures() + val grew = sigsAfter.size - sigsBefore.size + assert( + grew <= 1, + s"expected <= 1 new compiled-kernel signature for the composed subtree, grew by $grew; " + + s"new=${sigsAfter -- sigsBefore}") + val after = CometScalaUDFCodegen.stats() + assert( + after.compileCount + after.cacheHitCount >= 1, + s"expected codegen dispatcher activity, got $after") + } + + /** + * Asserts a kernel matching the given input Arrow vector classes and output type sits in the + * JVM-wide signature set. Pair with `assertCodegenRan` since the set is append-only. Compares + * by simple name to be robust to Arrow shading. + */ + protected def assertKernelSignaturePresent( + inputs: Seq[Class[_ <: ValueVector]], + output: DataType): Unit = { + val sigs = CometScalaUDFCodegen.snapshotCompiledSignatures() + val expectedNames = inputs.map(_.getSimpleName).toIndexedSeq + val present = sigs.exists { case (cached, dt) => + dt == output && cached.map(_.getSimpleName) == expectedNames + } + assert( + present, + s"expected kernel signature $expectedNames -> $output; " + + s"cache had ${sigs.map { case (c, d) => (c.map(_.getSimpleName), d) }}") + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometCodegenFuzzSuite.scala b/spark/src/test/scala/org/apache/comet/CometCodegenFuzzSuite.scala new file mode 100644 index 0000000000..9167713cad --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometCodegenFuzzSuite.scala @@ -0,0 +1,409 @@ +/* + * 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 + +import java.io.File +import java.text.SimpleDateFormat + +import scala.util.Random + +import org.apache.commons.io.FileUtils +import org.apache.spark.SparkConf +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import org.apache.comet.DataTypeSupport.isComplexType +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, ParquetGenerator, SchemaGenOptions} + +/** + * Randomized end-to-end tests for the Arrow-direct codegen dispatcher: schema-driven coverage of + * every input vector class against random parquet, plus a decimal precision-scale sweep across + * the `Decimal.MAX_LONG_DIGITS=18` boundary at varying null densities. Extends [[CometTestBase]] + * (not [[CometFuzzTestBase]]) because the base's `shuffle` x `nativeC2R` cross-product is + * irrelevant for projection-only queries. + */ +class CometCodegenFuzzSuite + extends CometTestBase + with AdaptiveSparkPlanHelper + with CometCodegenAssertions { + + /** Random schema with primitives plus shallow arrays and structs. No maps, no deep nesting. */ + private var mixedTypesFilename: String = _ + + /** Random schema with deeply nested arrays / structs / maps. */ + private var nestedTypesFilename: String = _ + + /** Asia/Kathmandu has a non-zero minute offset (UTC+5:45); good for timezone edge cases. */ + private val defaultTimezone = "Asia/Kathmandu" + + override def beforeAll(): Unit = { + super.beforeAll() + val tempDir = System.getProperty("java.io.tmpdir") + val random = new Random(42) + val dataGenOptions = DataGenOptions( + generateNegativeZero = false, + baseDate = new SimpleDateFormat("YYYY-MM-DD hh:mm:ss") + .parse("2024-05-25 12:34:56") + .getTime) + + mixedTypesFilename = s"$tempDir/CometCodegenFuzzSuite_${System.currentTimeMillis()}.parquet" + withSQLConf( + CometConf.COMET_ENABLED.key -> "false", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> defaultTimezone) { + val schemaGenOptions = + SchemaGenOptions(generateArray = true, generateStruct = true) + ParquetGenerator.makeParquetFile( + random, + spark, + mixedTypesFilename, + 1000, + schemaGenOptions, + dataGenOptions) + } + + nestedTypesFilename = + s"$tempDir/CometCodegenFuzzSuite_nested_${System.currentTimeMillis()}.parquet" + withSQLConf( + CometConf.COMET_ENABLED.key -> "false", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> defaultTimezone) { + val schemaGenOptions = + SchemaGenOptions(generateArray = true, generateStruct = true, generateMap = true) + val schema = FuzzDataGenerator.generateNestedSchema( + random, + numCols = 10, + minDepth = 2, + maxDepth = 4, + options = schemaGenOptions) + ParquetGenerator.makeParquetFile( + random, + spark, + nestedTypesFilename, + schema, + 1000, + dataGenOptions) + } + + spark.read.parquet(mixedTypesFilename).createOrReplaceTempView("t1") + spark.read.parquet(nestedTypesFilename).createOrReplaceTempView("t2") + } + + protected override def afterAll(): Unit = { + super.afterAll() + FileUtils.deleteDirectory(new File(mixedTypesFilename)) + FileUtils.deleteDirectory(new File(nestedTypesFilename)) + } + + private val RowCount: Int = 512 + private val nullDensities: Seq[Double] = Seq(0.0, 0.1, 0.5, 1.0) + // (precision, scale) shapes spanning both sides of `Decimal.MAX_LONG_DIGITS=18`: small short, + // boundary short with varying scale, just-past-boundary long, and max decimal128. + private val decimalShapes: Seq[(Int, Int)] = Seq((9, 2), (18, 0), (18, 9), (19, 0), (38, 10)) + + override protected def sparkConf: SparkConf = + super.sparkConf + .set(CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.key, "true") + + /** + * Identity ScalaUDF for one of the 14 primitive types in + * [[org.apache.comet.testing.SchemaGenOptions.defaultPrimitiveTypes]]. Returns the registered + * name when the type maps to a known Scala arg, or `None` for shapes we choose not to probe. + * `BigDecimal` UDF args are encoded as `DecimalType(38, 18)`; Spark inserts an implicit cast + * around the call but the underlying column read still hits our kernel's `getDecimal` at the + * column's native precision. + */ + private def registerIdentityUdfFor(dt: DataType, name: String): Option[String] = dt match { + case _: BooleanType => spark.udf.register(name, (x: Boolean) => x); Some(name) + case _: ByteType => spark.udf.register(name, (x: Byte) => x); Some(name) + case _: ShortType => spark.udf.register(name, (x: Short) => x); Some(name) + case _: IntegerType => spark.udf.register(name, (x: Int) => x); Some(name) + case _: LongType => spark.udf.register(name, (x: Long) => x); Some(name) + case _: FloatType => spark.udf.register(name, (x: Float) => x); Some(name) + case _: DoubleType => spark.udf.register(name, (x: Double) => x); Some(name) + case _: DecimalType => + spark.udf.register(name, (x: java.math.BigDecimal) => x); Some(name) + case _: DateType => spark.udf.register(name, (x: java.sql.Date) => x); Some(name) + case _: TimestampType => + spark.udf.register(name, (x: java.sql.Timestamp) => x); Some(name) + case _: TimestampNTZType => + spark.udf.register(name, (x: java.time.LocalDateTime) => x); Some(name) + case _: StringType => spark.udf.register(name, (x: String) => x); Some(name) + case _: BinaryType => spark.udf.register(name, (x: Array[Byte]) => x); Some(name) + case _ => None + } + + /** + * Identity-Int UDF for the cardinality-based complex probe. One UDF covers every Array and Map + * column, regardless of element type. + * + * Avoids `Seq[T]` / `Map[K, V]` UDF arg materialization: Spark's `MapObjects.doGenCode` reads + * each element unconditionally and null-checks afterward, so on null positions of a + * dictionary-encoded primitive Arrow vector the garbage ID buffer feeds + * `dictionary.decodeToLong/decodeToFloat` and throws `ArrayIndexOutOfBoundsException`. Bug + * reproduces in pure Spark; `cardinality(col)` exercises `getArray`/`getMap` without entering + * the element deserializer. + */ + private lazy val cardinalityProbeUdf: String = { + val name = "sz_complex" + spark.udf.register(name, (i: Int) => i) + name + } + + test("identity ScalaUDF over every primitive column") { + val primitiveFields = + spark.table("t1").schema.fields.filterNot(f => isComplexType(f.dataType)) + assert(primitiveFields.nonEmpty, "expected at least one primitive column in random schema") + for (field <- primitiveFields) { + val udfName = s"id_${field.name}" + registerIdentityUdfFor(field.dataType, udfName) match { + case Some(_) => + assertCodegenRan { + checkSparkAnswerAndOperator(s"SELECT $udfName(${field.name}) FROM t1") + } + case None => + fail( + s"primitive column ${field.name}: ${field.dataType} not in identity UDF catalog; " + + "extend registerIdentityUdfFor") + } + } + } + + test("complex-probe ScalaUDF on every complex column") { + val complexFields = spark.table("t1").schema.fields.filter(f => isComplexType(f.dataType)) + assert(complexFields.nonEmpty, "expected at least one complex column in random schema") + for (field <- complexFields) { + probeComplexColumn(field, viewName = "t1") + } + } + + test("complex-probe ScalaUDF on top-level columns of deeply nested schema") { + for (field <- spark.table("t2").schema.fields) { + probeComplexColumn(field, viewName = "t2") + } + } + + /** + * Element-level fuzz for nested array reads: `ArrayMax.doGenCode` walks every element of every + * row, calling the kernel's nested element getter, the path the unsafe-getter optimization + * touches and which the cardinality probe deliberately skips. + */ + test("array_max element fuzz: every Array column") { + val arrayPrimitiveFields = spark.table("t1").schema.fields.filter { + case StructField(_, ArrayType(elemDt, _), _, _) if !isComplexType(elemDt) => true + case _ => false + } + assert( + arrayPrimitiveFields.nonEmpty, + "expected at least one Array column in random schema") + for (field <- arrayPrimitiveFields) { + val ArrayType(elemDt, _) = field.dataType: @unchecked + val udfName = s"id_arrmax_${field.name}" + registerIdentityUdfFor(elemDt, udfName) match { + case Some(_) => + assertCodegenRan { + checkSparkAnswerAndOperator(s"SELECT $udfName(array_max(${field.name})) FROM t1") + } + case None => + fail( + s"array column ${field.name} elem ${elemDt} not in identity UDF catalog; " + + "extend registerIdentityUdfFor") + } + } + } + + /** + * Map variant of the array element fuzz: `map_keys` / `map_values` produce arrays the kernel + * walks via `ArrayMax`, exercising the map's per-row offset chain (MapVector -> entries + * StructVector -> child) that the array test alone wouldn't catch. + */ + test("array_max element fuzz: map_keys / map_values on Map columns") { + val mapPrimitiveFields = spark.table("t2").schema.fields.filter { + case StructField(_, MapType(kDt, vDt, _), _, _) + if !isComplexType(kDt) && !isComplexType(vDt) => + true + case _ => false + } + for (field <- mapPrimitiveFields) { + val MapType(kDt, vDt, _) = field.dataType: @unchecked + registerIdentityUdfFor(kDt, s"id_mapk_${field.name}").foreach { udf => + assertCodegenRan { + checkSparkAnswerAndOperator(s"SELECT $udf(array_max(map_keys(${field.name}))) FROM t2") + } + } + registerIdentityUdfFor(vDt, s"id_mapv_${field.name}").foreach { udf => + assertCodegenRan { + checkSparkAnswerAndOperator( + s"SELECT $udf(array_max(map_values(${field.name}))) FROM t2") + } + } + } + } + + /** + * Doubly-nested array element fuzz: `flatten(arr)` collapses `Array>` into `Array` + * (exercising the outer-array element getter that returns each inner ArrayData), then + * `array_max` walks the leaf X primitives. Closes the gap that the singly-nested + * `array_max(arr)` test alone leaves on doubly-nested primitive arrays. + */ + test("array_max element fuzz: flatten on Array> columns") { + val nestedArrayPrimitiveFields = spark.table("t2").schema.fields.filter { + case StructField(_, ArrayType(ArrayType(elemDt, _), _), _, _) if !isComplexType(elemDt) => + true + case _ => false + } + for (field <- nestedArrayPrimitiveFields) { + val ArrayType(ArrayType(elemDt, _), _) = field.dataType: @unchecked + val udfName = s"id_arrflat_${field.name}" + registerIdentityUdfFor(elemDt, udfName).foreach { _ => + assertCodegenRan { + checkSparkAnswerAndOperator( + s"SELECT $udfName(array_max(flatten(${field.name}))) FROM t2") + } + } + } + } + + /** + * Element-level fuzz for `Array>`. `array_distinct` is a non-HOF unary expression + * that hashes each element to dedupe. Struct hashing is field-wise, so the kernel emits element + * reads on each struct's fields. `cardinality` consumes the result without materialization. + * Asserts the optimizer keeps `ArrayDistinct` so the coverage isn't vacuously folded. + */ + test("array_distinct element fuzz: Array> columns") { + val arrayStructFields = spark.table("t1").schema.fields.filter { + case StructField(_, ArrayType(_: StructType, _), _, _) => true + case _ => false + } + spark.udf.register("id_int_arrdistinct", (i: Int) => i) + for (field <- arrayStructFields) { + val q = s"SELECT id_int_arrdistinct(cardinality(array_distinct(${field.name}))) FROM t1" + val df = sql(q) + val plan = df.queryExecution.optimizedPlan.toString + val planLower = plan.toLowerCase + assert( + planLower.contains("array_distinct") || planLower.contains("arraydistinct"), + s"optimizer eliminated array_distinct on column ${field.name}; coverage would be " + + s"vacuous. plan=\n$plan") + assertCodegenRan { + checkSparkAnswerAndOperator(df) + } + } + } + + private def probeCardinality(accessor: String, dt: DataType, viewName: String): Unit = { + // `Size` only supports `ArrayType` in Comet, so for `MapType` we route through `map_keys` to + // reach a `Size(ArrayType)`. Spark still calls `getMap` on the column vector to extract the + // keys, which is the accessor path this probe is intended to exercise. + val sizeExpr = dt match { + case _: MapType => s"size(map_keys($accessor))" + case _ => s"cardinality($accessor)" + } + assertCodegenRan { + checkSparkAnswerAndOperator(s"SELECT $cardinalityProbeUdf($sizeExpr) FROM $viewName") + } + } + + /** + * Top-level Array / Map produces a cardinality probe. Struct drills into each scalar child via + * `GetStructField`. Nested Array / Map sub-fields also get the cardinality probe (depth bound: + * deeper struct-of-struct nesting is skipped to keep the sweep finite). + */ + private def probeComplexColumn(field: StructField, viewName: String): Unit = { + field.dataType match { + case _: ArrayType | _: MapType => + probeCardinality(field.name, field.dataType, viewName) + + case st: StructType => + for (subField <- st.fields) { + val accessor = s"${field.name}.${subField.name}" + subField.dataType match { + case _: ArrayType | _: MapType => + probeCardinality(accessor, subField.dataType, viewName) + case dt if !isComplexType(dt) => + val udfName = s"id_${field.name}_${subField.name}" + registerIdentityUdfFor(dt, udfName).foreach { _ => + assertCodegenRan { + checkSparkAnswerAndOperator(s"SELECT $udfName($accessor) FROM $viewName") + } + } + case _ => // deeper struct nesting skipped + } + } + + case _ => + } + } + + /** Random `BigDecimal` values fitting `(precision, scale)`, with `nullDensity` of them null. */ + private def generateDecimals( + seed: Long, + precision: Int, + scale: Int, + nullDensity: Double): Seq[java.math.BigDecimal] = { + val rng = new Random(seed) + val intDigits = precision - scale + // `BigInt.apply(bits, rng)` samples uniformly on `[0, 2^bits - 1]`; bound to the decimal's + // integer-part range (10^intDigits - 1) so the result fits the schema. `BigInteger.bitLength` + // would overshoot slightly. Min with the exact max is cheap insurance. + val intMax = BigInt(10).pow(intDigits) - 1 + val bits = math.max(intMax.bitLength, 1) + (0 until RowCount).map { _ => + if (rng.nextDouble() < nullDensity) null + else { + val mag = BigInt(bits, rng).min(intMax) + val signed = if (rng.nextBoolean()) -mag else mag + new java.math.BigDecimal(signed.bigInteger, scale) + } + } + } + + private def withDecimalTable(decimalType: String, values: Seq[java.math.BigDecimal])( + f: => Unit): Unit = { + withTable("t") { + sql(s"CREATE TABLE t (d $decimalType) USING parquet") + if (values.nonEmpty) { + val rows = values.map { v => + if (v == null) "(NULL)" else s"(${v.toPlainString})" + } + rows.grouped(64).foreach { batch => + sql(s"INSERT INTO t VALUES ${batch.mkString(", ")}") + } + } + f + } + } + + for { + density <- nullDensities + (precision, scale) <- decimalShapes + } { + test(s"decimal identity precision=$precision scale=$scale nullDensity=$density") { + spark.udf.register("dec_id_fuzz", (d: java.math.BigDecimal) => d) + val seed = ((precision * 31L) + scale) * 31L + density.hashCode + val values = generateDecimals(seed, precision, scale, density) + withDecimalTable(s"DECIMAL($precision, $scale)", values) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT dec_id_fuzz(d) FROM t")) + } + } + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometCodegenHOFSuite.scala b/spark/src/test/scala/org/apache/comet/CometCodegenHOFSuite.scala new file mode 100644 index 0000000000..9b2511ce0d --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometCodegenHOFSuite.scala @@ -0,0 +1,112 @@ +/* + * 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 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +/** + * Higher-order function regression coverage for the codegen dispatcher. + * + * Spark's HOFs (`ArrayTransform`, `ArrayFilter`, `ArrayAggregate`, `ArrayExists`, `ZipWith`, + * `MapFilter`, etc.) all extend `CodegenFallback`. The dispatcher's `canHandle` admits them. + * `CodegenFallback.doGenCode` emits a single `((Expression) references[N]).eval(row)` call site + * per HOF. The kernel dispatches to `Expression.eval(InternalRow)`, which iterates the array, + * mutates `NamedLambdaVariable.value`'s `AtomicReference` per element, and recursively evaluates + * the lambda body. Lambda-body leaf reads resolve through the kernel's typed Arrow getters since + * the kernel is an `InternalRow`. + * + * Cost model: per-row interpreted-eval inside the HOF subtree. Surrounding native operators stay + * native. Surrounding non-HOF expressions stay codegen. + * + * Each Spark task gets its own `boundExpr` Java object. The dispatcher's compile cache lives on + * the per-task instance, not the companion, so concurrent partitions cannot race on a shared + * `NamedLambdaVariable.value`. The two-collects test below regresses this. + */ +class CometCodegenHOFSuite + extends CometTestBase + with AdaptiveSparkPlanHelper + with CometCodegenAssertions { + + override protected def sparkConf: SparkConf = + super.sparkConf + .set(CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.key, "true") + + private def withArrayIntTable(rows: String)(f: => Unit): Unit = { + withTable("t") { + sql("CREATE TABLE t (a ARRAY) USING parquet") + sql(s"INSERT INTO t VALUES $rows") + f + } + } + + test("ArrayTransform inside identity ScalaUDF over Array") { + // Regresses the simplest HOF shape: `idArr(transform(a, x -> x + 1))`. Tree contains one + // CodegenFallback HOF. The kernel splices its interpreted-eval call site into the per-row + // body and the result ArrayData feeds the ListVector output writer. Null and empty rows + // exercise the HOF's null-on-null-arg path and the empty-iteration path. + spark.udf.register("idArr", (arr: Seq[Int]) => arr) + withArrayIntTable("(array(1, 2, 3)), (array(-5, 5)), (array()), (null)") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT idArr(transform(a, x -> x + 1)) FROM t")) + } + } + } + + test("array_max over ArrayTransform inside identity ScalaUDF") { + // Regresses composed CodegenFallback subtrees: array_max consumes the ArrayData transform + // produces. Both run interpreted. The kernel splices both eval call sites into the same + // per-row body. Empty/null rows exercise array_max's null-on-empty path. + spark.udf.register("idIntBoxed", (i: java.lang.Integer) => i) + withArrayIntTable("(array(1, 2, 3)), (array(-5, 5)), (null), (array(0))") { + assertCodegenRan { + checkSparkAnswerAndOperator( + sql("SELECT idIntBoxed(array_max(transform(a, x -> x * 2))) FROM t")) + } + } + } + + test("array_max over ArrayFilter inside identity ScalaUDF") { + // Regresses ArrayFilter (distinct HOF class from ArrayTransform). Filter producing an + // empty array from non-empty input exercises array_max(emptyArray) downstream. + spark.udf.register("idIntBoxed", (i: java.lang.Integer) => i) + withArrayIntTable("(array(1, -1, 2)), (array(-5, -2)), (array()), (null)") { + assertCodegenRan { + checkSparkAnswerAndOperator( + sql("SELECT idIntBoxed(array_max(filter(a, x -> x > 0))) FROM t")) + } + } + } + + test("HOF query produces correct results across two collects (per-task isolation regression)") { + // Regresses the per-task `boundExpr` isolation. When the dispatcher's compile cache lived on + // the companion object, multiple tasks shared one `boundExpr` and concurrent partitions + // raced on `NamedLambdaVariable.value`'s `AtomicReference`, producing off-by-one element + // values. The fix moved the cache to the per-task instance so each task deserializes its own + // boundExpr. Two collects of the same query must each match Spark's interpreter. + spark.udf.register("idArr", (arr: Seq[Int]) => arr) + withArrayIntTable("(array(1, 2)), (array(3, 4)), (array(5))") { + val q = "SELECT idArr(transform(a, x -> x + 1)) FROM t" + checkSparkAnswerAndOperator(sql(q)) + checkSparkAnswerAndOperator(sql(q)) + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometCodegenSourceSuite.scala b/spark/src/test/scala/org/apache/comet/CometCodegenSourceSuite.scala new file mode 100644 index 0000000000..27a5830c6d --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometCodegenSourceSuite.scala @@ -0,0 +1,1098 @@ +/* + * 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 + +import org.scalatest.funsuite.AnyFunSuite + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Add, BoundReference, Coalesce, Concat, CreateArray, CreateMap, ElementAt, Expression, GetStructField, LeafExpression, Length, Literal, Nondeterministic, Rand, Size, Unevaluable, Upper} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, CodegenFallback, ExprCode} +import org.apache.spark.sql.types._ + +import org.apache.comet.codegen.CometBatchKernelCodegen +import org.apache.comet.codegen.CometBatchKernelCodegen.{ArrayColumnSpec, ArrowColumnSpec, MapColumnSpec, ScalarColumnSpec, StructColumnSpec, StructFieldSpec} +import org.apache.comet.udf.codegen.CometScalaUDFCodegen + +// Resolve Arrow vector classes through the codegen object so tests see the same `Class` objects +// the codegen pattern-matches against, regardless of any future shading rearrangement. + +/** + * Generated-source inspection tests. These exercise `CometBatchKernelCodegen.generateSource` and + * assert on the emitted Java directly, without invoking Janino. The goal is to catch regressions + * in the optimizations we claim the dispatcher applies: + * + * - `NullIntolerant` short-circuit wraps `ev.code` in `if (any-input-null) { setNull } else { + * ev.code; write }`. + * - Non-nullable column declaration emits `return false;` from `isNullAt(ord)`, and a + * `BoundReference.nullable=false` (Catalyst sets this from schema-declared nullability) makes + * Spark's `doGenCode` skip emitting its own `row.isNullAt(ord)` probe entirely. + * - Zero-copy string reads route through `UTF8String.fromAddress`. + * + * These are the smallest durable tests that the claimed optimizations actually reach the + * generated Java, and they document the shapes future contributors should preserve. + */ +class CometCodegenSourceSuite extends AnyFunSuite { + + private val varCharVectorClass = + CometBatchKernelCodegen.vectorClassBySimpleName("VarCharVector") + + private val nullableString = ArrowColumnSpec(varCharVectorClass, nullable = true) + private val nonNullableString = ArrowColumnSpec(varCharVectorClass, nullable = false) + + private def gen( + expr: org.apache.spark.sql.catalyst.expressions.Expression, + specs: ArrowColumnSpec*): String = + CometBatchKernelCodegen.generateSource(expr, specs.toIndexedSeq).body + + test("non-nullable column emits literal-false isNullAt case") { + val expr = Length(BoundReference(0, StringType, nullable = false)) + val src = gen(expr, nonNullableString) + assert( + src.contains("case 0: return false;"), + s"expected non-nullable isNullAt to return literal false; got:\n$src") + } + + test("non-nullable BoundReference elides Spark's own isNullAt probe in the expression body") { + // When the BoundReference carries `nullable=false` (Catalyst sets this from schema-declared + // nullability), Spark's `doGenCode` skips the `row.isNullAt(ord)` branch at source level. + // The dispatcher does not derive runtime nullability anymore. The BoundReference's source + // flag is the sole signal, and schema-non-null columns get full elision for free. + val expr = Length(BoundReference(0, StringType, nullable = false)) + val src = gen(expr, nonNullableString) + assert( + !src.contains("row.isNullAt(0)"), + s"expected Spark's BoundReference null probe to be elided; got:\n$src") + } + + test("nullable column emits delegated isNullAt case") { + val expr = Length(BoundReference(0, StringType, nullable = true)) + val src = gen(expr, nullableString) + assert( + src.contains("case 0: return this.col0.isNull(this.rowIdx);"), + s"expected nullable isNullAt to delegate to the Arrow vector; got:\n$src") + } + + test("VarCharVector getUTF8String uses zero-copy fromAddress") { + val expr = Length(BoundReference(0, StringType, nullable = true)) + val src = gen(expr, nullableString) + assert( + src.contains("org.apache.spark.unsafe.types.UTF8String"), + s"expected UTF8String reference; got:\n$src") + assert(src.contains(".fromAddress("), s"expected zero-copy fromAddress read; got:\n$src") + } + + test("NullIntolerant expression emits input-null short-circuit before ev.code") { + // Upper is NullIntolerant (null in -> null out). Expect the default body to prepend + // `if (this.col0.isNull(i)) { setNull; } else { ... }` so null rows skip the whole + // expression eval, not just the setNull write. + val expr = Upper(BoundReference(0, StringType, nullable = true)) + val src = gen(expr, nullableString) + assert( + src.contains("this.col0.isNull(i)"), + s"expected NullIntolerant short-circuit on input ordinal 0; got:\n$src") + assert( + src.contains("output.setNull(i);"), + s"expected setNull emission for short-circuited null rows; got:\n$src") + } + + test("NullIntolerant short-circuit emitted when every node is NullIntolerant") { + // Length(Upper(BoundReference)): Length is NullIntolerant, Upper is NullIntolerant, + // BoundReference is a leaf. Every path from a leaf to the root propagates nulls, so the + // short-circuit heuristic ("any input null -> output null") holds. + val expr = Length(Upper(BoundReference(0, StringType, nullable = true))) + val src = gen(expr, nullableString) + assert( + src.contains("if (this.col0.isNull(i))"), + s"expected short-circuit on col0 when every node is NullIntolerant; got:\n$src") + } + + test("NullIntolerant short-circuit skipped when a non-NullIntolerant node breaks the chain") { + // Concat is not NullIntolerant. Null in some args doesn't necessarily produce a null + // result. The short-circuit heuristic would be incorrect here (short-circuiting on c0 or c1 + // being null would skip evaluation, but Concat's null handling differs). Expect the + // default path without the `if (colX.isNull(i) || colY.isNull(i))` wrapper, letting Spark's + // own `ev.code` handle nulls correctly. + val nullable1 = ArrowColumnSpec(varCharVectorClass, nullable = true) + val nullable2 = ArrowColumnSpec(varCharVectorClass, nullable = true) + val expr = Length( + Concat( + Seq( + BoundReference(0, StringType, nullable = true), + BoundReference(1, StringType, nullable = true)))) + val src = gen(expr, nullable1, nullable2) + assert( + !src.contains("this.col0.isNull(i) || this.col1.isNull(i)"), + "expected no pre-null short-circuit when Concat breaks the NullIntolerant chain; " + + s"got:\n$src") + } + + test("canHandle rejects CodegenFallback expressions") { + val expr = FakeCodegenFallback(BoundReference(0, StringType, nullable = true)) + val reason = CometBatchKernelCodegen.canHandle(expr) + assert(reason.isDefined, "expected canHandle to reject CodegenFallback") + assert( + reason.get.contains("FakeCodegenFallback"), + s"expected reason to name the rejected expression class; got: ${reason.get}") + } + + test("canHandle accepts Nondeterministic expressions (per-partition kernel handles state)") { + // Each cache entry holds one kernel instance with `init(partitionIndex)` called once, so + // Rand / Uuid / etc. produce the expected per-partition sequences across batches. The + // previous canHandle rejection was conservative. With that caching in place, accepting + // Nondeterministic is correct. + val expr = FakeNondeterministic() + val reason = CometBatchKernelCodegen.canHandle(expr) + assert(reason.isEmpty, s"expected canHandle to accept Nondeterministic; got $reason") + } + + test("canHandle rejects Unevaluable expressions") { + val expr = FakeUnevaluable() + val reason = CometBatchKernelCodegen.canHandle(expr) + assert(reason.isDefined, "expected canHandle to reject Unevaluable") + assert( + reason.get.contains("FakeUnevaluable"), + s"expected reason to name the rejected expression class; got: ${reason.get}") + } + + test("CSE collapses a repeated subtree to one evaluation in the generated body") { + // `Add(Length(Upper(c0)), Length(Upper(c0)))` has `Length(Upper(c0))` as a common subtree. + // Length.doGenCode emits `$value.numChars()` on every Spark version the project targets, + // which makes it a stable activation marker. Upper's own doGenCode text drifts across + // versions (Spark 3.5 emits `UTF8String.toUpperCase()`, Spark 4 emits + // `CollationSupport.Upper.exec*` via collation-aware codegen), so we avoid it as a marker. + // When CSE fires, `Length(Upper(c0))` compiles into one `subExpr_*` helper whose body calls + // `numChars()` once. Both uses in the `Add` read the cached result from mutable state. + // Without CSE, each Add child would emit its own `numChars()` call. + val upperOrd0 = Upper(BoundReference(0, StringType, nullable = true)) + val lenUpper = Length(upperOrd0) + val expr = Add(lenUpper, lenUpper) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(nullableString)) + val occurrences = "\\.numChars\\(\\)".r.findAllIn(result.body).size + assert( + occurrences == 1, + "expected CSE to collapse repeated Length evaluation to 1 numChars() call, " + + s"got $occurrences; src=\n${CodeFormatter.format(result.code)}") + // Additional proof: CSE emitted a `subExpr_` helper method. Without CSE the generator would + // have inlined the repeated subtree into the main body with no helper at all. + assert( + result.body.contains("subExpr_0(row)"), + s"expected CSE helper invocation; got:\n${CodeFormatter.format(result.code)}") + } + + test("CSE does not fire on non-deterministic expressions (regression guard)") { + // `Add(Rand(0), Rand(0))` is two structurally identical non-deterministic subtrees. CSE must + // not collapse them: each Rand call must produce an independent draw. Spark's CSE + // (`EquivalentExpressions.updateExprInMap`) filters non-deterministic expressions via + // `expr.deterministic`, so the two Rands stay separate. This test is a regression guard + // against Spark ever relaxing that check and against us accidentally applying CSE outside + // the `generateExpressions` path (which respects the filter). `Rand.doGenCode` emits one + // `$rng.nextDouble()` call per evaluation, so two Rands produce two `.nextDouble()` calls + // in the body. One-call output would indicate incorrect CSE. + val expr = Add(Rand(Literal(0L, LongType)), Rand(Literal(0L, LongType))) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq.empty) + val occurrences = "\\.nextDouble\\(\\)".r.findAllIn(result.body).size + assert( + occurrences == 2, + "expected two independent Rand evaluations (no CSE on nondeterministic), " + + s"got $occurrences; src=\n${CodeFormatter.format(result.code)}") + } + + test("DecimalVector getDecimal specializes to unscaled-long fast path for short precision") { + // Mirrors Spark's `UnsafeRow.getDecimal` split at `Decimal.MAX_LONG_DIGITS` (18), done at + // codegen time rather than at runtime. The dispatcher reads the `BoundReference`'s + // `DecimalType` at source-generation time and emits only the fast-path branch when + // `precision <= 18`. The fast path reads the low 8 bytes of the 16-byte Arrow decimal128 + // slot directly as a signed long via `ArrowBuf.getLong` and wraps with + // `Decimal.createUnsafe`, avoiding the `BigDecimal` allocation `DecimalVector.getObject` + // would perform. For precision > 18 the generator emits only the slow-path branch + // (`getObject + Decimal.apply`); see the companion test below. + val decimalVectorClass = CometBatchKernelCodegen.vectorClassBySimpleName("DecimalVector") + val spec = ArrowColumnSpec(decimalVectorClass, nullable = true) + val expr = BoundReference(0, DecimalType(18, 2), nullable = true) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(spec)) + assert( + result.body.contains(".createUnsafe("), + "expected Decimal.createUnsafe call on fast path; got:\n" + + CodeFormatter.format(result.code)) + assert( + result.body.contains("Platform.getLong(") && + result.body.contains("this.col0_valueAddr"), + "expected unsafe Platform.getLong against cached valueAddr; got:\n" + + CodeFormatter.format(result.code)) + assert( + !result.body.contains(".getObject("), + "expected specialized fast path (no BigDecimal fallback branch in source); got:\n" + + CodeFormatter.format(result.code)) + assert( + !result.body.contains("if (precision <= 18)"), + "expected no runtime precision branch for known short-precision column; got:\n" + + CodeFormatter.format(result.code)) + } + + test("DecimalVector getDecimal specializes to BigDecimal slow path for long precision") { + // Companion to the fast-path test. For `DecimalType(p, s)` with `p > 18`, the unscaled value + // can exceed 64 bits, so the generator emits only the `getObject + Decimal.apply` branch. + // The fast path markers must be absent so the generated source is minimal for this column. + val decimalVectorClass = CometBatchKernelCodegen.vectorClassBySimpleName("DecimalVector") + val spec = ArrowColumnSpec(decimalVectorClass, nullable = true) + val expr = BoundReference(0, DecimalType(38, 10), nullable = true) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(spec)) + assert( + result.body.contains(".getObject(") && result.body.contains(".apply("), + s"expected BigDecimal slow path; got:\n${CodeFormatter.format(result.code)}") + assert( + !result.body.contains(".createUnsafe("), + "expected no fast-path emission for long-precision column; got:\n" + + CodeFormatter.format(result.code)) + assert( + !result.body.contains("if (precision <= 18)"), + "expected no runtime precision branch for known long-precision column; got:\n" + + CodeFormatter.format(result.code)) + } + + test("DecimalVector setSafe uses unscaled-long fast path for short-precision output") { + // The output writer specializes on the root expression's DecimalType precision. For + // precision <= 18 the Decimal's unscaled long is passed directly to + // `DecimalVector.setSafe(int, long)`, avoiding the BigDecimal allocation that + // `toJavaBigDecimal()` performs. Use a simple expression that produces a DecimalType output: + // `BoundReference(0, DecimalType(18, 2))` has output type DecimalType(18, 2), which is what + // the generator specializes on. + val decimalVectorClass = CometBatchKernelCodegen.vectorClassBySimpleName("DecimalVector") + val spec = ArrowColumnSpec(decimalVectorClass, nullable = true) + val expr = BoundReference(0, DecimalType(18, 2), nullable = true) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(spec)) + assert( + result.body.contains(".toUnscaledLong()"), + s"expected toUnscaledLong call on fast path; got:\n${CodeFormatter.format(result.code)}") + assert( + !result.body.contains(".toJavaBigDecimal("), + "expected no BigDecimal allocation for short-precision output; got:\n" + + CodeFormatter.format(result.code)) + } + + test("DecimalVector setSafe uses BigDecimal slow path for long-precision output") { + // Companion to the fast-path output test. Precision > 18 can have unscaled values exceeding + // 64 bits, so the writer must fall back to the BigDecimal path. + val decimalVectorClass = CometBatchKernelCodegen.vectorClassBySimpleName("DecimalVector") + val spec = ArrowColumnSpec(decimalVectorClass, nullable = true) + val expr = BoundReference(0, DecimalType(38, 10), nullable = true) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(spec)) + assert( + result.body.contains(".toJavaBigDecimal("), + s"expected BigDecimal slow path; got:\n${CodeFormatter.format(result.code)}") + assert( + !result.body.contains(".toUnscaledLong()"), + "expected no unscaled-long write for long-precision output; got:\n" + + CodeFormatter.format(result.code)) + } + + test("VarCharVector setSafe uses on-heap UTF8String shortcut") { + // The UTF8String output writer avoids the `byte[] b = $value.getBytes()` allocation when + // the UTF8String is on-heap by passing its backing byte[] directly to + // `VarCharVector.setSafe(int, byte[], int, int)`. Spark's string functions allocate their + // result on-heap, so this path hits for typical string expressions. Off-heap fallback + // (for passthrough of zero-copy input reads) stays as the else branch. + // + // Markers: `getBaseObject()` (inspecting the backing), `instanceof byte[]` (the branch), + // and `Platform.BYTE_ARRAY_OFFSET` (the on-heap offset math). + val expr = Upper(BoundReference(0, StringType, nullable = true)) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(nullableString)) + assert( + result.body.contains(".getBaseObject()"), + s"expected UTF8String.getBaseObject call; got:\n${CodeFormatter.format(result.code)}") + assert( + result.body.contains("instanceof byte[]"), + s"expected on-heap instanceof branch; got:\n${CodeFormatter.format(result.code)}") + assert( + result.body.contains("Platform.BYTE_ARRAY_OFFSET"), + "expected on-heap offset math via Platform.BYTE_ARRAY_OFFSET; got:\n" + + CodeFormatter.format(result.code)) + assert( + result.body.contains(".getBytes()"), + s"expected off-heap getBytes fallback; got:\n${CodeFormatter.format(result.code)}") + } + + test("non-nullable root expression omits the `if (isNull)` branch in default body") { + // When the bound expression claims `nullable = false`, the default body drops the + // `if (ev.isNull) output.setNull(i);` guard entirely. `Length` on a non-nullable column is + // itself non-nullable (Length.nullable = child.nullable = false), so the writer goes + // straight to the setSafe/set call. This test uses a non-NullIntolerant-short-circuit + // shape by wrapping Length in Coalesce, so we exercise the default branch of defaultBody + // rather than the NullIntolerant one. Actually, Length is NullIntolerant, so the NI branch + // fires. Use an expression that's non-nullable but whose tree is not fully NullIntolerant + // to hit the default branch. `Coalesce(Seq(Length(col_non_null), Literal(0)))` has + // nullable=false (Coalesce is non-null when any child is) and Coalesce itself is not + // NullIntolerant, so the default branch runs. Assert `setNull` is absent. + val expr = Coalesce( + Seq(Length(BoundReference(0, StringType, nullable = false)), Literal(0, IntegerType))) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(nonNullableString)) + assert( + !result.body.contains("output.setNull(i);"), + "expected no setNull for a non-nullable root expression; got:\n" + + CodeFormatter.format(result.code)) + } + + test("nullable root expression keeps the `if (isNull)` branch in default body") { + // Baseline: when the root expression is nullable, the setNull branch must still be emitted. + // Uses Coalesce with a nullable child so the Coalesce itself remains nullable. Guards the + // NonNullableOutputShortCircuit optimization against over-firing. + val expr = Coalesce( + Seq( + Length(BoundReference(0, StringType, nullable = true)), + BoundReference(1, IntegerType, nullable = true))) + val result = CometBatchKernelCodegen.generateSource( + expr, + IndexedSeq( + nullableString, + ArrowColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true))) + assert( + result.body.contains("output.setNull(i);"), + "expected setNull branch for a nullable root expression; got:\n" + + CodeFormatter.format(result.code)) + } + + test("ArrayType(StringType) output emits ListVector startNewValue/endValue recursion") { + // CreateArray over a BoundReference(StringType) produces ArrayType(StringType). emitWrite's + // ArrayType case should emit: + // - ListVector cast of output + // - child VarCharVector extraction via getDataVector + // - startNewValue + per-element loop + endValue + // - the per-element write recursing into the StringType case (which uses the UTF8 on-heap + // shortcut marker `instanceof byte[]`) + // Focus markers: ListVector cast, VarCharVector child cast, startNewValue, endValue, and + // the inner UTF8 shortcut branch. + val expr = + CreateArray( + Seq(BoundReference(0, StringType, nullable = true), Literal.create("x", StringType))) + val result = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(nullableString)) + val src = result.body + val formatted = CodeFormatter.format(result.code) + assert(src.contains("ListVector"), s"expected ListVector in emitted body; got:\n$formatted") + assert(src.contains(".startNewValue("), s"expected startNewValue call; got:\n$formatted") + assert(src.contains(".endValue("), s"expected endValue call; got:\n$formatted") + assert( + src.contains(".getDataVector()"), + s"expected child vector extraction; got:\n$formatted") + assert( + src.contains("instanceof byte[]"), + s"expected inner UTF8 on-heap shortcut for string elements; got:\n$formatted") + } + + test("MapType output emits MapVector startNewValue/endValue + per-pair writes") { + // CreateMap produces MapType(k, v). emitWrite's MapType case should emit: + // - MapVector cast of output + // - entries StructVector extraction + // - typed key / value child casts via getChildByOrdinal(0) / (1) + // - startNewValue / endValue bracketing + // - setIndexDefined on each struct entry + // - keyArray() / valueArray() retrieval from the MapData source + // Non-null literals here mean `valueContainsNull == false`, so the value-side null guard is + // elided. The existence and elision of the `isNullAt` guard are exercised by the dedicated + // [[NullableElementElision]] tests below. + val expr = CreateMap( + Seq( + Literal.create("a", StringType), + Literal(1, IntegerType), + Literal.create("b", StringType), + Literal(2, IntegerType))) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq.empty).body + Seq( + "MapVector", + "StructVector", + ".startNewValue(", + ".endValue(", + ".setIndexDefined(", + ".keyArray()", + ".valueArray()").foreach { marker => + assert(src.contains(marker), s"expected $marker in MapType output emission; got:\n$src") + } + } + + test("ArrayType output elides isNullAt on the element loop when containsNull is false") { + // CreateArray over only-non-null Literals produces ArrayType(elementType, containsNull=false). + // The element write should drop the `arr.isNullAt(j)` guard at source level rather than + // relying on JIT folding. + val expr = CreateArray(Seq(Literal(1, IntegerType), Literal(2, IntegerType))) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq.empty).body + assert( + !src.contains(".isNullAt("), + s"expected no isNullAt in element loop when containsNull=false; got:\n$src") + assert(src.contains(".startNewValue("), s"expected startNewValue still emitted; got:\n$src") + } + + test("ArrayType output keeps isNullAt on the element loop when containsNull is true") { + // CreateArray with at least one nullable child produces containsNull=true. The element + // null-guard must survive. + val expr = + CreateArray(Seq(BoundReference(0, IntegerType, nullable = true), Literal(2, IntegerType))) + val intSpec = ArrowColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(intSpec)).body + assert( + src.contains(".isNullAt("), + s"expected isNullAt in element loop when containsNull=true; got:\n$src") + } + + test("MapType output keeps value isNullAt when valueContainsNull is true") { + // ElementAt with safe-index selection produces a nullable Int. Wrapping the value column in + // a CreateMap with that nullable Int makes valueContainsNull=true. The value-side null-guard + // must survive. + val expr = + CreateMap( + Seq(Literal.create("a", StringType), BoundReference(0, IntegerType, nullable = true))) + val intSpec = ArrowColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(intSpec)).body + assert( + src.contains(".isNullAt("), + s"expected isNullAt on the value-write branch when valueContainsNull=true; got:\n$src") + } + + test("ArrayType(StringType) input emits InputArray_col0 nested class with UTF8 child getter") { + // Array input with string elements: the kernel must expose a `getArray(0)` that hands Spark's + // `doGenCode` an `ArrayData` view onto the Arrow `ListVector`'s child `VarCharVector`. + // Markers: the nested class declaration with a slice constructor, the typed child getter + // using `fromAddress`, and a `getArray` switch on the ordinal that allocates a fresh view. + val varCharChildSpec = ScalarColumnSpec(varCharVectorClass, nullable = true) + val arraySpec = + ArrayColumnSpec(nullable = true, elementSparkType = StringType, element = varCharChildSpec) + val expr = Size(BoundReference(0, ArrayType(StringType), nullable = true)) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(arraySpec)).body + + assert( + src.contains("class InputArray_col0"), + s"expected nested ArrayData class for array col0; got:\n$src") + assert( + src.contains("InputArray_col0(int startIdx, int len)"), + s"expected InputArray_col0 to take a slice via constructor; got:\n$src") + assert( + src.contains("getElementStartIndex(") && src.contains("getElementEndIndex("), + s"expected list-offset reads at the call site; got:\n$src") + assert( + src.contains("public org.apache.spark.unsafe.types.UTF8String getUTF8String(int i)"), + s"expected element-type-specific UTF8String getter; got:\n$src") + assert( + src.contains(".fromAddress("), + s"expected zero-copy UTF8 read inside the nested ArrayData; got:\n$src") + assert( + src.contains("public org.apache.spark.sql.catalyst.util.ArrayData getArray(int ordinal)"), + s"expected kernel-level getArray switch; got:\n$src") + assert( + src.contains("return new InputArray_col0("), + s"expected getArray to allocate a fresh InputArray_col0 view; got:\n$src") + } + + test("ArrayType(IntegerType) input emits primitive int getter in nested class") { + val intChildSpec = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true) + val arraySpec = + ArrayColumnSpec(nullable = true, elementSparkType = IntegerType, element = intChildSpec) + val expr = Size(BoundReference(0, ArrayType(IntegerType), nullable = true)) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(arraySpec)).body + + assert( + src.contains("public int getInt(int i)"), + s"expected primitive int getter on nested array class; got:\n$src") + // Scalar-element fast path reads directly off the typed child vector. No BigDecimal / + // fromAddress scaffolding should leak in. + assert( + !src.contains(".fromAddress("), + s"int element getter should not wrap with UTF8 fromAddress; got:\n$src") + } + + test( + "ArrayType(DecimalType) short-precision input emits decimal128 fast-path via getLong in " + + "nested class") { + val decimalChildSpec = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("DecimalVector"), + nullable = true) + val arraySpec = ArrayColumnSpec( + nullable = true, + elementSparkType = DecimalType(10, 2), + element = decimalChildSpec) + val expr = + ElementAt( + BoundReference(0, ArrayType(DecimalType(10, 2)), nullable = true), + Literal(1, IntegerType)) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(arraySpec)).body + + // Fast path markers: reads the low 8 bytes of the decimal128 slot via getLong + createUnsafe. + // The slow path would go through getObject + Decimal.apply. + assert( + src.contains(".getLong(") && src.contains(".createUnsafe("), + s"expected decimal-input short-precision fast path in nested class; got:\n$src") + assert( + !src.contains(".getObject("), + s"short-precision decimal element should not use BigDecimal slow path; got:\n$src") + } + + test("ArrayType(DecimalType) long-precision input emits BigDecimal slow path in nested class") { + val decimalChildSpec = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("DecimalVector"), + nullable = true) + val arraySpec = ArrayColumnSpec( + nullable = true, + elementSparkType = DecimalType(30, 2), + element = decimalChildSpec) + val expr = + ElementAt( + BoundReference(0, ArrayType(DecimalType(30, 2)), nullable = true), + Literal(1, IntegerType)) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(arraySpec)).body + + assert( + src.contains(".getObject(") && src.contains("Decimal$.MODULE$"), + s"expected BigDecimal slow path for p>18 element; got:\n$src") + } + + private def generate(expr: Expression, specs: IndexedSeq[ArrowColumnSpec]): String = + CometBatchKernelCodegen.generateSource(expr, specs).body + + test("Array> emits outer + inner array classes with fresh inner allocation") { + val innerArray = ArrayColumnSpec( + nullable = true, + elementSparkType = IntegerType, + element = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true)) + val outerArray = ArrayColumnSpec( + nullable = true, + elementSparkType = ArrayType(IntegerType), + element = innerArray) + val expr = Size(BoundReference(0, ArrayType(ArrayType(IntegerType)), nullable = true)) + val src = generate(expr, IndexedSeq(outerArray)) + assert( + src.contains("class InputArray_col0 ") && src.contains("class InputArray_col0_e "), + s"expected both outer and inner array classes; got:\n$src") + assert( + src.contains("return new InputArray_col0_e("), + s"expected outer class to allocate a fresh inner array view per call; got:\n$src") + assert( + src.contains("public int getInt(int i)"), + s"expected innermost scalar getter for IntegerType element; got:\n$src") + } + + test("Array> emits array class allocating fresh InputStruct_col0_e") { + val innerStruct = StructColumnSpec( + nullable = true, + fields = Seq( + StructFieldSpec( + "a", + IntegerType, + nullable = true, + ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true)))) + val outerArray = ArrayColumnSpec( + nullable = true, + elementSparkType = StructType(Seq(StructField("a", IntegerType, nullable = true)).toArray), + element = innerStruct) + val elemType = StructType(Seq(StructField("a", IntegerType, nullable = true)).toArray) + val expr = Size(BoundReference(0, ArrayType(elemType), nullable = true)) + val src = generate(expr, IndexedSeq(outerArray)) + assert( + src.contains("class InputArray_col0 ") && src.contains("class InputStruct_col0_e "), + s"expected array-of-struct nested classes; got:\n$src") + assert( + src.contains("return new InputStruct_col0_e(startIndex + i)"), + s"expected array getStruct to allocate a fresh inner struct view; got:\n$src") + } + + test("Struct> emits outer + inner struct classes") { + val innerStruct = StructColumnSpec( + nullable = true, + fields = Seq( + StructFieldSpec( + "a", + IntegerType, + nullable = true, + ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true)))) + val outerStruct = StructColumnSpec( + nullable = true, + fields = Seq( + StructFieldSpec( + "s", + StructType(Seq(StructField("a", IntegerType, nullable = true)).toArray), + nullable = true, + innerStruct))) + val innerType = StructType(Seq(StructField("a", IntegerType, nullable = true)).toArray) + val outerType = StructType(Seq(StructField("s", innerType, nullable = true)).toArray) + val expr = GetStructField( + GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("s")), + 0, + Some("a")) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("class InputStruct_col0 ") && src.contains("class InputStruct_col0_f0 "), + s"expected outer + inner struct classes; got:\n$src") + assert( + src.contains("return new InputStruct_col0_f0(this.rowIdx)"), + s"expected outer struct getStruct to allocate a fresh inner struct view; got:\n$src") + assert( + src.contains("public int getInt(int ordinal)"), + s"expected innermost getInt on InputStruct_col0_f0; got:\n$src") + } + + test("Struct> emits struct class allocating fresh InputArray_col0_f0") { + val innerArray = ArrayColumnSpec( + nullable = true, + elementSparkType = IntegerType, + element = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true)) + val outerStruct = StructColumnSpec( + nullable = true, + fields = Seq(StructFieldSpec("a", ArrayType(IntegerType), nullable = true, innerArray))) + val structType = + StructType(Seq(StructField("a", ArrayType(IntegerType), nullable = true)).toArray) + val expr = Size(GetStructField(BoundReference(0, structType, nullable = true), 0, Some("a"))) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("class InputStruct_col0 ") && src.contains("class InputArray_col0_f0 "), + s"expected struct-of-array nested classes; got:\n$src") + assert( + src.contains("return new InputArray_col0_f0("), + s"expected struct getArray to allocate a fresh inner array view; got:\n$src") + } + + test("Map emits InputMap_col0 + keyArray / valueArray views") { + val keySpec = ScalarColumnSpec(varCharVectorClass, nullable = true) + val valueSpec = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true) + val mapSpec = MapColumnSpec( + nullable = true, + keySparkType = StringType, + valueSparkType = IntegerType, + key = keySpec, + value = valueSpec) + val expr = Size(BoundReference(0, MapType(StringType, IntegerType), nullable = true)) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(mapSpec)).body + assert( + src.contains("class InputMap_col0 "), + s"expected InputMap_col0 nested class; got:\n$src") + assert( + src.contains("class InputArray_col0_k ") && src.contains("class InputArray_col0_v "), + s"expected key/value array view classes; got:\n$src") + assert( + src.contains("return new InputArray_col0_k(this.startIndex, this.length)"), + s"expected keyArray to allocate a fresh view over the map slice; got:\n$src") + assert( + src.contains("return new InputArray_col0_v(this.startIndex, this.length)"), + s"expected valueArray to allocate a fresh view over the map slice; got:\n$src") + assert( + src.contains("public org.apache.spark.sql.catalyst.util.MapData getMap(int ordinal)"), + s"expected kernel-level getMap switch; got:\n$src") + assert( + src.contains("return new InputMap_col0("), + s"expected getMap to allocate a fresh InputMap_col0 view; got:\n$src") + } + + test("Map, Array> emits complex key and complex value views") { + val keyElem = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true) + val keyArraySpec = + ArrayColumnSpec(nullable = true, elementSparkType = IntegerType, element = keyElem) + val valueElem = ScalarColumnSpec(varCharVectorClass, nullable = true) + val valueArraySpec = + ArrayColumnSpec(nullable = true, elementSparkType = StringType, element = valueElem) + val mapSpec = MapColumnSpec( + nullable = true, + keySparkType = ArrayType(IntegerType), + valueSparkType = ArrayType(StringType), + key = keyArraySpec, + value = valueArraySpec) + val expr = Size( + BoundReference(0, MapType(ArrayType(IntegerType), ArrayType(StringType)), nullable = true)) + val src = CometBatchKernelCodegen.generateSource(expr, IndexedSeq(mapSpec)).body + // Full chain of nested classes should appear: top-level map view, the key/value array + // views, and the inner array classes for each complex key/value element. + Seq( + "class InputMap_col0 ", + "class InputArray_col0_k ", + "class InputArray_col0_v ", + "class InputArray_col0_k_e ", + "class InputArray_col0_v_e ").foreach { marker => + assert(src.contains(marker), s"expected $marker in emission; got:\n$src") + } + } + + /** + * Null-guard emission for nested reference-typed getters. Spark's + * `CodeGenerator.setArrayElement` only emits an `isNullAt` check before `update(i, getX(j))` + * for primitive elements. For reference types it relies on the source's `getX` to return null + * on null positions itself, matching `ColumnarArray.getBinary`. The emitter prepends `if + * (isNullAt(...)) return null;` when the element / field is nullable. + * + * Runtime regressions for the leaf reference types live in `CometCodegenSuite`; complex-type + * (Struct/Array/Map) coverage runs through HOFs in `CometCodegenHOFSuite`. + */ + private val nullableIntStruct = StructColumnSpec( + nullable = true, + fields = Seq( + StructFieldSpec( + "a", + IntegerType, + nullable = true, + ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true)))) + private val nullableIntStructType = + StructType(Seq(StructField("a", IntegerType, nullable = true)).toArray) + + private val nullableIntArray = ArrayColumnSpec( + nullable = true, + elementSparkType = IntegerType, + element = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = true)) + + private val nullableIntStrMap = MapColumnSpec( + nullable = true, + keySparkType = IntegerType, + valueSparkType = StringType, + key = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false), + value = ScalarColumnSpec(varCharVectorClass, nullable = true)) + + test("nested array of nullable Struct emits null guard before allocating InputStruct view") { + val outer = ArrayColumnSpec( + nullable = true, + elementSparkType = nullableIntStructType, + element = nullableIntStruct) + val expr = Size(BoundReference(0, ArrayType(nullableIntStructType), nullable = true)) + val src = generate(expr, IndexedSeq(outer)) + assert( + src.contains("if (isNullAt(i)) return null;") && + src.contains("new InputStruct_col0_e(startIndex + i)"), + s"expected null guard and InputStruct alloc on nullable Struct element; got:\n$src") + } + + test("nested array of non-nullable Struct elides null guard") { + // Fully non-nullable inner spec: outer struct nullable=false AND inner Int field + // nullable=false. Without the inner field also being non-nullable the inner + // primitive-Int getter wouldn't emit a guard anyway (we only guard reference types), but + // making everything non-nullable means the broad `!src.contains("if (isNullAt(...))")` + // assertion verifies "no guards anywhere" rather than passing because the inner happens + // to be a primitive we don't guard. + val nonNullableInner = StructColumnSpec( + nullable = false, + fields = Seq( + StructFieldSpec( + "a", + IntegerType, + nullable = false, + ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false)))) + val outer = ArrayColumnSpec( + nullable = true, + elementSparkType = nullableIntStructType, + element = nonNullableInner) + val expr = Size(BoundReference(0, ArrayType(nullableIntStructType), nullable = true)) + val src = generate(expr, IndexedSeq(outer)) + assert( + src.contains("new InputStruct_col0_e(startIndex + i)"), + s"sanity: alloc still emitted; got:\n$src") + assert( + !src.contains("if (isNullAt(i)) return null;") && + !src.contains("if (isNullAt(0)) return null;"), + s"expected no null guard anywhere on fully non-nullable Struct element; got:\n$src") + } + + test( + "nested array of nullable inner Array emits null guard before allocating InputArray view") { + val outer = ArrayColumnSpec( + nullable = true, + elementSparkType = ArrayType(IntegerType), + element = nullableIntArray) + val expr = Size(BoundReference(0, ArrayType(ArrayType(IntegerType)), nullable = true)) + val src = generate(expr, IndexedSeq(outer)) + assert( + src.contains("if (isNullAt(i)) return null;") && + src.contains("new InputArray_col0_e(__s, __e - __s)"), + s"expected null guard and InputArray alloc on nullable Array element; got:\n$src") + } + + test("nested array of non-nullable inner Array elides null guard") { + val nonNullableInner = ArrayColumnSpec( + nullable = false, + elementSparkType = IntegerType, + element = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false)) + val outer = ArrayColumnSpec( + nullable = true, + elementSparkType = ArrayType(IntegerType), + element = nonNullableInner) + val expr = Size(BoundReference(0, ArrayType(ArrayType(IntegerType)), nullable = true)) + val src = generate(expr, IndexedSeq(outer)) + assert( + src.contains("new InputArray_col0_e(__s, __e - __s)"), + s"sanity: alloc still emitted; got:\n$src") + assert( + !src.contains("if (isNullAt(i)) return null;"), + s"expected no null guard on non-nullable inner Array element; got:\n$src") + } + + test("nested array of nullable Map emits null guard before allocating InputMap view") { + val outer = ArrayColumnSpec( + nullable = true, + elementSparkType = MapType(IntegerType, StringType), + element = nullableIntStrMap) + val expr = + Size(BoundReference(0, ArrayType(MapType(IntegerType, StringType)), nullable = true)) + val src = generate(expr, IndexedSeq(outer)) + assert( + src.contains("if (isNullAt(i)) return null;") && + src.contains("new InputMap_col0_e(__s, __e - __s)"), + s"expected null guard and InputMap alloc on nullable Map element; got:\n$src") + } + + test("nested array of non-nullable Map elides null guard") { + val nonNullableMap = MapColumnSpec( + nullable = false, + keySparkType = IntegerType, + valueSparkType = StringType, + key = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false), + value = ScalarColumnSpec(varCharVectorClass, nullable = false)) + val outer = ArrayColumnSpec( + nullable = true, + elementSparkType = MapType(IntegerType, StringType), + element = nonNullableMap) + val expr = + Size(BoundReference(0, ArrayType(MapType(IntegerType, StringType)), nullable = true)) + val src = generate(expr, IndexedSeq(outer)) + assert( + src.contains("new InputMap_col0_e(__s, __e - __s)"), + s"sanity: alloc still emitted; got:\n$src") + assert( + !src.contains("if (isNullAt(i)) return null;"), + s"expected no null guard on non-nullable Map element; got:\n$src") + } + + test("struct with nullable struct field emits null guard in getStruct(ordinal) switch") { + val outerStruct = StructColumnSpec( + nullable = true, + fields = + Seq(StructFieldSpec("s", nullableIntStructType, nullable = true, nullableIntStruct))) + val outerType = + StructType(Seq(StructField("s", nullableIntStructType, nullable = true)).toArray) + val expr = GetStructField( + GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("s")), + 0, + Some("a")) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("if (isNullAt(0)) return null;") && + src.contains("new InputStruct_col0_f0(this.rowIdx)"), + s"expected null guard and InputStruct alloc for nullable struct field; got:\n$src") + } + + test("struct with non-nullable struct field elides null guard") { + val nonNullableInner = StructColumnSpec( + nullable = false, + fields = Seq( + StructFieldSpec( + "a", + IntegerType, + nullable = false, + ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false)))) + val outerStruct = StructColumnSpec( + nullable = true, + fields = + Seq(StructFieldSpec("s", nullableIntStructType, nullable = false, nonNullableInner))) + val outerType = + StructType(Seq(StructField("s", nullableIntStructType, nullable = false)).toArray) + val expr = GetStructField( + GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("s")), + 0, + Some("a")) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("new InputStruct_col0_f0(this.rowIdx)"), + s"sanity: alloc still emitted; got:\n$src") + assert( + !src.contains("if (isNullAt(0)) return null;") && + !src.contains("if (isNullAt(i)) return null;"), + s"expected no null guard anywhere on fully non-nullable struct field; got:\n$src") + } + + test("struct with nullable array field emits null guard in getArray(ordinal) switch") { + val outerStruct = StructColumnSpec( + nullable = true, + fields = + Seq(StructFieldSpec("a", ArrayType(IntegerType), nullable = true, nullableIntArray))) + val outerType = + StructType(Seq(StructField("a", ArrayType(IntegerType), nullable = true)).toArray) + val expr = + Size(GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("a"))) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("if (isNullAt(0)) return null;") && + src.contains("new InputArray_col0_f0(__s, __e - __s)"), + s"expected null guard and InputArray alloc for nullable array field; got:\n$src") + } + + test("struct with non-nullable array field elides null guard") { + val nonNullableInner = ArrayColumnSpec( + nullable = false, + elementSparkType = IntegerType, + element = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false)) + val outerStruct = StructColumnSpec( + nullable = true, + fields = + Seq(StructFieldSpec("a", ArrayType(IntegerType), nullable = false, nonNullableInner))) + val outerType = + StructType(Seq(StructField("a", ArrayType(IntegerType), nullable = false)).toArray) + val expr = + Size(GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("a"))) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("new InputArray_col0_f0(__s, __e - __s)"), + s"sanity: alloc still emitted; got:\n$src") + assert( + !src.contains("if (isNullAt(0)) return null;") && + !src.contains("if (isNullAt(i)) return null;"), + s"expected no null guard anywhere on fully non-nullable array field; got:\n$src") + } + + test("struct with nullable map field emits null guard in getMap(ordinal) switch") { + val outerStruct = StructColumnSpec( + nullable = true, + fields = Seq( + StructFieldSpec( + "m", + MapType(IntegerType, StringType), + nullable = true, + nullableIntStrMap))) + val outerType = + StructType(Seq(StructField("m", MapType(IntegerType, StringType), nullable = true)).toArray) + val expr = Size(GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("m"))) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("if (isNullAt(0)) return null;") && + src.contains("new InputMap_col0_f0(__s, __e - __s)"), + s"expected null guard and InputMap alloc for nullable map field; got:\n$src") + } + + test("struct with non-nullable map field elides null guard") { + val nonNullableMap = MapColumnSpec( + nullable = false, + keySparkType = IntegerType, + valueSparkType = StringType, + key = ScalarColumnSpec( + CometBatchKernelCodegen.vectorClassBySimpleName("IntVector"), + nullable = false), + value = ScalarColumnSpec(varCharVectorClass, nullable = false)) + val outerStruct = StructColumnSpec( + nullable = true, + fields = Seq( + StructFieldSpec("m", MapType(IntegerType, StringType), nullable = false, nonNullableMap))) + val outerType = StructType( + Seq(StructField("m", MapType(IntegerType, StringType), nullable = false)).toArray) + val expr = Size(GetStructField(BoundReference(0, outerType, nullable = true), 0, Some("m"))) + val src = generate(expr, IndexedSeq(outerStruct)) + assert( + src.contains("new InputMap_col0_f0(__s, __e - __s)"), + s"sanity: alloc still emitted; got:\n$src") + assert( + !src.contains("if (isNullAt(0)) return null;"), + s"expected no null guard on non-nullable map field; got:\n$src") + } + + test("CacheKey discriminates on ArrowColumnSpec.nullable") { + // Structural regression: same expression bytes and same Arrow vector class with different + // `nullable` must produce non-equal cache keys. The dispatcher today hardcodes `nullable=true` + // for top-level specs, so the two variants don't both arise from runtime data, but the case + // class equality contract still has to discriminate so that any future tiered cache or test + // construction can rely on it. The non-nullable variant's generated source emits a literal + // `false` from `isNullAt`, distinct codegen output that we never want to silently share with + // the nullable variant. + val bytes = java.nio.ByteBuffer.wrap(Array[Byte](1, 2, 3)) + val nullable = + IndexedSeq[ArrowColumnSpec](ArrowColumnSpec(varCharVectorClass, nullable = true)) + val nonNullable = + IndexedSeq[ArrowColumnSpec](ArrowColumnSpec(varCharVectorClass, nullable = false)) + val k1 = CometScalaUDFCodegen.CacheKey(bytes, nullable) + val k2 = CometScalaUDFCodegen.CacheKey(bytes, nonNullable) + assert( + k1 != k2, + "expected nullable=true and nullable=false specs to produce distinct cache keys") + assert( + k1.hashCode != k2.hashCode, + "case-class hashCode should also differ; identical hashCodes would degrade lookup but not " + + "equality, so the assertion is mainly a sanity check on Spec.hashCode") + } +} + +/** + * Minimal fake expressions for the `canHandle` rejection tests. Each opts into one of the marker + * traits whose presence forces a serde-level fallback. Bodies are unreachable; `canHandle` walks + * the tree structurally. + */ +private case class FakeCodegenFallback(child: Expression) + extends Expression + with CodegenFallback { + override def children: Seq[Expression] = Seq(child) + + override def nullable: Boolean = true + + override def dataType: DataType = StringType + + override def eval(input: InternalRow): Any = null + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): Expression = copy(child = newChildren.head) +} + +private case class FakeNondeterministic() extends LeafExpression with Nondeterministic { + override def nullable: Boolean = true + + override def dataType: DataType = IntegerType + + override protected def initializeInternal(partitionIndex: Int): Unit = {} + + override protected def evalInternal(input: InternalRow): Any = 0 + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + throw new UnsupportedOperationException("test fake; never reaches codegen") +} + +private case class FakeUnevaluable() extends LeafExpression with Unevaluable { + override def nullable: Boolean = true + + override def dataType: DataType = IntegerType +} diff --git a/spark/src/test/scala/org/apache/comet/CometCodegenSuite.scala b/spark/src/test/scala/org/apache/comet/CometCodegenSuite.scala new file mode 100644 index 0000000000..cdae68c90a --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometCodegenSuite.scala @@ -0,0 +1,1167 @@ +/* + * 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 + +import org.apache.arrow.vector._ +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.api.java.UDF1 +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.types._ + +import org.apache.comet.udf.codegen.CometScalaUDFCodegen + +/** + * End-to-end correctness for the Arrow-direct codegen dispatcher. Covers the scalar and complex + * type surface, composed UDF trees, subquery reuse, `TaskContext` propagation, per-task cache + * isolation, the `maxFields` plan-time gate, and regressions pinned from fuzz. + * + * Tests exercising fallback paths (config disabled, `maxFields` exceeded) use `checkSparkAnswer` + * rather than `checkSparkAnswerAndOperator` because ScalaUDF has no Comet-native path. Under + * fallback the project runs on the JVM Spark path. + */ +class CometCodegenSuite + extends CometTestBase + with AdaptiveSparkPlanHelper + with CometCodegenAssertions { + + override protected def sparkConf: SparkConf = + super.sparkConf + .set(CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.key, "true") + + private def withSubjects(values: String*)(f: => Unit): Unit = { + withTable("t") { + sql("CREATE TABLE t (s STRING) USING parquet") + val rows = values + .map(v => if (v == null) "(NULL)" else s"('${v.replace("'", "''")}')") + .mkString(", ") + sql(s"INSERT INTO t VALUES $rows") + f + } + } + + private def withTwoStringCols(rows: (String, String)*)(f: => Unit): Unit = { + withTable("t") { + sql("CREATE TABLE t (c1 STRING, c2 STRING) USING parquet") + if (rows.nonEmpty) { + val tuples = rows.map { case (a, b) => + val av = if (a == null) "NULL" else s"'${a.replace("'", "''")}'" + val bv = if (b == null) "NULL" else s"'${b.replace("'", "''")}'" + s"($av, $bv)" + } + sql(s"INSERT INTO t VALUES ${tuples.mkString(", ")}") + } + f + } + } + + test("ScalaUDF over concat(c1, c2) suppresses the null short-circuit") { + // Concat is not NullIntolerant. The dispatcher's short-circuit guard inspects every node in + // the bound tree and must skip the whole-tree null short-circuit because one child is + // non-NullIntolerant. The kernel therefore delegates null handling to Spark's generated + // code (which handles Concat(null, x) = x correctly) rather than returning null for any + // null input. Without the guard, null inputs would produce null outputs even where Spark + // produces a non-null concatenation. + spark.udf.register("tag", (s: String) => if (s == null) "N" else s"[${s}]") + withTwoStringCols(("abc", "123"), ("abc", null), (null, "123"), (null, null), ("zz", "zz")) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT tag(concat(c1, c2)) FROM t")) + } + } + } + + test("disabled mode bypasses the dispatcher") { + // When the per-feature config is off, `CometScalaUDF.convert` returns None and the enclosing + // operator falls back to Spark. The dispatcher's counters must not move. + spark.udf.register("noopStr", (s: String) => s) + CometScalaUDFCodegen.resetStats() + withSQLConf(CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.key -> "false") { + withSubjects("disabled_1", null) { + checkSparkAnswer(sql("SELECT noopStr(s) FROM t")) + } + } + val after = CometScalaUDFCodegen.stats() + assert( + after.compileCount == 0 && after.cacheHitCount == 0, + s"expected no dispatcher activity under disabled config, got $after") + } + + test("schema exceeding spark.sql.codegen.maxFields falls back to Spark") { + // `CometBatchKernelCodegen.canHandle` mirrors WSCG's `spark.sql.codegen.maxFields` gate by + // counting nested input fields plus the output field and refusing once the total exceeds the + // configured cap. Comet has no mid-execution fallback, so the gate must fire at plan time + // (in the serde) rather than letting an oversized kernel reach Janino. With 5 input + // BoundReferences and a 1-field output we have 6 fields total. Setting `maxFields=3` ensures + // the gate fires here regardless of test ordering or future schema additions. + spark.udf.register( + "sumFiveInts", + (a: Int, b: Int, c: Int, d: Int, e: Int) => a + b + c + d + e) + withTable("t") { + sql("CREATE TABLE t (a INT, b INT, c INT, d INT, e INT) USING parquet") + sql("INSERT INTO t VALUES (1, 2, 3, 4, 5), (10, 20, 30, 40, 50)") + CometScalaUDFCodegen.resetStats() + withSQLConf("spark.sql.codegen.maxFields" -> "3") { + checkSparkAnswer(sql("SELECT sumFiveInts(a, b, c, d, e) FROM t")) + } + val after = CometScalaUDFCodegen.stats() + assert( + after.compileCount == 0 && after.cacheHitCount == 0, + s"expected dispatcher fallback under maxFields=3, got $after") + } + } + + test("dispatcher caches the compiled kernel across batches of one query") { + // Within a single query, the dispatcher compiles a kernel for the (expression, schema) pair + // once and reuses it across every subsequent batch of the same shape. Force multiple batches + // by lowering the Comet batch size with a row count well above it, then assert at least one + // cache hit happened during the query. + // + // We deliberately do not assert cross-query cache reuse: Spark's analyzer produces a fresh + // `ScalaUDF` instance per query resolution, and the encoders embedded in that instance + // contain `AttributeReference`s with fresh `ExprId`s that our `BindReferences.bindReference` + // does not recurse into. The closure-serialized cache key bytes therefore drift across + // queries even when the registered function and schema are identical, so each new query of a + // ScalaUDF pays one compile up front and amortizes within itself. This is an acceptable + // amortization story (a few tens of milliseconds per query), not a behavior we can or do + // promise across queries. + spark.udf.register("kernelCacheMarker", (s: String) => if (s == null) null else s + "_kc") + val rows = (0 until 256).map(i => s"row_$i") + CometScalaUDFCodegen.resetStats() + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "32") { + withSubjects(rows: _*) { + checkSparkAnswerAndOperator(sql("SELECT kernelCacheMarker(s) FROM t")) + } + } + val stats = CometScalaUDFCodegen.stats() + assert(stats.compileCount >= 1, s"expected at least one compile during the query, got $stats") + assert( + stats.cacheHitCount >= 1, + s"expected at least one cache hit across batches of the same query, got $stats") + } + + test("per-partition kernel preserves Nondeterministic state across batches") { + // Wrap `monotonically_increasing_id()` as the argument of a ScalaUDF so the whole tree + // (including the stateful MonotonicallyIncreasingID child) routes through the dispatcher. + // Per-partition kernel caching means the id counter advances across batches within a + // partition. Without it, every batch would restart at 0 and the UDF output would disagree + // with Spark's. The UDF body is a trivial identity. We're testing state correctness of the + // Nondeterministic child across batches, not the UDF logic. + spark.udf.register("idPassthrough", (id: Long) => id) + val rows = (0 until 4096).map(i => s"row_$i") + withSubjects(rows: _*) { + assertCodegenRan { + checkSparkAnswerAndOperator( + sql("SELECT s, idPassthrough(monotonically_increasing_id()) FROM t")) + } + } + } + + test( + "same UDF over nullable and non-nullable columns gets distinct kernels with independent state") { + // Two columns, same type, different schema-declared nullability. Same UDF applied to each + // alongside a per-projection MonotonicallyIncreasingID. Each projection has its own MII + // child (different bytesKey), so each kernel must have its own counter advancing 0..N-1. + // If the dispatcher collapses them onto one kernel or shares state somehow, the counters + // would interleave and the output would diverge from Spark. + spark.udf.register("withId", (s: String, id: Long) => s"${s}_${id}") + withTempPath { dir => + import org.apache.spark.sql.Row + import org.apache.spark.sql.types.{StringType, StructField, StructType} + val schema = StructType( + Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = false))) + val rows = (0 until 64).map(i => Row(s"a_$i", s"b_$i")) + val rdd = spark.sparkContext.parallelize(rows, numSlices = 1) + spark.createDataFrame(rdd, schema).write.parquet(dir.getCanonicalPath) + withTable("t") { + sql(s"CREATE TABLE t USING parquet LOCATION '${dir.getCanonicalPath}'") + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "8") { + assertCodegenRan { + checkSparkAnswerAndOperator( + sql("SELECT withId(a, monotonically_increasing_id()), " + + "withId(b, monotonically_increasing_id()) FROM t")) + } + } + } + } + } + + test("Nondeterministic state persists across nullability flips within a partition") { + // Regression guard against re-introducing per-batch nullability into the cache key. Force a + // single parquet file with `spark.range(numPartitions=1)`, large enough that batch size 8 + // produces many batches in one scan partition. Null density varies by row range. If the + // dispatcher ever started deriving spec nullability from runtime data again, the cache key + // would flip mid-partition, the kernel would be re-allocated, and MII's counter would reset + // across the flip. + spark.udf.register("idPair", (id: Long, s: String) => (id, s)) + withTempPath { dir => + spark + .range(0, 200, 1, numPartitions = 1) + .selectExpr("CASE WHEN id >= 16 AND id < 32 THEN NULL ELSE concat('row_', id) END AS s") + .write + .parquet(dir.getCanonicalPath) + withTable("t") { + sql(s"CREATE TABLE t USING parquet LOCATION '${dir.getCanonicalPath}'") + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "8") { + assertCodegenRan { + checkSparkAnswerAndOperator( + sql("SELECT idPair(monotonically_increasing_id(), s) FROM t")) + } + } + } + } + } + + test("Nondeterministic state persists across two ScalaUDFs in one task") { + // The dispatcher is one instance per task (keyed by `(taskAttemptId, udfClassName)` in + // CometUdfBridge), so a plan with two distinct ScalaUDFs shares one CometScalaUDFCodegen. + // Two distinct closure-serialized expressions hit two cache entries. Per batch the + // dispatcher is invoked once for each. Each cache entry must stash its own kernel instance, + // otherwise the two expressions would fight for a shared kernel slot and stateful state + // (MII counter) would reset on every flip. + // + // Small batch size forces multiple batches over a small table so the per-key flip happens + // several times within one task. + spark.udf.register("idA", (id: Long) => id) + spark.udf.register("idB", (id: Long) => -id) + val rows = (0 until 64).map(i => s"row_$i") + withSubjects(rows: _*) { + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "8") { + assertCodegenRan { + checkSparkAnswerAndOperator( + sql( + "SELECT s, " + + "idA(monotonically_increasing_id()) AS a, " + + "idB(monotonically_increasing_id()) AS b FROM t")) + } + } + } + } + + test("per-task cache isolates UDF state across sequential task runs in one session") { + // Regression guard for the cache-scoping invariant on CometUdfBridge: instances live for + // exactly one Spark task and are dropped on task completion, so a stateful kernel sees a + // fresh instance per task. The query has to actually route through the dispatcher for this + // to test anything, so wrap `monotonically_increasing_id()` in a ScalaUDF identity. Running + // it twice in one session must produce results matching Spark each time. Under a cache that + // outlived a task and got reused by the next one, the counter would continue from the + // previous run's final value and the second run's IDs would diverge from Spark. Under a + // cache that was keyed by Tokio worker thread rather than task attempt ID, worker reuse + // across tasks would cause the same leak whenever the second task happened to be polled by + // the same worker. Two `checkSparkAnswerAndOperator` calls are stronger than asserting + // first == second: equality alone could pass if both runs are wrong-but-consistent (e.g. + // `init(partitionIndex)` never fires); matching Spark on both runs rules that out and + // implies cross-run equality because Spark is deterministic on the same query. + spark.udf.register("idPassthrough", (id: Long) => id) + val rows = (0 until 2048).map(i => s"row_$i") + withSubjects(rows: _*) { + val q = "SELECT s, idPassthrough(monotonically_increasing_id()) AS mid FROM t" + checkSparkAnswerAndOperator(sql(q)) + checkSparkAnswerAndOperator(sql(q)) + } + } + + /** + * Scalar ScalaUDF smoke tests. These prove that user-registered UDFs route through the codegen + * dispatcher rather than forcing a whole-plan Spark fallback. Spark's `ScalaUDF.doGenCode` + * already emits compilable Java that calls the user function via `ctx.addReferenceObj`, so the + * dispatcher's compile path picks it up for free. Tests that user-registered UDFs route through + * the dispatcher rather than forcing whole-plan Spark fallback. + */ + + test("registered string ScalaUDF routes through dispatcher") { + spark.udf.register("shout", (s: String) => if (s == null) null else s.toUpperCase + "!") + withSubjects("Abc", "xyz", null, "mixed") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT shout(s) FROM t")) + } + } + } + + test("registered Java UDF1 routes through dispatcher") { + // Java API path: `spark.udf.register(name, UDF1<...>, returnType)`. Spark wraps the Java + // functional interface in a Scala function and produces a `ScalaUDF` expression at plan + // time, so the dispatcher handles it the same as a Scala-registered UDF. Sanity check that + // both registration paths land on the same routing code. + spark.udf.register( + "javaLen", + new UDF1[String, Integer] { + override def call(s: String): Integer = if (s == null) -1 else s.length + }, + IntegerType) + withSubjects("abc", "hello", null, "x") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT javaLen(s) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarCharVector]), IntegerType) + } + } + + test("multi-arg ScalaUDF over string + literal routes through dispatcher") { + spark.udf.register( + "prepend", + (prefix: String, s: String) => if (s == null) null else prefix + s) + withSubjects("one", "two", null) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT prepend('[', s) FROM t")) + } + } + } + + test("ScalaUDF as a child of a native Spark expression") { + // The ScalaUDF routes through the dispatcher as a sub-expression. The surrounding `length` + // runs through Comet's native scalar function path. This exercises the cross-boundary + // composition where a dispatcher-compiled kernel returns a UTF8String that a native Comet + // expression then consumes. + spark.udf.register("wrap", (s: String) => if (s == null) null else s"|$s|") + withSubjects("abc", "def", null) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT length(wrap(s)) FROM t")) + } + } + } + + test("composed ScalaUDFs outer(inner(s)) fuse into one kernel") { + // Two user UDFs stacked, both operating on String. The dispatcher binds the whole tree and + // Spark's codegen emits two `ctx.addReferenceObj` calls inside one generated method. Races + // on the `ExpressionEncoder` serializers in `references` would show up here since each UDF + // contributes its own stateful serializer. The `freshReferences` closure in `CompiledKernel` + // is what keeps this correct across partitions. + spark.udf.register("inner", (s: String) => if (s == null) null else s.toUpperCase) + spark.udf.register("outer", (s: String) => if (s == null) null else s"<$s>") + withSubjects("abc", null, "xyz", "MiXeD") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT outer(inner(s)) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarCharVector]), StringType) + } + } + + test("ScalaUDFs of different types compose: isShort(len(s))") { + // Exercises an input type transition: String -> Int -> Boolean. Two user UDFs with + // different I/O type shapes in one tree, one Janino compile. + spark.udf.register("len", (s: String) => if (s == null) -1 else s.length) + spark.udf.register("isShort", (i: Int) => i < 5) + withSubjects("ab", "abcdef", null, "hi") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT isShort(len(s)) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarCharVector]), BooleanType) + } + } + + test("three-deep ScalaUDF composition lvl3(lvl2(lvl1(s)))") { + // Three user UDFs stacked in one tree: String -> String -> String -> Int. The fused kernel + // carries three `ctx.addReferenceObj` calls. `assertOneKernelForSubtree` asserts that the + // whole chain collapses into a single compile rather than one per nesting level. + // Null handling through composed UDFs is covered by the other composition tests above. + spark.udf.register("lvl1", (s: String) => if (s == null) null else s.toUpperCase) + spark.udf.register("lvl2", (s: String) => if (s == null) null else s.reverse) + spark.udf.register("lvl3", (s: String) => if (s == null) -1 else s.length) + withSubjects("abc", "hello world", "x") { + assertOneKernelForSubtree { + checkSparkAnswerAndOperator(sql("SELECT lvl3(lvl2(lvl1(s))) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarCharVector]), IntegerType) + } + } + + test("multi-column ScalaUDF composition join(upperU(c1), lowerU(c2))") { + // One multi-arg user UDF consuming two other user UDFs, each on a different input column. + // The bound tree has two BoundReferences, and the kernel is specialized on two VarCharVector + // columns. `assertOneKernelForSubtree` asserts that the two-branch composition fuses into a + // single kernel rather than one per branch or one per UDF. + // Input rows intentionally exclude nulls (see note on the three-deep test above). + spark.udf.register("upperU", (s: String) => if (s == null) null else s.toUpperCase) + spark.udf.register("lowerU", (s: String) => if (s == null) null else s.toLowerCase) + spark.udf.register( + "joinU", + (a: String, b: String) => if (a == null || b == null) null else s"$a-$b") + withTwoStringCols(("Abc", "XYZ"), ("Foo", "bar"), ("baz", "Bar"), ("Hi", "Lo")) { + assertOneKernelForSubtree { + checkSparkAnswerAndOperator(sql("SELECT joinU(upperU(c1), lowerU(c2)) FROM t")) + } + assertKernelSignaturePresent( + Seq(classOf[VarCharVector], classOf[VarCharVector]), + StringType) + } + } + + /** + * Per-primitive identity-UDF coverage. Each entry registers a `T => T` UDF over a parquet + * column declared at `sqlType` and asserts the dispatcher compiled a kernel for the matching + * `(vector class, output type)` pair. Parquet-backed (rather than `spark.range`-cast) tables + * keep the column's Arrow vector class aligned with the UDF signature. + */ + private def withTypedCol(sqlType: String, valueLiterals: String*)(f: => Unit): Unit = { + withTable("t") { + sql(s"CREATE TABLE t (c $sqlType) USING parquet") + if (valueLiterals.nonEmpty) { + val rows = valueLiterals.map(v => s"($v)").mkString(", ") + sql(s"INSERT INTO t VALUES $rows") + } + f + } + } + + private case class IdentityUdfCase( + label: String, + sqlType: String, + values: Seq[String], + vec: Class[_ <: ValueVector], + output: DataType, + udfName: String, + register: () => Unit) + + private val identityScalarCases: Seq[IdentityUdfCase] = Seq( + IdentityUdfCase( + "Boolean", + "BOOLEAN", + Seq("TRUE", "FALSE", "TRUE"), + classOf[BitVector], + BooleanType, + "u_bool", + () => spark.udf.register("u_bool", (b: Boolean) => !b)), + IdentityUdfCase( + "Byte", + "TINYINT", + Seq("CAST(1 AS TINYINT)", "CAST(2 AS TINYINT)", "CAST(100 AS TINYINT)"), + classOf[TinyIntVector], + ByteType, + "u_byte", + () => spark.udf.register("u_byte", (b: Byte) => (b + 1).toByte)), + IdentityUdfCase( + "Short", + "SMALLINT", + Seq("CAST(1 AS SMALLINT)", "CAST(2 AS SMALLINT)", "CAST(30000 AS SMALLINT)"), + classOf[SmallIntVector], + ShortType, + "u_short", + () => spark.udf.register("u_short", (s: Short) => (s + 1).toShort)), + IdentityUdfCase( + "Int", + "INT", + Seq("1", "2", "100"), + classOf[IntVector], + IntegerType, + "u_int", + () => spark.udf.register("u_int", (i: Int) => i * 2)), + IdentityUdfCase( + "Long", + "BIGINT", + Seq("1", "2", "100"), + classOf[BigIntVector], + LongType, + "u_long", + () => spark.udf.register("u_long", (l: Long) => l + 1L)), + IdentityUdfCase( + "Float", + "FLOAT", + Seq("CAST(1.5 AS FLOAT)", "CAST(2.5 AS FLOAT)"), + classOf[Float4Vector], + FloatType, + "u_float", + () => spark.udf.register("u_float", (f: Float) => f * 1.5f)), + IdentityUdfCase( + "Double", + "DOUBLE", + Seq("1.5", "2.5", "100.0"), + classOf[Float8Vector], + DoubleType, + "u_double", + () => spark.udf.register("u_double", (d: Double) => d / 2.0)), + IdentityUdfCase( + "Date", + "DATE", + Seq("DATE'2024-01-01'", "DATE'2024-06-15'", "DATE'1970-01-01'"), + classOf[DateDayVector], + DateType, + "u_date", + () => + spark.udf.register( + "u_date", + (d: java.sql.Date) => + if (d == null) null else new java.sql.Date(d.getTime + 86400000L))), + IdentityUdfCase( + "Timestamp", + "TIMESTAMP", + Seq("TIMESTAMP'2024-01-01 12:00:00'", "TIMESTAMP'2024-06-15 23:59:59'"), + classOf[TimeStampMicroTZVector], + TimestampType, + "u_ts", + () => + spark.udf.register( + "u_ts", + (t: java.sql.Timestamp) => + if (t == null) null else new java.sql.Timestamp(t.getTime + 1000L))), + IdentityUdfCase( + "TimestampNTZ", + "TIMESTAMP_NTZ", + Seq("TIMESTAMP_NTZ'2024-01-01 12:00:00'", "TIMESTAMP_NTZ'2024-06-15 23:59:59'"), + classOf[TimeStampMicroVector], + TimestampNTZType, + "u_tsntz", + () => + spark.udf.register( + "u_tsntz", + (ldt: java.time.LocalDateTime) => if (ldt == null) null else ldt.plusDays(1)))) + + identityScalarCases.foreach { c => + test(s"identity ScalaUDF on ${c.label} routes through dispatcher") { + c.register() + withTypedCol(c.sqlType, c.values: _*) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql(s"SELECT ${c.udfName}(c) FROM t")) + } + assertKernelSignaturePresent(Seq(c.vec), c.output) + } + } + } + + test("ScalaUDF returning a different type than its input") { + // String -> Int output transition. Identity-loop above keeps input == output. This asserts + // the writer can switch types per the UDF's declared return. + spark.udf.register("codePoint", (s: String) => if (s == null) 0 else s.codePointAt(0)) + withSubjects("abc", "A", null, "!") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT codePoint(s) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarCharVector]), IntegerType) + } + } + + test("ScalaUDF returning BinaryType") { + // Binary output writer path, exercised here by a user UDF for the first time. Before this + // the writer only had direct-compile unit tests. + spark.udf.register("bytes", (s: String) => if (s == null) null else s.getBytes("UTF-8")) + withSubjects("abc", null, "hello") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT bytes(s) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarCharVector]), BinaryType) + } + } + + test("ScalaUDF on BinaryType") { + // Binary input getter path: VarBinaryVector with byte[] reads via Spark's `getBinary` getter. + spark.udf.register("blen", (b: Array[Byte]) => if (b == null) -1 else b.length) + withTable("t") { + sql("CREATE TABLE t (b BINARY) USING parquet") + sql("INSERT INTO t VALUES (CAST('abc' AS BINARY)), (CAST('hello' AS BINARY)), (NULL)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT blen(b) FROM t")) + } + assertKernelSignaturePresent(Seq(classOf[VarBinaryVector]), IntegerType) + } + } + + test("ScalaUDF returning ArrayType(StringType)") { + // First use of the ArrayType output path end-to-end. The UDF returns a `Seq[String]`, + // which Spark encodes as `ArrayType(StringType, containsNull = true)`. The dispatcher's + // canHandle accepts it (ArrayType is supported when its element type is supported), + // allocateOutput builds a ListVector with an inner VarCharVector, and emitWrite recurses + // into the StringType case for the per-element UTF8 on-heap shortcut. End-to-end answer + // matches Spark. + spark.udf.register( + "splitComma", + (s: String) => if (s == null) null else s.split(",", -1).toSeq) + withSubjects("a,b,c", "x", null, "", "one,,three") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT splitComma(s) FROM t")) + } + } + } + + test("ScalaUDF returning ArrayType(IntegerType)") { + // Exercises ArrayType output with a primitive element. emitWrite's ArrayType case + // recurses into the IntegerType case for the inner write. No byte[] allocation involved. + spark.udf.register( + "asLengths", + (s: String) => if (s == null) null else s.split(",").map(_.length).toSeq) + withSubjects("a,bb,ccc", null, "xyzzy") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT asLengths(s) FROM t")) + } + } + } + + test("zero-column ScalaUDF produces one row per input row") { + // Non-deterministic (so Spark doesn't constant-fold) with a deterministic body (so + // Spark-vs-Comet comparison stays honest). The expression has no `AttributeReference`, + // so the serde produces an empty data-arg list and the dispatcher has no data column to + // read the batch size from. Guards the `numRows` path through the JNI bridge. + import org.apache.spark.sql.functions.udf + val alwaysHello = udf(() => "hello").asNondeterministic() + spark.udf.register("helloU", alwaysHello) + withSubjects("a", "b", null, "c") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT helloU() FROM t")) + } + } + } + + /** + * Decimal end-to-end: the dispatcher's `getDecimal` specializes per `DecimalType.precision` at + * source-generation time. Two representative cases here; `CometCodegenFuzzSuite` sweeps every + * shape across the boundary at varying null densities. + */ + private def withDecimalTable(decimalType: String, values: Seq[String])(f: => Unit): Unit = { + withTable("t") { + sql(s"CREATE TABLE t (d $decimalType) USING parquet") + val rows = values.map(v => if (v == null) "(NULL)" else s"($v)").mkString(", ") + if (values.nonEmpty) sql(s"INSERT INTO t VALUES $rows") + f + } + } + + test("ScalaUDF over Decimal(18, 9) routes through the unscaled-long fast path") { + // Boundary precision (18 == `MAX_LONG_DIGITS`) with a non-zero scale exercises the fractional + // branch of the fast-path encoding. + spark.udf.register("decIdShort", (d: java.math.BigDecimal) => d) + withDecimalTable( + "DECIMAL(18, 9)", + Seq("0.000000000", "1.123456789", "-1.123456789", "999999999.999999999", null)) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT decIdShort(d) FROM t")) + } + } + } + + test("ScalaUDF over Decimal(38, 10) routes through the BigDecimal slow path") { + // Pin the return type to Decimal(38, 10). TypeTag inference for `BigDecimal` would default to + // Decimal(38, 18), and under Spark 4 ANSI the encoder's CheckOverflow throws on the 28-digit + // boundary value below when rescaling 10 -> 18. + spark.udf.register( + "decIdLong", + new UDF1[java.math.BigDecimal, java.math.BigDecimal] { + override def call(d: java.math.BigDecimal): java.math.BigDecimal = d + }, + DecimalType(38, 10)) + withDecimalTable( + "DECIMAL(38, 10)", + Seq( + "0.0000000000", + "1.1234567890", + "-1.1234567890", + "9999999999999999999999999999.0000000000", + null)) { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT decIdLong(d) FROM t")) + } + } + } + + test("ScalaUDF sees TaskContext.partitionId() per partition") { + // Direct probe: register a ScalaUDF that reads TaskContext.partitionId() and returns it. + // Spark's own task thread has TaskContext set, so each partition's rows carry that + // partition's index. For the dispatcher to match Spark, the invocation thread must see a + // live TaskContext. With the `createPlan`-time TaskContext capture + bridge-side + // `TaskContext.setTaskContext` install (see `CometUdfBridge.evaluate` and + // `CometTaskContextShim`), Tokio workers see the propagated TaskContext and the UDF + // returns the real partitionId. Without that propagation, `TaskContext.get()` returns null + // on the Tokio thread and the sentinel (-1) leaks through, diverging from Spark. + spark.udf.register( + "pid", + (_: Long) => { + val tc = TaskContext.get() + if (tc != null) tc.partitionId() else -1 + }) + val df = spark + .range(0, 1024, 1, numPartitions = 4) + .selectExpr("id", "pid(id) as p") + checkSparkAnswerAndOperator(df) + } + + test("ScalaUDF sees TaskContext from fully-native parquet plan") { + // The `spark.range`-based test above runs through `CometSparkRowToColumnar`, which executes + // on a Spark task thread where TaskContext is live even without explicit propagation. The + // fully-native path through `CometNativeScan` runs the JVM UDF bridge on a Tokio worker + // thread where TaskContext.get() would otherwise be null. This test forces that path by + // sourcing from a Parquet table written as multiple files (so the native read produces + // multiple partitions) and asserting the UDF still sees the per-partition TaskContext via + // the `createPlan`-time capture + bridge-side install. + spark.udf.register( + "pidP", + (_: Int) => { + val tc = TaskContext.get() + if (tc != null) tc.partitionId() else -1 + }) + withTable("t") { + sql("CREATE TABLE t (x INT) USING parquet") + // Multiple INSERT statements -> multiple parquet files -> multiple read splits -> + // multiple partitions. + sql("INSERT INTO t VALUES (1), (2), (3), (4)") + sql("INSERT INTO t VALUES (5), (6), (7), (8)") + sql("INSERT INTO t VALUES (9), (10), (11), (12)") + sql("INSERT INTO t VALUES (13), (14), (15), (16)") + checkSparkAnswerAndOperator(sql("SELECT x, pidP(x) AS p FROM t")) + } + } + + test("Rand seeded per partition across a multi-partition table") { + // Rand.doGenCode registers an XORShiftRandom via ctx.addMutableState and seeds it via + // ctx.addPartitionInitializationStatement. That init statement runs inside our kernel's + // `init(int partitionIndex)`, called once per kernel allocation. Spark seeds + // `XORShiftRandom(seed + partitionIndex)` per partition, so different partitions produce + // different sequences for the same seed. Matching Spark across partitions requires the + // kernel to see the real partition index, which the dispatcher derives from + // `TaskContext.get().partitionId()`, live on this path thanks to the bridge-level + // TaskContext propagation. Composing with a ScalaUDF (identity on Double here) forces the + // tree through codegen dispatch so the Rand evaluation runs inside our kernel's init + // rather than via Spark's normal codegen. + spark.udf.register("dblId", (d: Double) => d) + val df = spark + .range(0, 1024, 1, numPartitions = 4) + .selectExpr("id", "dblId(rand(42)) as r") + checkSparkAnswerAndOperator(df) + } + + test("ScalaUDF composed with reused scalar subquery across projection and filter") { + // The same scalar subquery appears in two sites: the projection (which the dispatcher + // compiles into a fused kernel) and the filter (a separate operator). Each site holds its + // own `ScalarSubquery` expression instance with its own `@volatile result` field. Each + // surrounding operator's inherited `SparkPlan.waitForSubqueries` populates its instance's + // `result` before the dispatcher's bridge serializes the expression. The populated value + // travels through closure serialization into the cache key's bytes, so different subquery + // values compile distinct kernels. Exercises the full subquery-correctness invariant + // documented on `CometBatchKernelCodegen.canHandle`. + spark.udf.register("addOne", (i: Int) => i + 1) + withTable("t", "t2") { + sql("CREATE TABLE t (x INT) USING parquet") + sql("INSERT INTO t VALUES (1), (2), (3), (4), (5)") + sql("CREATE TABLE t2 (v INT) USING parquet") + sql("INSERT INTO t2 VALUES (2), (4)") + checkSparkAnswerAndOperator( + sql("SELECT addOne(x) + (SELECT max(v) FROM t2) AS r " + + "FROM t WHERE addOne(x) < (SELECT max(v) FROM t2) * 2")) + } + } + + /** + * ArrayType input. The dispatcher emits a nested `InputArray_col0` final class per array-typed + * input column; Spark's generated `getArray(ord)` resolves to our kernel's switch which returns + * the pre-allocated instance after resetting its start/length against the list's offsets. + * Element reads go through the typed child-vector field with no `ArrayData` copy or boxing. + * + * Each smoke test exercises the same serde/transport path at a different element type so the + * nested getter emitter's scalar-element cases are each covered: `StringType` (zero-copy + * `UTF8String.fromAddress`), `IntegerType` (primitive direct), and `DecimalType(p <= 18)` + * (decimal128 fast path). + */ + private def withArrayTable(colType: String, insertRows: String)(f: => Unit): Unit = { + withTable("t") { + sql(s"CREATE TABLE t (a $colType) USING parquet") + sql(s"INSERT INTO t VALUES $insertRows") + f + } + } + + test("ScalaUDF taking Seq[String] reads element by element") { + spark.udf.register( + "headOrNull", + (arr: Seq[String]) => if (arr == null || arr.isEmpty) null else arr.head) + withArrayTable( + "ARRAY", + "(array('a', 'b', 'c')), (array('x')), (null), (array()), (array('alone'))") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT headOrNull(a) FROM t")) + } + } + } + + test("ScalaUDF taking Seq[String] iterating all elements") { + spark.udf.register( + "concatArr", + (arr: Seq[String]) => if (arr == null) null else arr.mkString("|")) + withArrayTable( + "ARRAY", + "(array('one', 'two', 'three')), (array('solo')), (null), (array())") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT concatArr(a) FROM t")) + } + } + } + + test("ScalaUDF taking Seq[Int] reads primitive elements") { + spark.udf.register("sumArr", (arr: Seq[Int]) => if (arr == null) -1 else arr.sum) + withArrayTable( + "ARRAY", + "(array(1, 2, 3)), (array(-5, 5)), (array()), (null), (array(42))") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT sumArr(a) FROM t")) + } + } + } + + test("ScalaUDF taking Seq[BigDecimal] hits short-precision decimal fast path") { + // DecimalType(10, 2) is well inside p <= 18, so the nested-array `getDecimal` emits the + // unscaled-long fast path (see `emitNestedArrayElementGetter`). A `BigDecimal` UDF argument + // forces Spark's encoder to call `getDecimal(i, 10, 2)` on our nested ArrayData for each + // element, which exercises that code path end to end. + spark.udf.register( + "sumDecArr", + (arr: Seq[java.math.BigDecimal]) => + if (arr == null) null + else { + var acc = java.math.BigDecimal.ZERO + arr.foreach(v => if (v != null) acc = acc.add(v)) + acc + }) + withArrayTable( + "ARRAY", + "(array(1.23, 4.56)), (array(-9.99)), (null), (array())") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT sumDecArr(a) FROM t")) + } + } + } + + test("ScalaUDF composes with struct-field access reading Struct.age") { + // Keeps the UDF arg scalar (Int) but puts a `GetStructField` under it so the codegen + // dispatcher compiles the struct-input read path (`row.getStruct(0, 2).getInt(1)`). + spark.udf.register("doubleInt", (i: Int) => i * 2) + withTable("t") { + sql("CREATE TABLE t (s STRUCT) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(named_struct('name', 'alice', 'age', 30)), " + + "(named_struct('name', 'bob', 'age', 42)), " + + "(null)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT doubleInt(s.age) FROM t")) + } + } + } + + test("ScalaUDF taking full Struct value (case class arg)") { + // Case-class UDF arguments: test data must not include null top-level rows. + // `ScalaUDF.scalaConverter` applies Spark's `ExpressionEncoder.Deserializer` on every row + // to materialize the case-class instance. The generated deserializer has a + // `newInstance(NameAgePair)` step that throws `EXPRESSION_DECODING_FAILED` on a null input, + // independent of the dispatcher. Case-class UDF tests omit null top-level rows. Other + // tests with plain `Seq` / `Map` args can include nulls because the deserializer hands null + // to the UDF body which handles it. + spark.udf.register("fmtPair", (r: NameAgePair) => s"${r.name}:${r.age}") + withTable("t") { + sql("CREATE TABLE t (s STRUCT) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(named_struct('name', 'alice', 'age', 30)), " + + "(named_struct('name', 'bob', 'age', 42))") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT fmtPair(s) FROM t")) + } + } + } + + test("ScalaUDF returning Struct (case class output)") { + spark.udf.register("makePair", (i: Int) => NameAgePair(s"n$i", i)) + withTypedCol("INT", "1", "2", "3") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT makePair(c) FROM t")) + } + } + } + + test("ScalaUDF taking Map") { + spark.udf.register("sumMap", (m: Map[String, Int]) => if (m == null) -1 else m.values.sum) + withTable("t") { + sql("CREATE TABLE t (m MAP) USING parquet") + sql("INSERT INTO t VALUES (map('a', 1, 'b', 2)), (map()), (null)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT sumMap(m) FROM t")) + } + } + } + + test("ScalaUDF round-trips Map (primitive key and value)") { + // Map with non-string keys: exercises the primitive-key element getter on the input side + // and the corresponding writer on the output side. Spark's encoder for `Map[Int, Int]` calls + // `getInt(0)` / `getInt(1)` on the entries struct, hitting the kernel's typed scalar getter + // for each side rather than the UTF8 path. + spark.udf.register( + "incValues", + (m: Map[Int, Int]) => if (m == null) null else m.map { case (k, v) => k -> (v + 1) }) + withTable("t") { + sql("CREATE TABLE t (m MAP) USING parquet") + sql("INSERT INTO t VALUES (map(1, 10, 2, 20)), (map()), (null)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT incValues(m) FROM t")) + } + } + } + + test("ScalaUDF returning Map") { + spark.udf.register( + "singletonMap", + (s: String, i: Int) => if (s == null) null else Map(s -> i)) + withTable("t") { + sql("CREATE TABLE t (s STRING, i INT) USING parquet") + sql("INSERT INTO t VALUES ('a', 1), ('b', 2), (null, 3)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT singletonMap(s, i) FROM t")) + } + } + } + + test("ScalaUDF taking Map> exercises nested composition") { + spark.udf.register( + "totalLens", + (m: Map[String, Seq[Int]]) => if (m == null) -1 else m.values.flatten.sum) + withTable("t") { + sql("CREATE TABLE t (m MAP>) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(map('a', array(1, 2, 3), 'b', array(10))), " + + "(map()), " + + "(null)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT totalLens(m) FROM t")) + } + } + } + + test("ScalaUDF round-trips Array> (nested array input + output)") { + // Exercises nested-array input reads and nested-list output writes in one call: the inner + // `InputArray_col0_e` class on the input side and the recursive emitWrite on the output. + spark.udf.register( + "reverseRows", + (arr: Seq[Seq[Int]]) => if (arr == null) null else arr.map(_.reverse)) + withTable("t") { + sql("CREATE TABLE t (a ARRAY>) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(array(array(1, 2, 3), array(4, 5))), " + + "(array(array())), " + + "(null)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT reverseRows(a) FROM t")) + } + } + } + + test("ScalaUDF round-trips Struct>") { + // Struct with a complex field on both sides: input reads go through InputStruct_col0 + + // InputArray_col0_f1, output writes through StructVector + ListVector. + // Null top-level rows omitted - case-class arg. See the note on `fmtPair` above. + spark.udf.register( + "growItems", + (r: NameItems) => + if (r == null) null else NameItems(r.name, if (r.items == null) null else r.items :+ 0)) + withTable("t") { + sql("CREATE TABLE t (s STRUCT>) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(named_struct('name', 'a', 'items', array(1, 2))), " + + "(named_struct('name', 'b', 'items', array()))") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT growItems(s) FROM t")) + } + } + } + + test("ScalaUDF round-trips Map> (nested value both sides)") { + // Map input read goes through InputMap_col0 + InputArray_col0_v (the complex-value side); + // output write emits MapVector + entries Struct + per-value ListVector inside the map's + // entries struct. + spark.udf.register( + "sortValues", + (m: Map[String, Seq[Int]]) => + if (m == null) null + else m.map { case (k, v) => k -> (if (v == null) null else v.sorted) }) + withTable("t") { + sql("CREATE TABLE t (m MAP>) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(map('a', array(3, 1, 2), 'b', array(10))), " + + "(map()), " + + "(null)") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT sortValues(m) FROM t")) + } + } + } + + test("ScalaUDF round-trips Map>") { + // Struct value inside a map, both sides. Null top-level rows omitted - the map value is a + // case class. See the note on `fmtPair` above. + spark.udf.register( + "tagValues", + (m: Map[String, XyPair]) => + if (m == null) null + else + m.map { case (k, v) => k -> (if (v == null) null else XyPair(v.x + 1, s"<${v.y}>")) }) + withTable("t") { + sql("CREATE TABLE t (m MAP>) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(map('a', named_struct('x', 1, 'y', 'one'))), " + + "(map())") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT tagValues(m) FROM t")) + } + } + } + + test("array_distinct on Array> retains element identity across hash set") { + // Fuzz signal: cardinality(array_distinct(arr_of_struct)) returns 1 where Spark returns 2. + // Hypothesis: the kernel's InputStruct wrapper backing array_distinct's element reads is + // reused without resetting per-element state, so every hashed element looks identical and + // distinct collapses the array to a single entry. + spark.udf.register("idIntDistinct", (i: Int) => i) + withTable("t") { + sql("CREATE TABLE t (s ARRAY>) USING parquet") + sql( + "INSERT INTO t VALUES " + + "(array(named_struct('a', 1, 'b', 'x'), named_struct('a', 1, 'b', 'x'))), " + + "(array(named_struct('a', 1, 'b', 'x'), named_struct('a', 2, 'b', 'y'))), " + + "(array(named_struct('a', 1, 'b', 'x'), named_struct('a', 2, 'b', 'y'), " + + "named_struct('a', 1, 'b', 'x')))") + assertCodegenRan { + checkSparkAnswerAndOperator( + sql("SELECT idIntDistinct(cardinality(array_distinct(s))) FROM t")) + } + } + } + + test("array_max(flatten(arr)) on Array> with mixed null inner arrays") { + // Fuzz signal: array_max(flatten(arr)) returns empty byte arrays where Spark returns the + // actual max binary, with the empties sorting to the front of the output. Pattern points at + // cross-batch state pollution. Generate 100 rows of varied outer/inner shape, longer + // binaries, mixed nulls. Force multiple batches with a small batch size. + spark.udf.register("idBinFlat", (b: Array[Byte]) => b) + withSQLConf(CometConf.COMET_BATCH_SIZE.key -> "16") { + withTable("t") { + sql("CREATE TABLE t (a ARRAY>) USING parquet") + val rows = (0 until 100).map { i => + if (i % 11 == 0) { + "(NULL)" + } else { + val outerSize = (i % 5) + 1 + val inners = (0 until outerSize).map { j => + val pick = (i * 7 + j) % 13 + if (pick == 0) "array()" + else if (pick == 1) "NULL" + else { + val innerSize = ((i + j) % 4) + 1 + val bytes = (0 until innerSize).map { k => + val len = ((i + j + k) % 8) + 1 + val hex = (0 until len) + .map(b => f"${(i * 13 + j * 17 + k * 5 + b) & 0xff}%02x") + .mkString + s"X'$hex'" + } + "array(" + bytes.mkString(", ") + ")" + } + } + s"(array(${inners.mkString(", ")}))" + } + } + sql(s"INSERT INTO t VALUES ${rows.mkString(", ")}") + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT idBinFlat(array_max(flatten(a))) FROM t")) + } + } + } + } + + /** + * Regressions for nested reference-typed getter null handling. Spark's + * `CodeGenerator.setArrayElement` only emits an `isNullAt` check before `array.update(i, + * getX(j))` for Java primitives. For reference-typed elements (Binary, String, Decimal, Struct, + * Array, Map) it relies on the source's `getX` to return `null` itself, matching + * `ColumnarArray.getBinary`. Without that contract, inner nulls become empty bytes / empty + * strings / garbage decimals / non-null shells in the flattened output. + */ + + test("array_max(flatten(arr)) on Array> with null inner Binary returns null") { + spark.udf.register("idBin", (b: Array[Byte]) => b) + withArrayTable( + "ARRAY>", + "(array(array(NULL))), " + + "(array(array(NULL, NULL))), " + + "(array(array(), array(NULL)))") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT idBin(array_max(flatten(a))) FROM t")) + } + } + } + + test("array_max(flatten(arr)) on Array> with null inner String returns null") { + spark.udf.register("idStr", (s: String) => s) + withArrayTable( + "ARRAY>", + "(array(array(NULL))), " + + "(array(array(NULL, NULL))), " + + "(array(array(), array(NULL)))") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT idStr(array_max(flatten(a))) FROM t")) + } + } + } + + test( + "array_max(flatten(arr)) on Array> with null inner Decimal " + + "(short-precision fast path)") { + spark.udf.register("idDec10", (d: java.math.BigDecimal) => d) + withArrayTable( + "ARRAY>", + "(array(array(CAST(NULL AS DECIMAL(10, 2))))), " + + "(array(array(" + + "CAST(NULL AS DECIMAL(10, 2)), CAST(NULL AS DECIMAL(10, 2))))), " + + "(array(array(), array(CAST(NULL AS DECIMAL(10, 2)))))") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT idDec10(array_max(flatten(a))) FROM t")) + } + } + } + + test( + "array_max(flatten(arr)) on Array> with null inner Decimal " + + "(long-precision slow path)") { + spark.udf.register("idDec30", (d: java.math.BigDecimal) => d) + withArrayTable( + "ARRAY>", + "(array(array(CAST(NULL AS DECIMAL(30, 2))))), " + + "(array(array(" + + "CAST(NULL AS DECIMAL(30, 2)), CAST(NULL AS DECIMAL(30, 2))))), " + + "(array(array(), array(CAST(NULL AS DECIMAL(30, 2)))))") { + assertCodegenRan { + checkSparkAnswerAndOperator(sql("SELECT idDec30(array_max(flatten(a))) FROM t")) + } + } + } + + // Runtime coverage for nullable nested `getStruct` / `getArray` / `getMap` element reads is + // exercised through HOFs in `CometCodegenHOFSuite`. Static emitter assertions live in + // `CometCodegenSourceSuite`. +} + +/** + * Case class used by the struct-input / struct-output smoke tests. Must be declared at file scope + * (not inside the test class) so Spark's TypeTag-based UDF encoder can resolve the Spark + * `StructType` schema from the Scala class. + */ +private case class NameAgePair(name: String, age: Int) + +private case class NameItems(name: String, items: Seq[Int]) + +private case class XyPair(x: Int, y: String) diff --git a/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala index 399462d5ea..f7e6b03d33 100644 --- a/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala @@ -70,9 +70,7 @@ class CometCsvExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper test("to_csv - with configurable formatting options") { val table = "t1" - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT, - CometConf.getExprAllowIncompatConfigKey(classOf[StructsToCsv]) -> "true") { + withSQLConf(CometConf.getExprAllowIncompatConfigKey(classOf[StructsToCsv]) -> "true") { withTable(table) { val newLinesStr = """ abc diff --git a/spark/src/test/scala/org/apache/comet/CometDateTimeUtilsSuite.scala b/spark/src/test/scala/org/apache/comet/CometDateTimeUtilsSuite.scala index 0d900e8105..072009d7ab 100644 --- a/spark/src/test/scala/org/apache/comet/CometDateTimeUtilsSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometDateTimeUtilsSuite.scala @@ -178,7 +178,7 @@ class CometDateTimeUtilsSuite extends CometTestBase { test("string to timestamp - invalid formats return null") { // All of these should produce null (not throw) in non-ANSI mode. for (tz <- Seq("UTC", "America/Los_Angeles")) { - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz) { + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz, SQLConf.ANSI_ENABLED.key -> "false") { checkCastToTimestamp( Seq( "238", @@ -210,7 +210,9 @@ class CometDateTimeUtilsSuite extends CometTestBase { // "SPARK-35780: support full range of timestamp string" test("SPARK-35780: full range of timestamp string") { - withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC", + SQLConf.ANSI_ENABLED.key -> "false") { // Normal-range cases: collect() as TimestampType directly. checkCastToTimestamp( Seq( @@ -262,16 +264,21 @@ class CometDateTimeUtilsSuite extends CometTestBase { } test("SPARK-15379: invalid calendar dates in string to date cast") { - // Feb 29 on a non-leap year and Apr 31 must produce null for both DATE and TIMESTAMP. - checkCastToDate(Seq("2015-02-29 00:00:00", "2015-04-31 00:00:00", "2015-02-29", "2015-04-31")) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + // Feb 29 on a non-leap year and Apr 31 must produce null for both DATE and TIMESTAMP. + checkCastToDate( + Seq("2015-02-29 00:00:00", "2015-04-31 00:00:00", "2015-02-29", "2015-04-31")) - checkCastToTimestamp( - Seq("2015-02-29 00:00:00", "2015-04-31 00:00:00", "2015-02-29", "2015-04-31")) + checkCastToTimestamp( + Seq("2015-02-29 00:00:00", "2015-04-31 00:00:00", "2015-02-29", "2015-04-31")) + } } test("trailing characters while converting string to timestamp") { // Garbage after a valid ISO timestamp must make the whole value null. - checkCastToTimestamp(Seq("2019-10-31T10:59:23Z:::")) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + checkCastToTimestamp(Seq("2019-10-31T10:59:23Z:::")) + } } test("DST spring-forward gap and fall-back overlap") { diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 9b5e85be3b..a172538f45 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -23,9 +23,6 @@ import java.time.{Duration, Period} import scala.util.Random -import org.scalactic.source.Position -import org.scalatest.Tag - import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, FromUnixTime, Literal, StructsToJson, TruncDate, TruncTimestamp} @@ -44,15 +41,6 @@ import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator} class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ - override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { - super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { - testFun - } - } - } - val ARITHMETIC_OVERFLOW_EXCEPTION_MSG = """[ARITHMETIC_OVERFLOW] integer overflow. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error""" val DIVIDE_BY_ZERO_EXCEPTION_MSG = @@ -139,7 +127,9 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("compare true/false to negative zero") { Seq(false, true).foreach { dictionary => - withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { + withSQLConf( + SQLConf.ANSI_ENABLED.key -> "false", + "parquet.enable.dictionary" -> dictionary.toString) { val table = "test" withTable(table) { sql(s"create table $table(col1 boolean, col2 float) using parquet") @@ -165,6 +155,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("decimals divide by zero") { Seq(true, false).foreach { dictionary => withSQLConf( + SQLConf.ANSI_ENABLED.key -> "false", SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false", "parquet.enable.dictionary" -> dictionary.toString) { withTempPath { dir => @@ -184,12 +175,14 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("Integral Division Overflow Handling Matches Spark Behavior") { - withTable("t1") { - val value = Long.MinValue - sql("create table t1(c1 long, c2 short) using parquet") - sql(s"insert into t1 values($value, -1)") - val res = sql("select c1 div c2 from t1 order by c1") - checkSparkAnswerAndOperator(res) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withTable("t1") { + val value = Long.MinValue + sql("create table t1(c1 long, c2 short) using parquet") + sql(s"insert into t1 values($value, -1)") + val res = sql("select c1 div c2 from t1 order by c1") + checkSparkAnswerAndOperator(res) + } } } @@ -414,19 +407,21 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("try_divide") { - val data = Seq((15121991, 0)) - withParquetTable(data, "tbl") { - checkSparkAnswerAndOperator("SELECT try_divide(_1, _2) FROM tbl") - checkSparkAnswerAndOperator(""" - |SELECT - | try_divide(10, 0), - | try_divide(NULL, 5), - | try_divide(5, NULL), - | try_divide(-2147483648, -1), - | try_divide(-9223372036854775808, -1), - | try_divide(DECIMAL('9999999999999999999999999999'), 0.1) - | from tbl - |""".stripMargin) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + val data = Seq((15121991, 0)) + withParquetTable(data, "tbl") { + checkSparkAnswerAndOperator("SELECT try_divide(_1, _2) FROM tbl") + checkSparkAnswerAndOperator(""" + |SELECT + | try_divide(10, 0), + | try_divide(NULL, 5), + | try_divide(5, NULL), + | try_divide(-2147483648, -1), + | try_divide(-9223372036854775808, -1), + | try_divide(DECIMAL('9999999999999999999999999999'), 0.1) + | from tbl + |""".stripMargin) + } } } @@ -1707,6 +1702,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("from_unixtime") { Seq(false, true).foreach { dictionary => withSQLConf( + SQLConf.ANSI_ENABLED.key -> "false", "parquet.enable.dictionary" -> dictionary.toString, CometConf.getExprAllowIncompatConfigKey(classOf[FromUnixTime]) -> "true") { val table = "test" @@ -1746,6 +1742,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("Decimal binary ops multiply is aligned to Spark") { Seq(true, false).foreach { allowPrecisionLoss => withSQLConf( + SQLConf.ANSI_ENABLED.key -> "false", "spark.sql.decimalOperations.allowPrecisionLoss" -> allowPrecisionLoss.toString) { testSingleLineQuery( @@ -1840,6 +1837,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { sql(s"insert into $table values $values") Seq(true, false).foreach { allowPrecisionLoss => withSQLConf( + SQLConf.ANSI_ENABLED.key -> "false", "spark.sql.decimalOperations.allowPrecisionLoss" -> allowPrecisionLoss.toString) { val a = makeNum(p1, s1) val b = makeNum(p2, s2) @@ -1869,15 +1867,17 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - // Supported boolean values as true by both Arrow and Spark - testCastedColumn(inputValues = Seq("t", "true", "y", "yes", "1", "T", "TrUe", "Y", "YES")) - // Supported boolean values as false by both Arrow and Spark - testCastedColumn(inputValues = Seq("f", "false", "n", "no", "0", "F", "FaLSe", "N", "No")) - // Supported boolean values by Arrow but not Spark - testCastedColumn(inputValues = - Seq("TR", "FA", "tr", "tru", "ye", "on", "fa", "fal", "fals", "of", "off")) - // Invalid boolean casting values for Arrow and Spark - testCastedColumn(inputValues = Seq("car", "Truck")) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + // Supported boolean values as true by both Arrow and Spark + testCastedColumn(inputValues = Seq("t", "true", "y", "yes", "1", "T", "TrUe", "Y", "YES")) + // Supported boolean values as false by both Arrow and Spark + testCastedColumn(inputValues = Seq("f", "false", "n", "no", "0", "F", "FaLSe", "N", "No")) + // Supported boolean values by Arrow but not Spark + testCastedColumn(inputValues = + Seq("TR", "FA", "tr", "tru", "ye", "on", "fa", "fal", "fals", "of", "off")) + // Invalid boolean casting values for Arrow and Spark + testCastedColumn(inputValues = Seq("car", "Truck")) + } } test("explain comet") { @@ -2516,7 +2516,6 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true") { val df = spark.read.parquet(dir.toString()) @@ -2546,7 +2545,6 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { withSQLConf( CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true") { val df = spark.read.parquet(dir.toString()) @@ -3014,7 +3012,6 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "false", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> "native_datafusion", SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "true", SQLConf.COLUMN_VECTOR_OFFHEAP_ENABLED.key -> offheapEnabled.toString, // SPARK-53535 (Spark 4.1+) flipped the default to "false", which preserves the parent @@ -3042,12 +3039,15 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } test("test length function") { - withTable("t1") { - sql( - "create table t1 using parquet as select cast(id as string) as c1, cast(id as binary) as c2 from range(10)") - // FIXME: Change checkSparkAnswer to checkSparkAnswerAndOperator after resolving - // https://github.com/apache/datafusion-comet/issues/2348 - checkSparkAnswer("select length(c1), length(c2) AS x FROM t1 ORDER BY c1") + // cast(id as binary) is rejected by Spark 4 ANSI analyzer + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + withTable("t1") { + sql( + "create table t1 using parquet as select cast(id as string) as c1, cast(id as binary) as c2 from range(10)") + // FIXME: Change checkSparkAnswer to checkSparkAnswerAndOperator after resolving + // https://github.com/apache/datafusion-comet/issues/2348 + checkSparkAnswer("select length(c1), length(c2) AS x FROM t1 ORDER BY c1") + } } } diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzMathSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzMathSuite.scala index d48712ed54..50a4425f97 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzMathSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzMathSuite.scala @@ -19,10 +19,18 @@ package org.apache.comet +import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, IntegerType, LongType} class CometFuzzMathSuite extends CometFuzzTestBase { + // The integer-math fuzz tests intentionally cover overflowing inputs; under Spark 4's + // default ANSI mode those would throw rather than produce a result for both Spark and + // Comet to compare against. Pin ANSI off so the parity comparison can run. + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.ANSI_ENABLED.key, "false") + for (op <- Seq("+", "-", "*", "/", "div")) { test(s"integer math: $op") { val df = spark.read.parquet(filename) diff --git a/spark/src/test/scala/org/apache/comet/CometHashExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometHashExpressionSuite.scala index 563ee18520..23a539fa1e 100644 --- a/spark/src/test/scala/org/apache/comet/CometHashExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometHashExpressionSuite.scala @@ -21,9 +21,6 @@ package org.apache.comet import scala.util.Random -import org.scalactic.source.Position -import org.scalatest.Tag - import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -37,15 +34,6 @@ import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, ParquetGener */ class CometHashExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { - override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { - super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { - testFun - } - } - } - test("hash - boolean") { withTable("t") { sql("CREATE TABLE t(c BOOLEAN) USING parquet") diff --git a/spark/src/test/scala/org/apache/comet/CometIcebergRewriteActionSuite.scala b/spark/src/test/scala/org/apache/comet/CometIcebergRewriteActionSuite.scala index 9622960932..53454d0034 100644 --- a/spark/src/test/scala/org/apache/comet/CometIcebergRewriteActionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometIcebergRewriteActionSuite.scala @@ -65,11 +65,11 @@ class CometIcebergRewriteActionSuite extends CometTestBase with CometIcebergTest } // Single-column zOrder is bit-pattern-equivalent to a natural sort (no second dimension to - // interleave with), so we expect the same ascending output as the sort test. The shuffle here - // is CometColumnarExchange rather than CometExchange because the z-value column is computed - // by a Spark Project (Iceberg's INTERLEAVE_BYTES / INT_ORDERED_BYTES are not recognised by - // Comet), so the path crosses a JVM-row boundary before the shuffle. - test("single-column zOrder rewrite runs scan, columnar exchange, and sort natively in Comet") { + // interleave with), so we expect the same ascending output as the sort test. Iceberg's + // `INT_ORDERED_BYTES` / `INTERLEAVE_BYTES` are `ScalaUDF`s that route through Comet's codegen + // dispatcher, so the project stays native and the shuffle picks `CometExchange` / + // `CometNativeShuffle` rather than the columnar-row roundtrip path. + test("single-column zOrder rewrite runs scan, native exchange, and sort natively in Comet") { runRewriteTest( RewriteCase( table = s"$catalog.db.zorder_test", @@ -77,7 +77,7 @@ class CometIcebergRewriteActionSuite extends CometTestBase with CometIcebergTest verifyDataAfter = assertSortedById, verifyPlans = { rewritePlans => assertReadsAreComet(rewritePlans) - assertOperator(rewritePlans, "CometColumnarExchange") + assertOperator(rewritePlans, "CometExchange") assertOperator(rewritePlans, "CometSort") })) } @@ -416,7 +416,8 @@ class CometIcebergRewriteActionSuite extends CometTestBase with CometIcebergTest s"spark.sql.catalog.$catalog.warehouse" -> warehouseDir.getAbsolutePath, CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true")(body) + CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + CometConf.COMET_SCALA_UDF_CODEGEN_ENABLED.key -> "true")(body) /** Creates an Iceberg table with `numFiles` separate appends, each producing one data file. */ private def createMultiFileTable(table: String, numFiles: Int): Unit = { diff --git a/spark/src/test/scala/org/apache/comet/CometMathExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometMathExpressionSuite.scala index 9d27f2d25f..fdad9e3d3d 100644 --- a/spark/src/test/scala/org/apache/comet/CometMathExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometMathExpressionSuite.scala @@ -32,11 +32,13 @@ import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator} class CometMathExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("abs") { - val df = createTestData(generateNegativeZero = false) - df.createOrReplaceTempView("tbl") - for (field <- df.schema.fields) { - val col = field.name - checkSparkAnswerAndOperator(s"SELECT $col, abs($col) FROM tbl ORDER BY $col") + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + val df = createTestData(generateNegativeZero = false) + df.createOrReplaceTempView("tbl") + for (field <- df.schema.fields) { + val col = field.name + checkSparkAnswerAndOperator(s"SELECT $col, abs($col) FROM tbl ORDER BY $col") + } } } diff --git a/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala index fd6d3ef535..9c34b3a3ce 100644 --- a/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala @@ -54,19 +54,6 @@ class CometNativeSuite extends CometTestBase { assert(exception.getMessage contains "java.lang.NullPointerException") } - test("handling NPE when closing null handle of parquet reader") { - assert(NativeBase.isLoaded) - val exception1 = intercept[NullPointerException] { - parquet.Native.closeRecordBatchReader(0) - } - assert(exception1.getMessage contains "null batch context handle") - - val exception2 = intercept[NullPointerException] { - parquet.Native.closeColumnReader(0) - } - assert(exception2.getMessage contains "null context handle") - } - test("Comet native should use spark local dir as temp dir") { withParquetTable((0 until 100000).map(i => (i, i + 1)), "table") { val dirs = SparkEnv.get.blockManager.getLocalDiskDirs diff --git a/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala b/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala index dbb62d5a48..268fdf94eb 100644 --- a/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometSparkSessionExtensionsSuite.scala @@ -29,6 +29,9 @@ class CometSparkSessionExtensionsSuite extends CometTestBase { test("isCometLoaded") { val conf = new SQLConf + // Disable Comet shuffle so this test can focus on other checks without needing + // spark.shuffle.manager to be set. + conf.setConfString(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "false") conf.setConfString(CometConf.COMET_ENABLED.key, "false") assert(!isCometLoaded(conf)) @@ -50,6 +53,25 @@ class CometSparkSessionExtensionsSuite extends CometTestBase { NativeBase.setLoaded(true) } + test("isCometLoaded requires CometShuffleManager when shuffle.enabled=true") { + val conf = new SQLConf + conf.setConfString(CometConf.COMET_ENABLED.key, "true") + + // Default: shuffle.enabled=true. Without spark.shuffle.manager set, Comet must be disabled. + assert(!isCometLoaded(conf)) + + // Opt out: shuffle.enabled=false. Comet should load (assumes native lib is available). + conf.setConfString(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "false") + assert(isCometLoaded(conf)) + + // shuffle.enabled=true with the Comet shuffle manager registered: Comet should load. + conf.setConfString(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") + conf.setConfString( + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") + assert(isCometLoaded(conf)) + } + test("Arrow properties") { NativeBase.setLoaded(false) NativeBase.load() diff --git a/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala index 5a0b34e056..47642f2357 100644 --- a/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala @@ -21,23 +21,12 @@ package org.apache.comet import java.io.File -import org.scalactic.source.Position -import org.scalatest.Tag - import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.internal.SQLConf class CometSqlFileTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { - override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { - super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { - testFun - } - } - } - /** Check if the current Spark version meets a minimum version requirement. */ private def meetsMinSparkVersion(minVersion: String): Boolean = { val current = org.apache.spark.SPARK_VERSION.split("[.-]").take(2).map(_.toInt) @@ -77,8 +66,13 @@ class CometSqlFileTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { "spark.sql.optimizer.excludedRules" -> "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") + // Most SQL fixtures here predate Spark 4 ANSI default and expect non-ANSI semantics + // (silent overflow/null on bad input). Individual files can opt in via their own + // --CONFIG line, which appears later in the pair list and wins. + private val ansiDisabled = Seq(SQLConf.ANSI_ENABLED.key -> "false") + private def runTestFile(relativePath: String, file: SqlTestFile): Unit = { - val allConfigs = file.configs ++ constantFoldingExcluded + val allConfigs = ansiDisabled ++ file.configs ++ constantFoldingExcluded withSQLConf(allConfigs: _*) { withTable(file.tables: _*) { file.records.foreach { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index 756efcd109..cd0beb56cc 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -22,6 +22,7 @@ package org.apache.comet.exec import scala.util.Random import org.apache.hadoop.fs.Path +import org.apache.spark.SparkConf import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.Cast import org.apache.spark.sql.catalyst.optimizer.EliminateSorts @@ -42,6 +43,11 @@ import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, ParquetGener class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { import testImplicits._ + // Several aggregate tests exercise overflow behavior expected to wrap around silently; + // ANSI-mode variants opt in to ANSI explicitly via withSQLConf. + override protected def sparkConf: SparkConf = + super.sparkConf.set(SQLConf.ANSI_ENABLED.key, "false") + test("min/max floating point with negative zero") { val r = new Random(42) val schema = StructType( diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 4a02a26119..16601d056b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -60,9 +60,7 @@ class CometExecSuite extends CometTestBase { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { super.test(testName, testTags: _*) { - withSQLConf( - CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { testFun } } @@ -2269,50 +2267,41 @@ class CometExecSuite extends CometTestBase { } test("Comet native metrics: scan") { - Seq(CometConf.SCAN_NATIVE_DATAFUSION, CometConf.SCAN_NATIVE_ICEBERG_COMPAT).foreach { - scanMode => - withSQLConf( - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanMode) { - withTempDir { dir => - val path = new Path(dir.toURI.toString, "native-scan.parquet") - makeParquetFileAllPrimitiveTypes(path, dictionaryEnabled = true, 10000) - withParquetTable(path.toString, "tbl") { - val df = sql("SELECT * FROM tbl") - df.collect() + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "true") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "native-scan.parquet") + makeParquetFileAllPrimitiveTypes(path, dictionaryEnabled = true, 10000) + withParquetTable(path.toString, "tbl") { + val df = sql("SELECT * FROM tbl") + df.collect() - val scan = find(df.queryExecution.executedPlan)(s => - s.isInstanceOf[CometScanExec] || s.isInstanceOf[CometNativeScanExec]) - assert(scan.isDefined, s"Expected to find a Comet scan node for $scanMode") - val metrics = scan.get.metrics + val scan = find(df.queryExecution.executedPlan)(s => + s.isInstanceOf[CometScanExec] || s.isInstanceOf[CometNativeScanExec]) + assert(scan.isDefined, "Expected to find a Comet scan node") + val metrics = scan.get.metrics - assert( - metrics.contains("time_elapsed_scanning_total"), - s"[$scanMode] Missing time_elapsed_scanning_total. Available: ${metrics.keys}") - assert(metrics.contains("bytes_scanned")) - assert(metrics.contains("output_rows")) - assert(metrics.contains("time_elapsed_opening")) - assert(metrics.contains("time_elapsed_processing")) - assert(metrics.contains("time_elapsed_scanning_until_data")) - assert( - metrics("time_elapsed_scanning_total").value > 0, - s"[$scanMode] time_elapsed_scanning_total should be > 0") - assert( - metrics("bytes_scanned").value > 0, - s"[$scanMode] bytes_scanned should be > 0") - assert(metrics("output_rows").value > 0, s"[$scanMode] output_rows should be > 0") - assert( - metrics("time_elapsed_opening").value > 0, - s"[$scanMode] time_elapsed_opening should be > 0") - assert( - metrics("time_elapsed_processing").value > 0, - s"[$scanMode] time_elapsed_processing should be > 0") - assert( - metrics("time_elapsed_scanning_until_data").value > 0, - s"[$scanMode] time_elapsed_scanning_until_data should be > 0") - } - } + assert( + metrics.contains("time_elapsed_scanning_total"), + s"Missing time_elapsed_scanning_total. Available: ${metrics.keys}") + assert(metrics.contains("bytes_scanned")) + assert(metrics.contains("output_rows")) + assert(metrics.contains("time_elapsed_opening")) + assert(metrics.contains("time_elapsed_processing")) + assert(metrics.contains("time_elapsed_scanning_until_data")) + assert( + metrics("time_elapsed_scanning_total").value > 0, + "time_elapsed_scanning_total should be > 0") + assert(metrics("bytes_scanned").value > 0, "bytes_scanned should be > 0") + assert(metrics("output_rows").value > 0, "output_rows should be > 0") + assert(metrics("time_elapsed_opening").value > 0, "time_elapsed_opening should be > 0") + assert( + metrics("time_elapsed_processing").value > 0, + "time_elapsed_processing should be > 0") + assert( + metrics("time_elapsed_scanning_until_data").value > 0, + "time_elapsed_scanning_until_data should be > 0") } + } } } @@ -3605,6 +3594,8 @@ class CometExecSuite extends CometTestBase { "struct(id)").foreach { valueType => { withSQLConf( + // cast(id as tinyint) overflows for id >= 128, which throws under ANSI + SQLConf.ANSI_ENABLED.key -> "false", SQLConf.USE_V1_SOURCE_LIST.key -> v1List, CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "false", CometConf.COMET_CONVERT_FROM_PARQUET_ENABLED.key -> "true", @@ -3943,8 +3934,7 @@ class CometExecSuite extends CometTestBase { withSQLConf( CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> "native_datafusion") { + CometConf.COMET_EXEC_ENABLED.key -> "true") { val df = spark.read.parquet(path) // Trigger two different actions to ensure metrics are not duplicated diff --git a/spark/src/test/scala/org/apache/comet/exec/CometGenerateExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometGenerateExecSuite.scala index a9ac3deb34..d03522afe1 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometGenerateExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometGenerateExecSuite.scala @@ -232,4 +232,158 @@ class CometGenerateExecSuite extends CometTestBase { } } + test("posexplode with simple array") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq((1, Array(10, 20, 30)), (2, Array(40, 50)), (3, Array(60))) + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode with empty array") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq((1, Array(1, 2)), (2, Array.empty[Int]), (3, Array(3))) + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode with null array") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq((1, Some(Array(1, 2))), (2, None), (3, Some(Array(3)))) + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode_outer with simple array") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[GenerateExec]) -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq((1, Array(10, 20, 30)), (2, Array(40, 50)), (3, Array(60))) + .toDF("id", "arr") + .selectExpr("id", "posexplode_outer(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode with array of strings") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq((1, Array("a", "b", "c")), (2, Array("d", "e")), (3, Array("f"))) + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode with nullable elements") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq( + (1, Array[Option[Int]](Some(1), None, Some(3))), + (2, Array[Option[Int]](None, Some(5))), + (3, Array[Option[Int]](Some(6)))) + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode with multiple projected columns") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = + Seq((1, "A", Array(10, 20, 30)), (2, "B", Array(40, 50)), (3, "C", Array(60))) + .toDF("id", "name", "arr") + .selectExpr("id", "name", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode with map input falls back") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq((1, Map("a" -> 1, "b" -> 2)), (2, Map("c" -> 3))) + .toDF("id", "map") + .selectExpr("id", "posexplode(map) as (pos, key, value)") + checkSparkAnswerAndFallbackReason( + df, + "Comet only supports explode/explode_outer for arrays, not maps") + } + } + + test("posexplode with array of structs") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq( + (1, Array((10, "a"), (20, "b"))), + (2, Array((30, "c"))), + (3, Array.empty[(Int, String)])) + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + .selectExpr("id", "pos", "value._1 as v1", "value._2 as v2") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode in lateral view") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + withTempView("t") { + Seq((1, Array(10, 20, 30)), (2, Array(40, 50)), (3, Array(60))) + .toDF("id", "arr") + .createOrReplaceTempView("t") + val df = + sql("SELECT t.id, p.pos, p.col FROM t LATERAL VIEW posexplode(t.arr) p AS pos, col") + checkSparkAnswerAndOperator(df) + } + } + } + + test("posexplode of literal array") { + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true") { + val df = Seq(1, 2, 3) + .toDF("id") + .selectExpr("id", "posexplode(array(100, 200, 300)) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + + test("posexplode across batch boundary with small batch size") { + // Force ScanExec to emit multiple small batches so that UnnestExec sees the parallel + // positions/values lists across batch boundaries. Element values are non-trivial so wrong + // alignment between pos and value would be visible in the answer. + withSQLConf( + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true", + CometConf.COMET_EXEC_EXPLODE_ENABLED.key -> "true", + CometConf.COMET_BATCH_SIZE.key -> "4") { + val rows = (1 to 12).map { i => + (i, (0 until (i % 5 + 1)).map(j => i * 100 + j).toArray) + } + val df = rows + .toDF("id", "arr") + .selectExpr("id", "posexplode(arr) as (pos, value)") + checkSparkAnswerAndOperator(df) + } + } + } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala index 0e9bd9dd6f..b858fe5c83 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeColumnarToRowSuite.scala @@ -46,9 +46,7 @@ class CometNativeColumnarToRowSuite extends CometTestBase with AdaptiveSparkPlan override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { super.test(testName, testTags: _*) { - withSQLConf( - CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { + withSQLConf(CometConf.COMET_NATIVE_COLUMNAR_TO_ROW_ENABLED.key -> "true") { testFun } } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeReaderSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeReaderSuite.scala index 69397272cb..19f2c7d4f6 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeReaderSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeReaderSuite.scala @@ -42,17 +42,15 @@ class CometNativeReaderSuite extends CometTestBase with AdaptiveSparkPlanHelper override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { - Seq(CometConf.SCAN_NATIVE_DATAFUSION, CometConf.SCAN_NATIVE_ICEBERG_COMPAT).foreach(scan => - super.test(s"$testName - $scan", testTags: _*) { - withSQLConf( - CometConf.COMET_EXEC_ENABLED.key -> "true", - SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "false", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> scan) { - testFun - } - }) + super.test(testName, testTags: _*) { + withSQLConf( + CometConf.COMET_EXEC_ENABLED.key -> "true", + SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "false") { + testFun + } + } } test("native reader case sensitivity") { diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index b673be509c..e0ef1df1f4 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -84,9 +84,7 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper Seq("false", "true").foreach { _ => Seq(10, 201).foreach { numPartitions => Seq("1.0", "10.0").foreach { ratio => - withSQLConf( - CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> ratio, - CometConf.COMET_NATIVE_SCAN_IMPL.key -> "native_datafusion") { + withSQLConf(CometConf.COMET_SHUFFLE_PREFER_DICTIONARY_RATIO.key -> ratio) { withParquetTable( (0 until 50).map(i => (i, Seq(Seq(i + 1), Seq(i + 2), Seq(i + 3)), i + 1)), "tbl") { @@ -528,7 +526,6 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper .parquet(dir.toString) } withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, CometConf.COMET_EXEC_SHUFFLE_WITH_ROUND_ROBIN_PARTITIONING_ENABLED.key -> "true") { val testDF = spark.read.parquet(dir.toString).repartition(10) // Verify CometShuffleExchangeExec is in the plan diff --git a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala index 23acc2b16d..544cd91bd2 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala @@ -42,8 +42,7 @@ class CometWindowExecSuite extends CometTestBase { super.test(testName, testTags: _*) { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_EXEC_WINDOW_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { + CometConf.COMET_EXEC_WINDOW_ENABLED.key -> "true") { testFun } } diff --git a/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCaseWhenSuite.scala b/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCaseWhenSuite.scala index baae516981..8e825462c4 100644 --- a/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCaseWhenSuite.scala +++ b/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCaseWhenSuite.scala @@ -19,25 +19,11 @@ package org.apache.comet.expressions.conditional -import org.scalactic.source.Position -import org.scalatest.Tag - import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.comet.CometConf - class CometCaseWhenSuite extends CometTestBase with AdaptiveSparkPlanHelper { - override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { - super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { - testFun - } - } - } - test("case_when") { Seq(false, true).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { diff --git a/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCoalesceSuite.scala b/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCoalesceSuite.scala index 40f2b45dd8..b4e4ceb6d9 100644 --- a/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCoalesceSuite.scala +++ b/spark/src/test/scala/org/apache/comet/expressions/conditional/CometCoalesceSuite.scala @@ -19,27 +19,13 @@ package org.apache.comet.expressions.conditional -import org.scalactic.source.Position -import org.scalatest.Tag - import org.apache.hadoop.fs.Path import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.internal.SQLConf -import org.apache.comet.CometConf - class CometCoalesceSuite extends CometTestBase with AdaptiveSparkPlanHelper { - override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { - super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { - testFun - } - } - } - test("coalesce should return correct datatype") { Seq(true, false).foreach { dictionaryEnabled => withTempDir { dir => diff --git a/spark/src/test/scala/org/apache/comet/expressions/conditional/CometIfSuite.scala b/spark/src/test/scala/org/apache/comet/expressions/conditional/CometIfSuite.scala index a206b21ad4..788667aad2 100644 --- a/spark/src/test/scala/org/apache/comet/expressions/conditional/CometIfSuite.scala +++ b/spark/src/test/scala/org/apache/comet/expressions/conditional/CometIfSuite.scala @@ -19,25 +19,11 @@ package org.apache.comet.expressions.conditional -import org.scalactic.source.Position -import org.scalatest.Tag - import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.comet.CometConf - class CometIfSuite extends CometTestBase with AdaptiveSparkPlanHelper { - override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit - pos: Position): Unit = { - super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { - testFun - } - } - } - test("if expression") { Seq(false, true).foreach { dictionary => withSQLConf("parquet.enable.dictionary" -> dictionary.toString) { diff --git a/spark/src/test/scala/org/apache/comet/objectstore/NativeConfigSuite.scala b/spark/src/test/scala/org/apache/comet/objectstore/NativeConfigSuite.scala index 4c4e275cac..7ba40192f4 100644 --- a/spark/src/test/scala/org/apache/comet/objectstore/NativeConfigSuite.scala +++ b/spark/src/test/scala/org/apache/comet/objectstore/NativeConfigSuite.scala @@ -21,21 +21,12 @@ package org.apache.comet.objectstore import java.net.URI -import scala.collection.mutable - -import org.scalatest.BeforeAndAfterEach import org.scalatest.funsuite.AnyFunSuite import org.scalatest.matchers.should.Matchers import org.apache.hadoop.conf.Configuration -import org.apache.comet.rules.CometScanRule - -class NativeConfigSuite extends AnyFunSuite with Matchers with BeforeAndAfterEach { - - override protected def beforeEach(): Unit = { - CometScanRule.configValidityMap.clear() - } +class NativeConfigSuite extends AnyFunSuite with Matchers { test("extractObjectStoreOptions - multiple cloud provider configurations") { val hadoopConf = new Configuration() @@ -79,61 +70,4 @@ class NativeConfigSuite extends AnyFunSuite with Matchers with BeforeAndAfterEac new URI("unsupported://test-bucket/test-object")) assert(unsupportedOptions.isEmpty, "Unsupported scheme should return empty options") } - - test("validate object store config - no provider") { - val hadoopConf = new Configuration() - validate(hadoopConf) - } - - test("validate object store config - valid providers") { - val hadoopConf = new Configuration() - val provider1 = "com.amazonaws.auth.EnvironmentVariableCredentialsProvider" - val provider2 = "com.amazonaws.auth.WebIdentityTokenCredentialsProvider" - hadoopConf.set("fs.s3a.aws.credentials.provider", Seq(provider1, provider2).mkString(",")) - validate(hadoopConf) - } - - test("validate object store config - invalid provider") { - val hadoopConf = new Configuration() - hadoopConf.set("fs.s3a.aws.credentials.provider", "invalid") - val fallbackReasons = validate(hadoopConf) - val expectedError = "Unsupported credential provider: invalid" - assert(fallbackReasons.exists(_.contains(expectedError))) - } - - test("validate object store config - mixed anonymous providers") { - val hadoopConf = new Configuration() - val provider1 = "com.amazonaws.auth.AnonymousAWSCredentials" - val provider2 = "software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider" - hadoopConf.set("fs.s3a.aws.credentials.provider", Seq(provider1, provider2).mkString(",")) - val fallbackReasons = validate(hadoopConf) - val expectedError = - "Anonymous credential provider cannot be mixed with other credential providers" - assert(fallbackReasons.exists(_.contains(expectedError))) - } - - test("validity cache") { - val hadoopConf = new Configuration() - val provider1 = "com.amazonaws.auth.AnonymousAWSCredentials" - val provider2 = "software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider" - hadoopConf.set("fs.s3a.aws.credentials.provider", Seq(provider1, provider2).mkString(",")) - - assert(CometScanRule.configValidityMap.isEmpty) - for (_ <- 0 until 5) { - assert(validate(hadoopConf).nonEmpty) - assert(CometScanRule.configValidityMap.size == 1) - } - - // set the same providers but in a different order - hadoopConf.set("fs.s3a.aws.credentials.provider", Seq(provider2, provider1).mkString(",")) - assert(validate(hadoopConf).nonEmpty) - assert(CometScanRule.configValidityMap.size == 2) - } - - private def validate(hadoopConf: Configuration): Set[String] = { - val path = "s3a://path/to/file.parquet" - val fallbackReasons = mutable.ListBuffer[String]() - CometScanRule.validateObjectStoreConfig(path, hadoopConf, fallbackReasons) - fallbackReasons.toSet - } } diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 815f03f213..f6795b91a3 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -73,21 +73,19 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_OPERATOR_DATA_WRITING_COMMAND_ALLOW_INCOMPAT.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true") { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> "native_datafusion") { - val capturedPlan = writeWithCometNativeWriteExec(inputPath, outputPath) - capturedPlan.foreach { plan => - val hasNativeScan = plan.exists { - case _: CometNativeScanExec => true - case _ => false - } - - assert( - hasNativeScan, - s"Expected CometNativeScanExec in the plan, but got:\n${plan.treeString}") + val capturedPlan = writeWithCometNativeWriteExec(inputPath, outputPath) + capturedPlan.foreach { plan => + val hasNativeScan = plan.exists { + case _: CometNativeScanExec => true + case _ => false } - verifyWrittenFile(outputPath) + assert( + hasNativeScan, + s"Expected CometNativeScanExec in the plan, but got:\n${plan.treeString}") } + + verifyWrittenFile(outputPath) } } } @@ -470,8 +468,6 @@ class CometParquetWriterSuite extends CometTestBase { // enable experimental native writes CometConf.COMET_OPERATOR_DATA_WRITING_COMMAND_ALLOW_INCOMPAT.key -> "true", CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", - // explicitly set scan impl to override CI defaults - CometConf.COMET_NATIVE_SCAN_IMPL.key -> "auto", // Disable unsigned small int safety check for ShortType columns CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "false", // use a different timezone to make sure that timezone handling works with nested types @@ -535,10 +531,7 @@ class CometParquetWriterSuite extends CometTestBase { private def readCometRows(path: String): Array[Row] = { var rows: Array[Row] = null - withSQLConf( - CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "true", - // Override CI setting to use a scan impl that supports complex types - CometConf.COMET_NATIVE_SCAN_IMPL.key -> "auto") { + withSQLConf(CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "true") { val df = spark.read.parquet(path) val plan = df.queryExecution.executedPlan assert( diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala index b8db737a3c..851a5c9fa6 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadFromFakeHadoopFsSuite.scala @@ -76,16 +76,14 @@ class ParquetReadFromFakeHadoopFsSuite extends CometTestBase with AdaptiveSparkP // This test fails for 'hdfs' but succeeds for 'open-dal'. 'hdfs' requires this fix // https://github.com/datafusion-contrib/fs-hdfs/pull/29 - test("test native_datafusion scan on fake fs") { + test("native scan on fake fs") { // Skip test if HDFS feature is not enabled in native library assume(isFeatureEnabled("hdfs-opendal")) val testFilePath = s"${FakeHDFSFileSystem.PREFIX}${fake_root_dir.getAbsolutePath}/data/test-file.parquet" writeTestParquetFile(testFilePath) - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { - val df = spark.read.format("parquet").load(testFilePath).agg(sum(col("id"))) - assertCometNativeScanOnFakeFs(df) - assert(df.first().getLong(0) == 499500) - } + val df = spark.read.format("parquet").load(testFilePath).agg(sum(col("id"))) + assertCometNativeScanOnFakeFs(df) + assert(df.first().getLong(0) == 499500) } } diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala index 3a634bb2b0..a88613a9d0 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -25,7 +25,6 @@ import java.time.{ZoneId, ZoneOffset} import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import scala.util.control.Breaks.breakable import org.scalactic.source.Position import org.scalatest.Tag @@ -699,7 +698,7 @@ abstract class ParquetReadSuite extends CometTestBase { checkSparkAnswer(df) // Missing optional struct field with nested required field - // TODO: This produces incorrect results in both native_datafusion and native_iceberg_compat + // TODO: This produces incorrect results in native_datafusion // df = sql("select a, c.c1 from complex_types") // checkSparkAnswer(df) @@ -712,7 +711,7 @@ abstract class ParquetReadSuite extends CometTestBase { } } - // TODO: This test fails for both native_datafusion and native_iceberg_compat + // TODO: This test fails for native_datafusion ignore(" Missing optional struct field with nested required field") { Seq(true, false).foreach { dictionaryEnabled => def makeRawParquetFile(path: Path): Unit = { @@ -761,7 +760,7 @@ abstract class ParquetReadSuite extends CometTestBase { withParquetTable(spark.read.format("parquet").load(path.toString), "complex_types") { // Missing optional struct field with nested required field - // TODO: This produces incorrect results in both native_datafusion and native_iceberg_compat + // TODO: This produces incorrect results in native_datafusion val df = sql("select a, c.c1 from complex_types") checkSparkAnswer(df) @@ -967,32 +966,21 @@ abstract class ParquetReadSuite extends CometTestBase { } test("schema evolution") { - Seq(true, false).foreach { enableSchemaEvolution => - Seq(true, false).foreach { useDictionary => - { - withSQLConf( - CometConf.COMET_SCHEMA_EVOLUTION_ENABLED.key -> enableSchemaEvolution.toString) { - val data = (0 until 100).map(i => { - val v = if (useDictionary) i % 5 else i - (v, v.toFloat) - }) - val readSchema = - StructType( - Seq(StructField("_1", LongType, false), StructField("_2", DoubleType, false))) - - withParquetDataFrame(data, schema = Some(readSchema)) { df => - val scan = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) - val isNativeDataFusionScan = - scan == CometConf.SCAN_NATIVE_DATAFUSION || scan == CometConf.SCAN_AUTO - if (enableSchemaEvolution || isNativeDataFusionScan) { - // native_datafusion has more permissive schema evolution - // https://github.com/apache/datafusion-comet/issues/3720 - checkAnswer(df, data.map(Row.fromTuple)) - } else { - assertThrows[SparkException](df.collect()) - } - } - } + // Comet's widening behavior tracks the Spark version (see ShimCometConf): + // 3.x rejects INT32 -> LONG and FLOAT -> DOUBLE on read, 4.x accepts. + Seq(true, false).foreach { useDictionary => + val data = (0 until 100).map(i => { + val v = if (useDictionary) i % 5 else i + (v, v.toFloat) + }) + val readSchema = + StructType(Seq(StructField("_1", LongType, false), StructField("_2", DoubleType, false))) + + withParquetDataFrame(data, schema = Some(readSchema)) { df => + if (CometConf.COMET_SCHEMA_EVOLUTION_ENABLED) { + checkAnswer(df, data.map(Row.fromTuple)) + } else { + assertThrows[SparkException](df.collect()) } } } @@ -1005,9 +993,7 @@ abstract class ParquetReadSuite extends CometTestBase { // TypeUtil.checkParquetType, BINARY case). The native_datafusion scan // must do the same in its schema adapter rather than letting DataFusion's // cast silently parse the bytes or reinterpret them. - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { withTempPath { dir => val path = dir.getCanonicalPath Seq("a", "b", "c").toDF("c").write.parquet(path) @@ -1022,70 +1008,156 @@ abstract class ParquetReadSuite extends CometTestBase { } } + test("native_datafusion rejects BINARY (no decimal annotation) read as DecimalType") { + // Regression guard for https://github.com/apache/datafusion-comet/issues/4351, + // mirroring the BINARY -> DECIMAL(37, 1) iteration in SPARK-34212. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTempPath { dir => + val path = dir.getCanonicalPath + // CAST('1.2' AS BINARY) writes BYTE_ARRAY with no decimal annotation. + sql("SELECT CAST('1.2' AS BINARY) c").write.parquet(path) + Seq("DECIMAL(3, 2)", "DECIMAL(18, 1)", "DECIMAL(37, 1)").foreach { schema => + val outer = intercept[SparkException] { + spark.read.schema(s"c $schema").parquet(path).collect() + } + // Walk the cause chain: Comet's shim adds an extra SparkException + // wrap on Spark 3.x compared to vanilla Spark. + val chain = Iterator + .iterate[Throwable](outer)(_.getCause) + .takeWhile(_ != null) + .toSeq + assert( + chain.exists(_.isInstanceOf[ + org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException]), + s"expected SchemaColumnConvertNotSupportedException for $schema; chain was:\n" + + chain.map(t => s" ${t.getClass.getName}: ${t.getMessage}").mkString("\n")) + } + } + } + } + test("native_datafusion rejects incompatible decimal precision/scale") { - // Regression guard for https://github.com/apache/datafusion-comet/issues/4089. - // Reading Decimal(10,2) under a Decimal(5,0) read schema is unconditionally - // lossy: target precision is smaller than source precision and scales differ. - // Spark's vectorized reader throws SchemaColumnConvertNotSupportedException - // here on all versions. The native_datafusion scan must reject this in its - // schema adapter rather than letting Spark Cast silently rescale/truncate. - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + // Regression guard for #4089 and #4343. Spark's `isDecimalTypeMatched` + // accepts decimal-to-decimal only when `scaleIncrease >= 0` AND + // `precisionIncrease >= scaleIncrease`. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + val cases = Seq( + // (file_p, file_s, read_p, read_s) + (10, 2, 5, 0), // #4089: scale narrows. + (10, 2, 5, 2), // precision-only narrowing. + (10, 4, 5, 2), // integer-precision narrowing (int-prec 6 -> 3). + (5, 2, 5, 3) + ) // scale widening overflows the integer side. + cases.foreach { case (filePrec, fileScale, readPrec, readScale) => + withTempPath { dir => + val path = dir.getCanonicalPath + spark + .sql(s"select cast('1.23' as decimal($filePrec,$fileScale)) as d " + + s"union all select cast('4.56' as decimal($filePrec,$fileScale))") + .write + .parquet(path) + val df = spark.read.schema(s"d decimal($readPrec,$readScale)").parquet(path) + assertThrows[SparkException](df.collect()) + } + } + } + } + + test("native_datafusion rejects integer read as too-narrow decimal") { + // Regression guard for #4344. Spark's `canReadAsDecimal` requires + // `precision - scale >= 10` for INT32 sources and `>= 20` for INT64. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + // INT32 source (Byte/Short/Int all written as INT32 by Spark). + Seq("byte", "short", "int").foreach { writeType => + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(1, 4).selectExpr(s"cast(id as $writeType) as c").write.parquet(path) + val df = spark.read.schema("c decimal(9, 0)").parquet(path) + assertThrows[SparkException](df.collect()) + } + } + // INT64 source. withTempPath { dir => val path = dir.getCanonicalPath - spark - .sql("select cast('123.45' as decimal(10,2)) as d " + - "union all select cast('67.89' as decimal(10,2))") - .write - .parquet(path) - val df = spark.read.schema("d decimal(5,0)").parquet(path) + spark.range(1, 4).selectExpr("cast(id as long) as c").write.parquet(path) + val df = spark.read.schema("c decimal(19, 0)").parquet(path) assertThrows[SparkException](df.collect()) } } } + test("native_datafusion rejects primitive Parquet conversions Spark rejects") { + // Regression guard for #4297. `getUpdater` has no branch for these + // (write_type, read_type) pairs. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + val cases = Seq( + ("bigint", "8589934592", "int"), + ("double", "1e40", "float"), + ("float", "1.5", "bigint"), + ("bigint", "1", "double"), + ("int", "33554433", "float"), + ("int", "1", "timestamp"), + ("int", "1", "timestamp_ntz"), + ("bigint", "1", "date"), + ("double", "1.0", "bigint"), + ("date", "DATE'2020-01-01'", "timestamp"), + ("timestamp", "TIMESTAMP'2020-01-01 00:00:00'", "date"), + ("timestamp_ntz", "TIMESTAMP_NTZ'2020-01-01 00:00:00'", "date")) + cases.foreach { case (writeType, sourceLiteral, readType) => + withTempPath { dir => + val path = dir.getCanonicalPath + spark.sql(s"select cast($sourceLiteral as $writeType) as c").write.parquet(path) + val df = spark.read.schema(s"c $readType").parquet(path) + withClue(s"$writeType -> $readType: ") { + assertThrows[SparkException](df.collect()) + } + } + } + } + } + test("type widening: byte → short/int/long, short → int/long, int → long") { - withSQLConf(CometConf.COMET_SCHEMA_EVOLUTION_ENABLED.key -> "true") { - withTempPath { dir => - val path = dir.getCanonicalPath - val values = 1 to 10 - val options: Map[String, String] = Map.empty[String, String] - - // Input types and corresponding DataFrames - val inputDFs = Seq( - "byte" -> values.map(_.toByte).toDF("col1"), - "short" -> values.map(_.toShort).toDF("col1"), - "int" -> values.map(_.toInt).toDF("col1")) - - // Target Spark read schemas for widening - val widenTargets = Seq( - "short" -> values.map(_.toShort).toDF("col1"), - "int" -> values.map(_.toInt).toDF("col1"), - "long" -> values.map(_.toLong).toDF("col1")) - - for ((inputType, inputDF) <- inputDFs) { - val writePath = s"$path/$inputType" - inputDF.write.format("parquet").options(options).save(writePath) - - for ((targetType, targetDF) <- widenTargets) { - // Only test valid widenings (e.g., don't test int → short) - val wideningValid = (inputType, targetType) match { - case ("byte", "short" | "int" | "long") => true - case ("short", "int" | "long") => true - case ("int", "long") => true - case _ => false - } + // Widening of INT32 -> LONG is only allowed when Comet's type-promotion + // default permits it (Spark 4.x). See ShimCometConf. + assume(CometConf.COMET_SCHEMA_EVOLUTION_ENABLED) + withTempPath { dir => + val path = dir.getCanonicalPath + val values = 1 to 10 + val options: Map[String, String] = Map.empty[String, String] + + // Input types and corresponding DataFrames + val inputDFs = Seq( + "byte" -> values.map(_.toByte).toDF("col1"), + "short" -> values.map(_.toShort).toDF("col1"), + "int" -> values.map(_.toInt).toDF("col1")) + + // Target Spark read schemas for widening + val widenTargets = Seq( + "short" -> values.map(_.toShort).toDF("col1"), + "int" -> values.map(_.toInt).toDF("col1"), + "long" -> values.map(_.toLong).toDF("col1")) + + for ((inputType, inputDF) <- inputDFs) { + val writePath = s"$path/$inputType" + inputDF.write.format("parquet").options(options).save(writePath) + + for ((targetType, targetDF) <- widenTargets) { + // Only test valid widenings (e.g., don't test int → short) + val wideningValid = (inputType, targetType) match { + case ("byte", "short" | "int" | "long") => true + case ("short", "int" | "long") => true + case ("int", "long") => true + case _ => false + } - if (wideningValid) { - val reader = spark.read - .schema(s"col1 $targetType") - .format("parquet") - .options(options) - .load(writePath) + if (wideningValid) { + val reader = spark.read + .schema(s"col1 $targetType") + .format("parquet") + .options(options) + .load(writePath) - checkAnswer(reader, targetDF) - } + checkAnswer(reader, targetDF) } } } @@ -1093,37 +1165,38 @@ abstract class ParquetReadSuite extends CometTestBase { } test("read byte, int, short, long together") { - withSQLConf(CometConf.COMET_SCHEMA_EVOLUTION_ENABLED.key -> "true") { - withTempPath { dir => - val path = dir.getCanonicalPath + // Reading INT32-encoded files under a LONG schema only succeeds when Comet's + // type-promotion default permits it (Spark 4.x). See ShimCometConf. + assume(CometConf.COMET_SCHEMA_EVOLUTION_ENABLED) + withTempPath { dir => + val path = dir.getCanonicalPath - val byteDF = (Byte.MaxValue - 2 to Byte.MaxValue).map(_.toByte).toDF("col1") - val shortDF = (Short.MaxValue - 2 to Short.MaxValue).map(_.toShort).toDF("col1") - val intDF = (Int.MaxValue - 2 to Int.MaxValue).toDF("col1") - val longDF = (Long.MaxValue - 2 to Long.MaxValue).toDF("col1") - val unionDF = byteDF.union(shortDF).union(intDF).union(longDF) + val byteDF = (Byte.MaxValue - 2 to Byte.MaxValue).map(_.toByte).toDF("col1") + val shortDF = (Short.MaxValue - 2 to Short.MaxValue).map(_.toShort).toDF("col1") + val intDF = (Int.MaxValue - 2 to Int.MaxValue).toDF("col1") + val longDF = (Long.MaxValue - 2 to Long.MaxValue).toDF("col1") + val unionDF = byteDF.union(shortDF).union(intDF).union(longDF) - val byteDir = s"$path${File.separator}part=byte" - val shortDir = s"$path${File.separator}part=short" - val intDir = s"$path${File.separator}part=int" - val longDir = s"$path${File.separator}part=long" + val byteDir = s"$path${File.separator}part=byte" + val shortDir = s"$path${File.separator}part=short" + val intDir = s"$path${File.separator}part=int" + val longDir = s"$path${File.separator}part=long" - val options: Map[String, String] = Map.empty[String, String] + val options: Map[String, String] = Map.empty[String, String] - byteDF.write.format("parquet").options(options).save(byteDir) - shortDF.write.format("parquet").options(options).save(shortDir) - intDF.write.format("parquet").options(options).save(intDir) - longDF.write.format("parquet").options(options).save(longDir) + byteDF.write.format("parquet").options(options).save(byteDir) + shortDF.write.format("parquet").options(options).save(shortDir) + intDF.write.format("parquet").options(options).save(intDir) + longDF.write.format("parquet").options(options).save(longDir) - val df = spark.read - .schema(unionDF.schema) - .format("parquet") - .options(options) - .load(path) - .select("col1") + val df = spark.read + .schema(unionDF.schema) + .format("parquet") + .options(options) + .load(path) + .select("col1") - checkAnswer(df, unionDF) - } + checkAnswer(df, unionDF) } } @@ -1327,15 +1400,10 @@ abstract class ParquetReadSuite extends CometTestBase { } } - def testScanner( - cometEnabled: String, - cometNativeScanImpl: String, - scanner: String, - v1: Option[String] = None): Unit = { + def testScanner(cometEnabled: String, scanner: String, v1: Option[String] = None): Unit = { withSQLConf( CometConf.COMET_ENABLED.key -> cometEnabled, CometConf.COMET_EXEC_ENABLED.key -> cometEnabled, - CometConf.COMET_NATIVE_SCAN_IMPL.key -> cometNativeScanImpl, SQLConf.USE_V1_SOURCE_LIST.key -> v1.getOrElse("")) { withParquetTable(Seq((Long.MaxValue, 1), (Long.MaxValue, 2)), "tbl") { val df = spark.sql("select * from tbl") @@ -1421,10 +1489,9 @@ abstract class ParquetReadSuite extends CometTestBase { } // Based on Spark ParquetFieldIdIOSuite.test("Parquet reads infer fields using field ids - // correctly"). Forces SCAN_NATIVE_DATAFUSION so we can prove that the gate in CometScanRule - // is removed and that the native_datafusion scan resolves columns by field id rather than by - // name (the read schema names differ from what is in the file). - test("native_datafusion: read by Parquet field id when names differ") { + // correctly"). Verifies that Comet's native Parquet scan resolves columns by field id + // rather than by name (the read schema names differ from what is in the file). + test("read by Parquet field id when names differ") { val writeSchema = StructType( Seq( StructField("random", IntegerType, nullable = true, withId(1)), @@ -1435,9 +1502,7 @@ abstract class ParquetReadSuite extends CometTestBase { StructField("b", IntegerType, nullable = true, withId(1)))) val writeData = Seq(Row(100, "text"), Row(200, "more")) - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { + withSQLConf(SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { withTempPath { dir => spark .createDataFrame(spark.sparkContext.parallelize(writeData), writeSchema) @@ -1451,9 +1516,9 @@ abstract class ParquetReadSuite extends CometTestBase { } // Based on Spark ParquetFieldIdIOSuite.test("SPARK-38094: absence of field ids: reading nested - // schema"). Exercises ID matching at every nesting level (struct, array, map) under - // SCAN_NATIVE_DATAFUSION. Names differ from the file at every level. - test("native_datafusion: read nested types by Parquet field id when names differ") { + // schema"). Exercises ID matching at every nesting level (struct, array, map). Names + // differ from the file at every level. + test("read nested types by Parquet field id when names differ") { val writeSchema = StructType( Seq(StructField( "outer", @@ -1498,9 +1563,7 @@ abstract class ParquetReadSuite extends CometTestBase { Row(Row(1, Seq(Row("x", 10), Row("y", 20)), Map("k1" -> 100))), Row(Row(2, Seq(Row("z", 30)), Map("k2" -> 200, "k3" -> 300)))) - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { + withSQLConf(SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { withTempPath { dir => spark .createDataFrame(spark.sparkContext.parallelize(data), writeSchema) @@ -1513,14 +1576,12 @@ abstract class ParquetReadSuite extends CometTestBase { } } - // Verbatim port of Spark `ParquetFieldIdIOSuite.test("multiple id matches")`, pinned to - // `SCAN_NATIVE_DATAFUSION` so the shim error path is exercised on both 3.x and 4.x. - // The stock suite is the CI signal but it requires the Spark test jars and - // `withAllParquetReaders`; keeping a copy here lets us iterate locally. + // Verbatim port of Spark `ParquetFieldIdIOSuite.test("multiple id matches")` so the shim + // error path is exercised on both 3.x and 4.x. The stock suite is the CI signal but it + // requires the Spark test jars and `withAllParquetReaders`; keeping a copy here lets us + // iterate locally. test("multiple id matches") { - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { + withSQLConf(SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { withTempPath { dir => val readSchema = new StructType() @@ -1550,11 +1611,9 @@ abstract class ParquetReadSuite extends CometTestBase { } // Verbatim port of Spark `ParquetFieldIdIOSuite.test("read parquet file without ids")`, - // pinned to `SCAN_NATIVE_DATAFUSION` for the same reason as the duplicate-id test above. + // for the same reason as the duplicate-id test above. test("read parquet file without ids") { - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { + withSQLConf(SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key -> "true") { withTempPath { dir => val readSchema = new StructType() @@ -1619,45 +1678,31 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { } test("Test V1 parquet scan uses respective scanner") { - Seq( - ("false", CometConf.SCAN_NATIVE_DATAFUSION, "FileScan parquet"), - ("true", CometConf.SCAN_NATIVE_DATAFUSION, "CometNativeScan"), - ("true", CometConf.SCAN_NATIVE_ICEBERG_COMPAT, "CometScan [native_iceberg_compat] parquet")) - .foreach { case (cometEnabled, cometNativeScanImpl, expectedScanner) => - testScanner( - cometEnabled, - cometNativeScanImpl, - scanner = expectedScanner, - v1 = Some("parquet")) - } + Seq(("false", "FileScan parquet"), ("true", "CometNativeScan")).foreach { + case (cometEnabled, expectedScanner) => + testScanner(cometEnabled, scanner = expectedScanner, v1 = Some("parquet")) + } } test("test V1 parquet native scan -- case insensitive") { withTempPath { path => spark.range(10).toDF("a").write.parquet(path.toString) - Seq(CometConf.SCAN_NATIVE_DATAFUSION, CometConf.SCAN_NATIVE_ICEBERG_COMPAT).foreach( - scanMode => { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanMode) { - withTable("test") { - sql("create table test (A long) using parquet options (path '" + path + "')") - val df = sql("select A from test") - checkSparkAnswer(df) - // TODO: pushed down filters do not used schema adapter in datafusion, will cause empty result - // val df = sql("select * from test where A > 5") - // checkSparkAnswer(df) - } - } - }) + withTable("test") { + sql("create table test (A long) using parquet options (path '" + path + "')") + val df = sql("select A from test") + checkSparkAnswer(df) + // TODO: pushed down filters do not used schema adapter in datafusion, will cause empty result + // val df = sql("select * from test where A > 5") + // checkSparkAnswer(df) + } } } - test("test V1 parquet scan filter pushdown of primitive types uses native_iceberg_compat") { + test("test V1 parquet scan filter pushdown of primitive types") { withTempPath { dir => val path = new Path(dir.toURI.toString, "test1.parquet") val rows = 1000 - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT, - CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "true") { + withSQLConf(CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "true") { makeParquetFileAllPrimitiveTypes( path, dictionaryEnabled = false, @@ -1665,104 +1710,95 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { rows, nullEnabled = false) } - Seq(CometConf.SCAN_NATIVE_DATAFUSION, CometConf.SCAN_NATIVE_ICEBERG_COMPAT).foreach { - scanMode => - Seq(true, false).foreach { pushDown => - breakable { - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanMode, - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushDown.toString) { - Seq( - ("_1 = true", Math.ceil(rows.toDouble / 2)), // Boolean - ("_2 = 1", Math.ceil(rows.toDouble / 256)), // Byte - ("_3 = 1", 1), // Short - ("_4 = 1", 1), // Integer - ("_5 = 1", 1), // Long - ("_6 = 1.0", 1), // Float - ("_7 = 1.0", 1), // Double - (s"_8 = '${1.toString * 48}'", 1), // String - ("_21 = to_binary('1', 'utf-8')", 1), // binary - ("_15 = 0.0", 1), // DECIMAL(5, 2) - ("_16 = 0.0", 1), // DECIMAL(18, 10) - ( - s"_17 = ${new BigDecimal(new BigInteger(("1" * 16).getBytes), 37).toString}", - Math.ceil(rows.toDouble / 10) - ), // DECIMAL(38, 37) - (s"_19 = TIMESTAMP '${DateTimeUtils.toJavaTimestamp(1)}'", 1), // Timestamp - ("_20 = DATE '1970-01-02'", 1) // Date - ).foreach { case (whereCause, expectedRows) => - val df = spark.read - .parquet(path.toString) - .where(whereCause) - val (_, cometPlan) = checkSparkAnswer(df) - val scan = collect(cometPlan) { - case p: CometScanExec => - assert(p.dataFilters.nonEmpty) - p - case p: CometNativeScanExec => - assert(p.dataFilters.nonEmpty) - p - } - assert(scan.size == 1) + Seq(true, false).foreach { pushDown => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushDown.toString) { + Seq( + ("_1 = true", Math.ceil(rows.toDouble / 2)), // Boolean + ("_2 = 1", Math.ceil(rows.toDouble / 256)), // Byte + ("_3 = 1", 1), // Short + ("_4 = 1", 1), // Integer + ("_5 = 1", 1), // Long + ("_6 = 1.0", 1), // Float + ("_7 = 1.0", 1), // Double + (s"_8 = '${1.toString * 48}'", 1), // String + ("_21 = to_binary('1', 'utf-8')", 1), // binary + ("_15 = 0.0", 1), // DECIMAL(5, 2) + ("_16 = 0.0", 1), // DECIMAL(18, 10) + ( + s"_17 = ${new BigDecimal(new BigInteger(("1" * 16).getBytes), 37).toString}", + Math.ceil(rows.toDouble / 10) + ), // DECIMAL(38, 37) + (s"_19 = TIMESTAMP '${DateTimeUtils.toJavaTimestamp(1)}'", 1), // Timestamp + ("_20 = DATE '1970-01-02'", 1) // Date + ).foreach { case (whereCause, expectedRows) => + val df = spark.read + .parquet(path.toString) + .where(whereCause) + val (_, cometPlan) = checkSparkAnswer(df) + val scan = collect(cometPlan) { + case p: CometScanExec => + assert(p.dataFilters.nonEmpty) + p + case p: CometNativeScanExec => + assert(p.dataFilters.nonEmpty) + p + } + assert(scan.size == 1) - if (pushDown) { - assert(scan.head.metrics("output_rows").value == expectedRows) - } else { - assert(scan.head.metrics("output_rows").value == rows) - } - } - } + if (pushDown) { + assert(scan.head.metrics("output_rows").value == expectedRows) + } else { + assert(scan.head.metrics("output_rows").value == rows) } } + } } } } test("read basic complex types") { - Seq(true, false).foreach(dictionaryEnabled => { - withTempPath { dir => - val path = new Path(dir.toURI.toString, "complex_types.parquet") - makeParquetFileComplexTypes(path, dictionaryEnabled, 10) - withParquetTable(path.toUri.toString, "complex_types") { - Seq(CometConf.SCAN_NATIVE_DATAFUSION, CometConf.SCAN_NATIVE_ICEBERG_COMPAT).foreach( - scanMode => { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanMode) { - checkSparkAnswerAndOperator(sql("select * from complex_types")) - // First level - checkSparkAnswerAndOperator(sql( - "select optional_array, array_of_struct, optional_map, complex_map from complex_types")) - // second nested level - checkSparkAnswerAndOperator( - sql( - "select optional_array[0], " + - "array_of_struct[0].field1, " + - "array_of_struct[0].optional_nested_array, " + - "optional_map.key, " + - "optional_map.value, " + - "map_keys(complex_map), " + - "map_entries(complex_map), " + - "map_values(complex_map) " + - "from complex_types")) - // leaf fields - checkSparkAnswerAndOperator( - sql( - "select optional_array[0], " + - "array_of_struct[0].field1, " + - "array_of_struct[0].optional_nested_array[0], " + - "optional_map.key, " + - "optional_map.value, " + - "map_keys(complex_map)[0].key_field1, " + - "map_keys(complex_map)[0].key_field2, " + - "map_entries(complex_map)[0].key, " + - "map_entries(complex_map)[0].value, " + - "map_values(complex_map)[0].value_field1, " + - "map_values(complex_map)[0].value_field2 " + - "from complex_types")) - } - }) + // Array indexing in the queries below would throw INVALID_ARRAY_INDEX under ANSI. + withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") { + Seq(true, false).foreach(dictionaryEnabled => { + withTempPath { dir => + val path = new Path(dir.toURI.toString, "complex_types.parquet") + makeParquetFileComplexTypes(path, dictionaryEnabled, 10) + withParquetTable(path.toUri.toString, "complex_types") { + checkSparkAnswerAndOperator(sql("select * from complex_types")) + // First level + checkSparkAnswerAndOperator(sql( + "select optional_array, array_of_struct, optional_map, complex_map from complex_types")) + // second nested level + checkSparkAnswerAndOperator( + sql( + "select optional_array[0], " + + "array_of_struct[0].field1, " + + "array_of_struct[0].optional_nested_array, " + + "optional_map.key, " + + "optional_map.value, " + + "map_keys(complex_map), " + + "map_entries(complex_map), " + + "map_values(complex_map) " + + "from complex_types")) + // leaf fields + checkSparkAnswerAndOperator( + sql( + "select optional_array[0], " + + "array_of_struct[0].field1, " + + "array_of_struct[0].optional_nested_array[0], " + + "optional_map.key, " + + "optional_map.value, " + + "map_keys(complex_map)[0].key_field1, " + + "map_keys(complex_map)[0].key_field2, " + + "map_entries(complex_map)[0].key, " + + "map_entries(complex_map)[0].value, " + + "map_values(complex_map)[0].value_field1, " + + "map_values(complex_map)[0].value_field2 " + + "from complex_types")) + } } - } - }) + }) + } } test("reading ancient dates before 1582") { @@ -1772,25 +1808,18 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { val file = getResourceParquetFilePath("test-data/before_1582_date_v3_2_0.snappy.parquet") - Seq(CometConf.SCAN_NATIVE_ICEBERG_COMPAT, CometConf.SCAN_NATIVE_DATAFUSION).foreach { - scanImpl => - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl) { - val df = spark.read.parquet(file) - - // Verify Comet scan is in the plan - val plan = df.queryExecution.executedPlan - checkCometOperators(plan) - - // Verify all 8 rows are read and contain dates before 1582 - val rows = df.collect() - assert(rows.length == 8, s"Expected 8 rows with $scanImpl, got ${rows.length}") - rows.foreach { row => - val date = row.getDate(0) - assert( - date.toLocalDate.getYear < 1582, - s"Expected date before 1582 with $scanImpl, got $date") - } - } + val df = spark.read.parquet(file) + + // Verify Comet scan is in the plan + val plan = df.queryExecution.executedPlan + checkCometOperators(plan) + + // Verify all 8 rows are read and contain dates before 1582 + val rows = df.collect() + assert(rows.length == 8, s"Expected 8 rows, got ${rows.length}") + rows.foreach { row => + val date = row.getDate(0) + assert(date.toLocalDate.getYear < 1582, s"Expected date before 1582, got $date") } } diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetTimestampLtzAsNtzSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetTimestampLtzAsNtzSuite.scala index cd7d2a6271..595059a3bc 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetTimestampLtzAsNtzSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetTimestampLtzAsNtzSuite.scala @@ -21,11 +21,9 @@ package org.apache.comet.parquet import java.sql.Timestamp -import org.apache.spark.SparkException import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.internal.SQLConf -import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus /** @@ -42,42 +40,6 @@ class ParquetTimestampLtzAsNtzSuite extends CometTestBase { private val tsTypes = Seq("INT96", "TIMESTAMP_MICROS", "TIMESTAMP_MILLIS") - tsTypes.foreach { tsType => - test(s"read TimestampLTZ ($tsType) as TimestampNTZ throws pre-Spark 4") { - assume(!isSpark40Plus, "Spark 4.0+ allows reading TimestampLTZ as TimestampNTZ") - - val scanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get() - assume( - scanImpl != CometConf.SCAN_AUTO && scanImpl != CometConf.SCAN_NATIVE_DATAFUSION, - s"https://github.com/apache/datafusion-comet/issues/4219 ($scanImpl scan does not " + - "reject TimestampLTZ read as TimestampNTZ)") - - val sessionTz = "America/Los_Angeles" - - withSQLConf( - SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionTz, - SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> tsType, - SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { - withTempPath { dir => - val path = dir.getCanonicalPath - Seq(Timestamp.valueOf("2020-01-01 12:00:00")).toDF("ts").write.parquet(path) - - // Spark refuses to read TimestampLTZ as TimestampNTZ (SPARK-36182) - withSQLConf(CometConf.COMET_ENABLED.key -> "false") { - intercept[SparkException] { - spark.read.schema("ts timestamp_ntz").parquet(path).collect() - } - } - - // Comet should also refuse - intercept[SparkException] { - spark.read.schema("ts timestamp_ntz").parquet(path).collect() - } - } - } - } - } - tsTypes.foreach { tsType => test(s"read TimestampLTZ ($tsType) as TimestampNTZ matches Spark") { assume(isSpark40Plus, "Spark 4.0+ allows reading TimestampLTZ as TimestampNTZ") @@ -91,9 +53,7 @@ class ParquetTimestampLtzAsNtzSuite extends CometTestBase { val path = dir.getCanonicalPath Seq(Timestamp.valueOf("2020-01-01 12:00:00")).toDF("ts").write.parquet(path) - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { - checkSparkAnswerAndOperator(spark.read.schema("ts timestamp_ntz").parquet(path)) - } + checkSparkAnswerAndOperator(spark.read.schema("ts timestamp_ntz").parquet(path)) } } } diff --git a/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala index 63714383ae..7f353c36e2 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometExecRuleSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleEx import org.apache.spark.sql.types.{DataTypes, StructField, StructType} import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator} /** @@ -131,9 +132,8 @@ class CometExecRuleSuite extends CometTestBase { } } - // TODO this test exposes the bug described in - // https://github.com/apache/datafusion-comet/issues/1389 - ignore("CometExecRule should not allow Comet partial and Spark final hash aggregate") { + // Regression test for https://github.com/apache/datafusion-comet/issues/1389 + test("CometExecRule should not allow Comet partial and Spark final hash aggregate") { withTempView("test_data") { createTestDataFrame.createOrReplaceTempView("test_data") @@ -149,7 +149,8 @@ class CometExecRuleSuite extends CometTestBase { CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { val transformedPlan = applyCometExecRule(sparkPlan) - // if the final aggregate cannot be converted to Comet, then neither should be + // SUM has incompatible intermediate buffers, so if the final aggregate cannot + // be converted to Comet, neither should be assert( countOperators(transformedPlan, classOf[HashAggregateExec]) == originalHashAggCount) assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 0) @@ -181,6 +182,78 @@ class CometExecRuleSuite extends CometTestBase { } } + test("CometExecRule should allow safe Comet partial and Spark final hash aggregate") { + withTempView("test_data") { + createTestDataFrame.createOrReplaceTempView("test_data") + + // Query uses only safe aggregates (MIN, MAX) with compatible intermediate buffers + val sparkPlan = + createSparkPlan(spark, "SELECT MIN(id), MAX(id) FROM test_data GROUP BY (id % 3)") + + val originalHashAggCount = countOperators(sparkPlan, classOf[HashAggregateExec]) + assert(originalHashAggCount == 2) + + withSQLConf( + CometConf.COMET_ENABLE_FINAL_HASH_AGGREGATE.key -> "false", + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + // Safe aggregates allow mixed execution: partial can be Comet, final stays Spark + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == 1) // final only + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 1) // partial + } + } + } + + test("CometExecRule should allow safe Spark partial and Comet final hash aggregate") { + withTempView("test_data") { + createTestDataFrame.createOrReplaceTempView("test_data") + + // Query uses only safe aggregates (MIN, MAX) with compatible intermediate buffers + val sparkPlan = + createSparkPlan(spark, "SELECT MIN(id), MAX(id) FROM test_data GROUP BY (id % 3)") + + val originalHashAggCount = countOperators(sparkPlan, classOf[HashAggregateExec]) + assert(originalHashAggCount == 2) + + withSQLConf( + CometConf.COMET_ENABLE_PARTIAL_HASH_AGGREGATE.key -> "false", + CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + // Safe aggregates allow mixed execution: partial stays Spark, final can be Comet + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == 1) // partial only + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 1) // final + } + } + } + + test("CometExecRule should not convert hash aggregate when grouping key contains map type") { + // Spark 3.4/3.5 reject `array>` as a grouping key in the analyzer (not orderable), + // so the plan never reaches CometExecRule on those versions. The guard we're exercising + // (containsMapType) only matters on Spark 4.0+, which permits the GROUP BY to be analyzed. + assume(isSpark40Plus) + // Arrow's row format, used by DataFusion's grouped hash aggregate for composite keys, does + // not support Map at any nesting level. Grouping by a type that transitively contains a map + // (e.g. array>) must stay on Spark to avoid a native row-encoding crash. + val sparkPlan = createSparkPlan( + spark, + """SELECT count(*) + |FROM VALUES (ARRAY(MAP(1, 2), MAP(1, 3))), + | (ARRAY(MAP(2, 3), MAP(1, 3))) AS t(a) + |GROUP BY a""".stripMargin) + + val originalHashAggCount = countOperators(sparkPlan, classOf[HashAggregateExec]) + assert(originalHashAggCount == 2) + + withSQLConf(CometConf.COMET_EXEC_LOCAL_TABLE_SCAN_ENABLED.key -> "true") { + val transformedPlan = applyCometExecRule(sparkPlan) + + assert(countOperators(transformedPlan, classOf[HashAggregateExec]) == originalHashAggCount) + assert(countOperators(transformedPlan, classOf[CometHashAggregateExec]) == 0) + } + } + test("CometExecRule should apply broadcast exchange transformations") { withTempView("test_data") { createTestDataFrame.createOrReplaceTempView("test_data") diff --git a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala index 6214513f48..f444fe62c9 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala @@ -123,9 +123,7 @@ class CometScanRuleSuite extends CometTestBase { val sparkPlan = createSparkPlan(spark, "SELECT id, value FROM unsupported_data WHERE id = 1") - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT, - CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "true") { + withSQLConf(CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "true") { val transformedPlan = applyCometScanRule(sparkPlan) // Should fallback to Spark due to ShortType (may be from unsigned UINT_8) diff --git a/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala b/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala index 675ff01710..fa5f368e33 100644 --- a/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/CometPluginsSuite.scala @@ -31,6 +31,9 @@ class CometPluginsSuite extends CometTestBase { conf.set("spark.executor.memory", "1G") conf.set("spark.executor.memoryOverhead", "2G") conf.set("spark.plugins", "org.apache.spark.CometPlugin") + conf.set( + "spark.shuffle.manager", + "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") conf.set("spark.comet.enabled", "true") conf.set("spark.comet.exec.enabled", "true") conf.set("spark.comet.exec.onHeap.enabled", "true") diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 767968b7c1..dd4aab0e6b 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -72,7 +72,6 @@ abstract class CometTestBase conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) conf.set("spark.ui.enabled", "false") conf.set(SQLConf.SHUFFLE_PARTITIONS, 10) // reduce parallelism in tests - conf.set(SQLConf.ANSI_ENABLED.key, "false") conf.set(SHUFFLE_MANAGER, shuffleManager) conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") @@ -86,6 +85,7 @@ abstract class CometTestBase conf.set(CometConf.COMET_SPARK_TO_ARROW_ENABLED.key, "true") conf.set(CometConf.COMET_NATIVE_SCAN_ENABLED.key, "true") conf.set(CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key, "false") + conf.set(CometConf.COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER.key, "true") conf.set(CometConf.COMET_ONHEAP_MEMORY_OVERHEAD.key, "2g") conf.set(CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key, "true") // SortOrder is incompatible for mixed zero and negative zero floating point values, but diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArrayExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArrayExpressionBenchmark.scala index 44ef1a4735..624abdebe8 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArrayExpressionBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArrayExpressionBenchmark.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.benchmark -import org.apache.comet.CometConf - // spotless:off /** * Benchmark to measure performance of Comet array expressions. To run this benchmark: @@ -112,14 +110,10 @@ object CometArrayExpressionBenchmark extends CometBenchmarkBase { | cast((value + 5) % 100 as int) as search_val |FROM $tbl""".stripMargin)) - val nativeScanConfig = - Map(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) - runExpressionBenchmark( "array_position - int array", values, - "SELECT array_position(int_arr, search_val) FROM parquetV1Table", - nativeScanConfig) + "SELECT array_position(int_arr, search_val) FROM parquetV1Table") } } @@ -144,14 +138,10 @@ object CometArrayExpressionBenchmark extends CometBenchmarkBase { | cast((value + 5) % 100 as string) as search_val |FROM $tbl""".stripMargin)) - val nativeScanConfig = - Map(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) - runExpressionBenchmark( "array_position - string array", values, - "SELECT array_position(str_arr, search_val) FROM parquetV1Table", - nativeScanConfig) + "SELECT array_position(str_arr, search_val) FROM parquetV1Table") } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala index 5e4ec734a8..9307c785c7 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBenchmarkBase.scala @@ -38,7 +38,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DecimalType import org.apache.comet.CometConf -import org.apache.comet.CometConf.{SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT} import org.apache.comet.CometSparkSessionExtensions trait CometBenchmarkBase @@ -172,15 +171,12 @@ trait CometBenchmarkBase } } - for (scanImpl <- Seq(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT)) { - benchmark.addCase(s"SQL Parquet - Comet ($scanImpl)$suffix") { _ => - withSQLConf( - (extraConf ++ Map( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl)).toSeq: _*) { - spark.sql(query).noop() - } + benchmark.addCase(s"SQL Parquet - Comet$suffix") { _ => + withSQLConf( + (extraConf ++ Map( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true")).toSeq: _*) { + spark.sql(query).noop() } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala index b7a14d408c..224f2ad344 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometDatetimeExpressionBenchmark.scala @@ -103,6 +103,34 @@ object CometDatetimeExpressionBenchmark extends CometBenchmarkBase { } } + def toTimeBenchmark(values: Int): Unit = { + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"select concat(cast(abs(value) % 24 as string), ':', lpad(cast(abs(value) % 60 as string), 2, '0'), ':', lpad(cast(abs(value) % 60 as string), 2, '0')) as s FROM $tbl")) + val name = "to_time" + val query = "select to_time(s) from parquetV1Table" + runExpressionBenchmark(name, values, query) + } + } + } + + def makeTimeBenchmark(values: Int): Unit = { + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable( + dir, + spark.sql( + s"select cast(abs(value) % 24 as int) as h, cast(abs(value) % 60 as int) as m, cast(abs(value) % 60 as decimal(16,6)) as s FROM $tbl")) + val name = "make_time" + val query = "select make_time(h, m, s) from parquetV1Table" + runExpressionBenchmark(name, values, query) + } + } + } + override def runCometBenchmark(mainArgs: Array[String]): Unit = { val values = 1024 * 1024; @@ -130,6 +158,15 @@ object CometDatetimeExpressionBenchmark extends CometBenchmarkBase { } } } + + withSQLConf("spark.sql.timeType.enabled" -> "true") { + runBenchmarkWithTable("ToTime", values) { v => + toTimeBenchmark(v) + } + runBenchmarkWithTable("MakeTime", values) { v => + makeTimeBenchmark(v) + } + } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 608d42bd95..c75669515a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -62,9 +62,6 @@ import org.apache.comet.CometSparkSessionExtensions.{isSpark35Plus, isSpark40Plu * }}} */ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBase { - protected val scanImpls: Seq[String] = - Seq(CometConf.SCAN_NATIVE_ICEBERG_COMPAT, CometConf.SCAN_NATIVE_DATAFUSION) - protected val baseResourcePath: File = { getWorkspaceFilePath("spark", "src", "test", "resources", "tpcds-plan-stability").toFile } @@ -90,13 +87,11 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa private val approvedAnsiPlans: Seq[String] = Seq("q83", "q83.sf100") private def getDirForTest(name: String): File = { - var goldenFileName = if (SQLConf.get.ansiEnabled && approvedAnsiPlans.contains(name)) { + val goldenFileName = if (SQLConf.get.ansiEnabled && approvedAnsiPlans.contains(name)) { name + ".ansi" } else { name } - val nativeImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get() - goldenFileName = s"$goldenFileName.$nativeImpl" val primary = new File(goldenFilePath, goldenFileName) if (regenerateGoldenFiles || primary.isDirectory) { primary @@ -353,13 +348,9 @@ class CometTPCDSV1_4_PlanStabilitySuite extends CometPlanStabilitySuite { override protected val fallbackGoldenFilePaths: Seq[String] = planNames.tail.map(new File(baseResourcePath, _).getAbsolutePath) - scanImpls.foreach { scan => - tpcdsQueries.foreach { q => - test(s"check simplified (tpcds-v1.4/$q) - $scan") { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> scan) { - testQuery("tpcds", q) - } - } + tpcdsQueries.foreach { q => + test(s"check simplified (tpcds-v1.4/$q)") { + testQuery("tpcds", q) } } } @@ -371,13 +362,9 @@ class CometTPCDSV2_7_PlanStabilitySuite extends CometPlanStabilitySuite { override protected val fallbackGoldenFilePaths: Seq[String] = planNames.tail.map(new File(baseResourcePath, _).getAbsolutePath) - scanImpls.foreach { scan => - tpcdsQueriesV2_7_0.foreach { q => - test(s"check simplified (tpcds-v2.7.0/$q) - $scan") { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> scan) { - testQuery("tpcds-v2.7.0", q) - } - } + tpcdsQueriesV2_7_0.foreach { q => + test(s"check simplified (tpcds-v2.7.0/$q)") { + testQuery("tpcds-v2.7.0", q) } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometTaskMetricsSuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometTaskMetricsSuite.scala index 05bdfceb1d..acadde674b 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometTaskMetricsSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometTaskMetricsSuite.scala @@ -206,11 +206,10 @@ class CometTaskMetricsSuite extends CometTestBase with AdaptiveSparkPlanHelper { "SELECT * FROM tbl where _1 > 2000", CometConf.COMET_ENABLED.key -> "false") - // Collect input metrics from Comet native_datafusion scan. + // Collect input metrics from Comet native scan. val (cometBytes, cometRecords, cometPlan) = collectInputMetrics( "SELECT * FROM tbl where _1 > 2000", - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) + CometConf.COMET_ENABLED.key -> "true") // Verify the plan actually used CometNativeScanExec assert( @@ -258,10 +257,8 @@ class CometTaskMetricsSuite extends CometTestBase with AdaptiveSparkPlanHelper { collectInputMetrics(joinQuery, CometConf.COMET_ENABLED.key -> "false") // Collect from Comet native scan - val (cometBytes, cometRecords, cometPlan) = collectInputMetrics( - joinQuery, - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) + val (cometBytes, cometRecords, cometPlan) = + collectInputMetrics(joinQuery, CometConf.COMET_ENABLED.key -> "true") // Verify the plan has multiple CometNativeScanExec nodes val scanCount = collect(cometPlan) { case s: CometNativeScanExec => @@ -329,10 +326,8 @@ class CometTaskMetricsSuite extends CometTestBase with AdaptiveSparkPlanHelper { collectInputMetrics(unionQuery, CometConf.COMET_ENABLED.key -> "false") // Collect from Comet native scan - val (cometBytes, cometRecords, cometPlan) = collectInputMetrics( - unionQuery, - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) + val (cometBytes, cometRecords, cometPlan) = + collectInputMetrics(unionQuery, CometConf.COMET_ENABLED.key -> "true") // Verify the plan has multiple CometNativeScanExec nodes val scanCount = collect(cometPlan) { case s: CometNativeScanExec => diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala index db07b91e93..cf9be34289 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.comet.{CometConf, IntegrationTestSuite} -import org.apache.comet.CometConf.{SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT} /** * A integration test suite that tests parquet modular encryption usage. @@ -236,7 +235,7 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { assert(parquetDF.inputFiles.nonEmpty) val readDataset = parquetDF.select("a", "b", "c") - // native_datafusion and native_iceberg_compat fall back due to Arrow-rs + // native_datafusion falls back due to Arrow-rs // https://github.com/apache/arrow-rs/blob/da9829728e2a9dffb8d4f47ffe7b103793851724/parquet/src/file/metadata/parser.rs#L494 checkAnswer(readDataset, inputDF) } @@ -435,8 +434,7 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { assert(parquetDF.inputFiles.nonEmpty) val readDataset = parquetDF.select("a", "b", "c") - // native_datafusion and native_iceberg_compat fall back due to Arrow-rs not - // supporting other key lengths + // native_datafusion falls back due to Arrow-rs not supporting other key lengths checkAnswer(readDataset, inputDF) } } @@ -456,26 +454,12 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { pos: Position): Unit = { Seq("true", "false").foreach { cometEnabled => - if (cometEnabled == "true") { - Seq(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT).foreach { scanImpl => - super.test(testName + s" Comet($cometEnabled)" + s" Scan($scanImpl)", testTags: _*) { - withSQLConf( - CometConf.COMET_ENABLED.key -> cometEnabled, - CometConf.COMET_EXEC_ENABLED.key -> "true", - SQLConf.ANSI_ENABLED.key -> "false", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> scanImpl) { - testFun - } - } - } - } else { - super.test(testName + s" Comet($cometEnabled)", testTags: _*) { - withSQLConf( - CometConf.COMET_ENABLED.key -> cometEnabled, - CometConf.COMET_EXEC_ENABLED.key -> "true", - SQLConf.ANSI_ENABLED.key -> "false") { - testFun - } + super.test(testName + s" Comet($cometEnabled)", testTags: _*) { + withSQLConf( + CometConf.COMET_ENABLED.key -> cometEnabled, + CometConf.COMET_EXEC_ENABLED.key -> "true", + SQLConf.ANSI_ENABLED.key -> "false") { + testFun } } }