From e763507b035fcdc4dea5bed36600e418e6bd5a22 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 11 May 2026 12:39:01 -0600 Subject: [PATCH 01/14] chore: remove config option for native_iceberg_compat COMET_NATIVE_SCAN_IMPL config now only allows auto or native_datafusion. CometScanRule always uses native_datafusion. Tests no longer parameterize on scan impl and iceberg_compat-specific tests are removed. Golden files are regenerated in a follow-up commit. --- .github/actions/java-test/action.yaml | 6 - .github/workflows/pr_build_linux.yml | 11 +- .github/workflows/spark_sql_test.yml | 18 +- .../spark_sql_test_native_iceberg_compat.yml | 72 ------ .../scala/org/apache/comet/CometConf.scala | 19 +- docs/source/contributor-guide/bug_triage.md | 8 +- docs/source/user-guide/latest/datasources.md | 12 +- .../apache/comet/rules/CometScanRule.scala | 25 +-- .../comet/CometArrayExpressionSuite.scala | 2 +- .../comet/CometCsvExpressionSuite.scala | 2 +- .../apache/comet/exec/CometExecSuite.scala | 75 +++---- .../comet/exec/CometNativeReaderSuite.scala | 21 +- .../comet/parquet/ParquetReadSuite.scala | 206 ++++++------------ .../comet/rules/CometScanRuleSuite.scala | 2 +- .../sql/benchmark/CometBenchmarkBase.scala | 18 +- .../sql/comet/CometPlanStabilitySuite.scala | 27 +-- .../sql/comet/ParquetEncryptionITCase.scala | 25 +-- 17 files changed, 166 insertions(+), 383 deletions(-) delete mode 100644 .github/workflows/spark_sql_test_native_iceberg_compat.yml 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/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 2826aeeecc..a0882ff0c0 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -285,32 +285,26 @@ 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 17" java_version: "17" maven_opts: "-Pspark-4.0" - scan_impl: "auto" - 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: | @@ -399,7 +393,7 @@ jobs: 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: ${{ github.repository_owner == 'apache' && format('runs-on={0},family=m8a+m7a+c8a,cpu=16,image=ubuntu24-full-x64,extras=s3-cache,disk=large,tag=datafusion-comet', github.run_id) || 'ubuntu-latest' }} container: image: amd64/rust @@ -437,10 +431,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/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 4fa70b399e..142cf4398e 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -132,16 +132,14 @@ 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: 17, 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: 17} + - {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 }} # Hive tests stay on the standard GitHub-hosted runner: HiveSparkSubmitSuite # relies on an Ivy 'local-m2-cache' resolver that the runs-on.com # ubuntu24-full-x64 image does not provide, so spark-submit fails there. @@ -171,7 +169,7 @@ 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' }} \ + 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 6144 ${{ 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 @@ -190,7 +188,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/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 9b376837f7..72a4e7a4e3 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -111,22 +111,23 @@ object CometConf extends ShimCometConf { .booleanConf .createWithEnvVarOrDefault("ENABLE_COMET_WRITE", false) + @deprecated val SCAN_NATIVE_DATAFUSION = "native_datafusion" + + @deprecated val SCAN_NATIVE_ICEBERG_COMPAT = "native_iceberg_compat" + + @deprecated val SCAN_AUTO = "auto" + @deprecated 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.") + .category(CATEGORY_TESTING) + .internal() + .doc("This configuration option is deprecated and has no effect on Comet behavior.") .stringConf .transform(_.toLowerCase(Locale.ROOT)) - .checkValues(Set(SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, SCAN_AUTO)) + .checkValues(Set(SCAN_NATIVE_DATAFUSION, SCAN_AUTO)) .createWithEnvVarOrDefault("COMET_PARQUET_SCAN_IMPL", SCAN_AUTO) val COMET_ICEBERG_NATIVE_ENABLED: ConfigEntry[Boolean] = 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/user-guide/latest/datasources.md b/docs/source/user-guide/latest/datasources.md index 065b719ba5..a61eda73db 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,7 +165,7 @@ 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 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 aee11d4ce4..7544e0e9a6 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -183,12 +183,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}") @@ -242,21 +237,6 @@ case class CometScanRule(session: SparkSession) 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)) - } - private def transformV2Scan(scanExec: BatchScanExec): SparkPlan = { scanExec.scan match { @@ -799,8 +779,7 @@ object CometScanRule extends Logging { Native.validateObjectStoreConfig(filePath, objectStoreOptions) } catch { case e: CometNativeException => - val reason = "Object store config not supported by " + - s"$SCAN_NATIVE_ICEBERG_COMPAT: ${e.getMessage}" + val reason = s"Object store config not supported: ${e.getMessage}" fallbackReasons += reason configValidityMap.put(cacheKey, Some(reason)) } diff --git a/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala index 63936a94b7..66f40e7a8a 100644 --- a/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala @@ -985,7 +985,7 @@ class CometArrayExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelp test("size - respect to legacySizeOfNull") { val table = "t1" - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT) { + withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { withTable(table) { sql(s"create table $table(col array) using parquet") sql(s"insert into $table values(null)") diff --git a/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala index 399462d5ea..c035abef9b 100644 --- a/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala @@ -71,7 +71,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.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, CometConf.getExprAllowIncompatConfigKey(classOf[StructsToCsv]) -> "true") { withTable(table) { val newLinesStr = 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..dbf60ce90a 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -2269,50 +2269,43 @@ 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", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { + 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") } + } } } 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..b616254983 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,16 @@ 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(s"$testName - ${CometConf.SCAN_NATIVE_DATAFUSION}", 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 -> CometConf.SCAN_NATIVE_DATAFUSION) { + testFun + } + } } test("native reader case sensitivity") { 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..cdb9f49016 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -20,12 +20,11 @@ package org.apache.comet.parquet import java.io.File -import java.math.{BigDecimal, BigInteger} +import java.math.BigDecimal 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) @@ -1621,99 +1620,28 @@ 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) => + ("true", CometConf.SCAN_NATIVE_DATAFUSION, "CometNativeScan")).foreach { + case (cometEnabled, cometNativeScanImpl, expectedScanner) => testScanner( cometEnabled, cometNativeScanImpl, 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) - } - } - }) - } - } - - test("test V1 parquet scan filter pushdown of primitive types uses native_iceberg_compat") { - 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") { - makeParquetFileAllPrimitiveTypes( - path, - dictionaryEnabled = false, - 0, - 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) - - if (pushDown) { - assert(scan.head.metrics("output_rows").value == expectedRows) - } else { - assert(scan.head.metrics("output_rows").value == rows) - } - } - } - } - } + withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { + 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) + } } } } @@ -1724,42 +1652,39 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { 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")) - } - }) + withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { + 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")) + } } } }) @@ -1772,25 +1697,20 @@ 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") - } - } + withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { + 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/rules/CometScanRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala index 6214513f48..cb560c9d91 100644 --- a/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala @@ -124,7 +124,7 @@ class CometScanRuleSuite extends CometTestBase { 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_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "true") { val transformedPlan = applyCometScanRule(sparkPlan) 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..2e8dd97a78 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,7 @@ 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.CometConf.SCAN_NATIVE_DATAFUSION import org.apache.comet.CometSparkSessionExtensions trait CometBenchmarkBase @@ -172,15 +172,13 @@ 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 ($SCAN_NATIVE_DATAFUSION)$suffix") { _ => + withSQLConf( + (extraConf ++ Map( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION)).toSeq: _*) { + spark.sql(query).noop() } } } 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/ParquetEncryptionITCase.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetEncryptionITCase.scala index db07b91e93..74a9d01096 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,7 @@ 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} +import org.apache.comet.CometConf.SCAN_NATIVE_DATAFUSION /** * A integration test suite that tests parquet modular encryption usage. @@ -236,7 +236,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 +435,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) } } @@ -457,15 +456,15 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { 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 - } + super.test( + testName + s" Comet($cometEnabled)" + s" Scan($SCAN_NATIVE_DATAFUSION)", + testTags: _*) { + withSQLConf( + CometConf.COMET_ENABLED.key -> cometEnabled, + CometConf.COMET_EXEC_ENABLED.key -> "true", + SQLConf.ANSI_ENABLED.key -> "false", + CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { + testFun } } } else { From 0b664bdcd248bf8ec8257a2e27e02b4bbce728bd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 11 May 2026 12:54:21 -0600 Subject: [PATCH 02/14] chore: regenerate TPC-DS plan-stability golden files Plan-stability suites no longer parameterize on scan impl, so each query has a single golden directory. native_datafusion dirs are renamed to the unsuffixed name, and native_iceberg_compat dirs are removed. --- .../q44.native_iceberg_compat/extended.txt | 64 - .../extended.txt | 0 .../q58.native_iceberg_compat/extended.txt | 112 - .../extended.txt | 0 .../q67.native_iceberg_compat/extended.txt | 42 - .../extended.txt | 0 .../q70.native_iceberg_compat/extended.txt | 60 - .../extended.txt | 0 .../q83.native_iceberg_compat/extended.txt | 105 - .../extended.txt | 0 .../q14b.native_iceberg_compat/extended.txt | 343 --- .../extended.txt | 0 .../q44.native_iceberg_compat/extended.txt | 65 - .../extended.txt | 0 .../q54.native_iceberg_compat/extended.txt | 95 - .../extended.txt | 0 .../q58.native_iceberg_compat/extended.txt | 116 - .../extended.txt | 0 .../q6.native_iceberg_compat/extended.txt | 59 - .../{q6.native_datafusion => q6}/extended.txt | 0 .../extended.txt | 105 - .../extended.txt | 0 .../q33.native_iceberg_compat/extended.txt | 96 - .../extended.txt | 0 .../q49.native_iceberg_compat/extended.txt | 92 - .../extended.txt | 0 .../q56.native_iceberg_compat/extended.txt | 99 - .../extended.txt | 0 .../q60.native_iceberg_compat/extended.txt | 99 - .../extended.txt | 0 .../q66.native_iceberg_compat/extended.txt | 69 - .../extended.txt | 0 .../q2.native_iceberg_compat/extended.txt | 60 - .../{q2.native_datafusion => q2}/extended.txt | 0 .../q5.native_iceberg_compat/extended.txt | 108 - .../{q5.native_datafusion => q5}/extended.txt | 0 .../q54.native_iceberg_compat/extended.txt | 101 - .../extended.txt | 0 .../q1.native_iceberg_compat/extended.txt | 53 - .../{q1.native_datafusion => q1}/extended.txt | 0 .../q10.native_iceberg_compat/extended.txt | 62 - .../extended.txt | 0 .../q11.native_iceberg_compat/extended.txt | 91 - .../extended.txt | 0 .../q12.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q13.native_iceberg_compat/extended.txt | 42 - .../extended.txt | 0 .../q14a.native_iceberg_compat/extended.txt | 469 ---- .../extended.txt | 0 .../q14b.native_iceberg_compat/extended.txt | 339 --- .../extended.txt | 0 .../q15.native_iceberg_compat/extended.txt | 32 - .../extended.txt | 0 .../q16.native_iceberg_compat/extended.txt | 42 - .../extended.txt | 0 .../q17.native_iceberg_compat/extended.txt | 62 - .../extended.txt | 0 .../q18.native_iceberg_compat/extended.txt | 51 - .../extended.txt | 0 .../q19.native_iceberg_compat/extended.txt | 38 - .../extended.txt | 0 .../q2.native_iceberg_compat/extended.txt | 48 - .../{q2.native_datafusion => q2}/extended.txt | 0 .../q20.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q21.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q22.native_iceberg_compat/extended.txt | 33 - .../extended.txt | 0 .../q23a.native_iceberg_compat/extended.txt | 146 - .../extended.txt | 0 .../q23b.native_iceberg_compat/extended.txt | 198 -- .../extended.txt | 0 .../q24a.native_iceberg_compat/extended.txt | 92 - .../extended.txt | 0 .../q24b.native_iceberg_compat/extended.txt | 92 - .../extended.txt | 0 .../q25.native_iceberg_compat/extended.txt | 62 - .../extended.txt | 0 .../q26.native_iceberg_compat/extended.txt | 39 - .../extended.txt | 0 .../q27.native_iceberg_compat/extended.txt | 40 - .../extended.txt | 0 .../q28.native_iceberg_compat/extended.txt | 72 - .../extended.txt | 0 .../q29.native_iceberg_compat/extended.txt | 67 - .../extended.txt | 0 .../q3.native_iceberg_compat/extended.txt | 21 - .../{q3.native_datafusion => q3}/extended.txt | 0 .../q30.native_iceberg_compat/extended.txt | 65 - .../extended.txt | 0 .../q31.native_iceberg_compat/extended.txt | 126 - .../extended.txt | 0 .../q32.native_iceberg_compat/extended.txt | 42 - .../extended.txt | 0 .../q33.native_iceberg_compat/extended.txt | 97 - .../extended.txt | 0 .../q34.native_iceberg_compat/extended.txt | 41 - .../extended.txt | 0 .../q35.native_iceberg_compat/extended.txt | 62 - .../extended.txt | 0 .../q36.native_iceberg_compat/extended.txt | 38 - .../extended.txt | 0 .../q37.native_iceberg_compat/extended.txt | 34 - .../extended.txt | 0 .../q38.native_iceberg_compat/extended.txt | 70 - .../extended.txt | 0 .../q39a.native_iceberg_compat/extended.txt | 65 - .../extended.txt | 0 .../q39b.native_iceberg_compat/extended.txt | 65 - .../extended.txt | 0 .../q4.native_iceberg_compat/extended.txt | 131 - .../{q4.native_datafusion => q4}/extended.txt | 0 .../q40.native_iceberg_compat/extended.txt | 40 - .../extended.txt | 0 .../q41.native_iceberg_compat/extended.txt | 21 - .../extended.txt | 0 .../q42.native_iceberg_compat/extended.txt | 21 - .../extended.txt | 0 .../q43.native_iceberg_compat/extended.txt | 21 - .../extended.txt | 0 .../q44.native_iceberg_compat/extended.txt | 63 - .../extended.txt | 0 .../q45.native_iceberg_compat/extended.txt | 46 - .../extended.txt | 0 .../q46.native_iceberg_compat/extended.txt | 49 - .../extended.txt | 0 .../q47.native_iceberg_compat/extended.txt | 105 - .../extended.txt | 0 .../q48.native_iceberg_compat/extended.txt | 37 - .../extended.txt | 0 .../q49.native_iceberg_compat/extended.txt | 94 - .../extended.txt | 0 .../q5.native_iceberg_compat/extended.txt | 90 - .../{q5.native_datafusion => q5}/extended.txt | 0 .../q50.native_iceberg_compat/extended.txt | 37 - .../extended.txt | 0 .../q51.native_iceberg_compat/extended.txt | 53 - .../extended.txt | 0 .../q52.native_iceberg_compat/extended.txt | 21 - .../extended.txt | 0 .../q53.native_iceberg_compat/extended.txt | 37 - .../extended.txt | 0 .../q54.native_iceberg_compat/extended.txt | 91 - .../extended.txt | 0 .../q55.native_iceberg_compat/extended.txt | 21 - .../extended.txt | 0 .../q56.native_iceberg_compat/extended.txt | 100 - .../extended.txt | 0 .../q57.native_iceberg_compat/extended.txt | 105 - .../extended.txt | 0 .../q58.native_iceberg_compat/extended.txt | 115 - .../extended.txt | 0 .../q59.native_iceberg_compat/extended.txt | 53 - .../extended.txt | 0 .../q6.native_iceberg_compat/extended.txt | 57 - .../{q6.native_datafusion => q6}/extended.txt | 0 .../q60.native_iceberg_compat/extended.txt | 100 - .../extended.txt | 0 .../q61.native_iceberg_compat/extended.txt | 88 - .../extended.txt | 0 .../q62.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q63.native_iceberg_compat/extended.txt | 37 - .../extended.txt | 0 .../q64.native_iceberg_compat/extended.txt | 247 -- .../extended.txt | 0 .../q65.native_iceberg_compat/extended.txt | 52 - .../extended.txt | 0 .../q66.native_iceberg_compat/extended.txt | 70 - .../extended.txt | 0 .../q67.native_iceberg_compat/extended.txt | 38 - .../extended.txt | 0 .../q68.native_iceberg_compat/extended.txt | 49 - .../extended.txt | 0 .../q69.native_iceberg_compat/extended.txt | 57 - .../extended.txt | 0 .../q7.native_iceberg_compat/extended.txt | 39 - .../{q7.native_datafusion => q7}/extended.txt | 0 .../q70.native_iceberg_compat/extended.txt | 59 - .../extended.txt | 0 .../q71.native_iceberg_compat/extended.txt | 53 - .../extended.txt | 0 .../q72.native_iceberg_compat/extended.txt | 72 - .../extended.txt | 0 .../q73.native_iceberg_compat/extended.txt | 41 - .../extended.txt | 0 .../q74.native_iceberg_compat/extended.txt | 90 - .../extended.txt | 0 .../q75.native_iceberg_compat/extended.txt | 172 -- .../extended.txt | 0 .../q76.native_iceberg_compat/extended.txt | 47 - .../extended.txt | 0 .../q77.native_iceberg_compat/extended.txt | 116 - .../extended.txt | 0 .../q78.native_iceberg_compat/extended.txt | 80 - .../extended.txt | 0 .../q79.native_iceberg_compat/extended.txt | 39 - .../extended.txt | 0 .../q8.native_iceberg_compat/extended.txt | 52 - .../{q8.native_datafusion => q8}/extended.txt | 0 .../q80.native_iceberg_compat/extended.txt | 131 - .../extended.txt | 0 .../q81.native_iceberg_compat/extended.txt | 65 - .../extended.txt | 0 .../q82.native_iceberg_compat/extended.txt | 34 - .../extended.txt | 0 .../q83.native_iceberg_compat/extended.txt | 119 - .../extended.txt | 0 .../q84.native_iceberg_compat/extended.txt | 35 - .../extended.txt | 0 .../q85.native_iceberg_compat/extended.txt | 56 - .../extended.txt | 0 .../q86.native_iceberg_compat/extended.txt | 32 - .../extended.txt | 0 .../q87.native_iceberg_compat/extended.txt | 70 - .../extended.txt | 0 .../q88.native_iceberg_compat/extended.txt | 216 -- .../extended.txt | 0 .../q89.native_iceberg_compat/extended.txt | 37 - .../extended.txt | 0 .../q9.native_iceberg_compat/extended.txt | 61 - .../{q9.native_datafusion => q9}/extended.txt | 0 .../q90.native_iceberg_compat/extended.txt | 55 - .../extended.txt | 0 .../q91.native_iceberg_compat/extended.txt | 51 - .../extended.txt | 0 .../q92.native_iceberg_compat/extended.txt | 42 - .../extended.txt | 0 .../q93.native_iceberg_compat/extended.txt | 24 - .../extended.txt | 0 .../q94.native_iceberg_compat/extended.txt | 42 - .../extended.txt | 0 .../q95.native_iceberg_compat/extended.txt | 64 - .../extended.txt | 0 .../q96.native_iceberg_compat/extended.txt | 27 - .../extended.txt | 0 .../q97.native_iceberg_compat/extended.txt | 37 - .../extended.txt | 0 .../q98.native_iceberg_compat/extended.txt | 34 - .../extended.txt | 0 .../q99.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q67a.native_iceberg_compat/extended.txt | 298 --- .../extended.txt | 0 .../q70a.native_iceberg_compat/extended.txt | 171 -- .../extended.txt | 0 .../q14.native_iceberg_compat/extended.txt | 343 --- .../extended.txt | 0 .../q6.native_iceberg_compat/extended.txt | 59 - .../{q6.native_datafusion => q6}/extended.txt | 0 .../q14a.native_iceberg_compat/extended.txt | 2209 ---------------- .../extended.txt | 0 .../q49.native_iceberg_compat/extended.txt | 92 - .../extended.txt | 0 .../q5a.native_iceberg_compat/extended.txt | 323 --- .../extended.txt | 0 .../q10a.native_iceberg_compat/extended.txt | 56 - .../extended.txt | 0 .../q11.native_iceberg_compat/extended.txt | 90 - .../extended.txt | 0 .../q12.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q14.native_iceberg_compat/extended.txt | 339 --- .../extended.txt | 0 .../q14a.native_iceberg_compat/extended.txt | 2350 ----------------- .../extended.txt | 0 .../q18a.native_iceberg_compat/extended.txt | 214 -- .../extended.txt | 0 .../q20.native_iceberg_compat/extended.txt | 31 - .../extended.txt | 0 .../q22.native_iceberg_compat/extended.txt | 33 - .../extended.txt | 0 .../q22a.native_iceberg_compat/extended.txt | 159 -- .../extended.txt | 0 .../q24.native_iceberg_compat/extended.txt | 95 - .../extended.txt | 0 .../q27a.native_iceberg_compat/extended.txt | 99 - .../extended.txt | 0 .../q34.native_iceberg_compat/extended.txt | 41 - .../extended.txt | 0 .../q35.native_iceberg_compat/extended.txt | 62 - .../extended.txt | 0 .../q35a.native_iceberg_compat/extended.txt | 56 - .../extended.txt | 0 .../q36a.native_iceberg_compat/extended.txt | 105 - .../extended.txt | 0 .../q47.native_iceberg_compat/extended.txt | 105 - .../extended.txt | 0 .../q49.native_iceberg_compat/extended.txt | 94 - .../extended.txt | 0 .../q51a.native_iceberg_compat/extended.txt | 212 -- .../extended.txt | 0 .../q57.native_iceberg_compat/extended.txt | 105 - .../extended.txt | 0 .../q5a.native_iceberg_compat/extended.txt | 269 -- .../extended.txt | 0 .../q6.native_iceberg_compat/extended.txt | 57 - .../{q6.native_datafusion => q6}/extended.txt | 0 .../q64.native_iceberg_compat/extended.txt | 247 -- .../extended.txt | 0 .../q67a.native_iceberg_compat/extended.txt | 294 --- .../extended.txt | 0 .../q70a.native_iceberg_compat/extended.txt | 168 -- .../extended.txt | 0 .../q72.native_iceberg_compat/extended.txt | 72 - .../extended.txt | 0 .../q74.native_iceberg_compat/extended.txt | 90 - .../extended.txt | 0 .../q75.native_iceberg_compat/extended.txt | 172 -- .../extended.txt | 0 .../q77a.native_iceberg_compat/extended.txt | 350 --- .../extended.txt | 0 .../q78.native_iceberg_compat/extended.txt | 80 - .../extended.txt | 0 .../q80a.native_iceberg_compat/extended.txt | 392 --- .../extended.txt | 0 .../q86a.native_iceberg_compat/extended.txt | 87 - .../extended.txt | 0 .../q98.native_iceberg_compat/extended.txt | 33 - .../extended.txt | 0 322 files changed, 19262 deletions(-) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q44.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/{q44.native_datafusion => q44}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q58.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/{q58.native_datafusion => q58}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q67.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/{q67.native_datafusion => q67}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/{q70.native_datafusion => q70}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q83.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/{q83.native_datafusion => q83}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/{q14b.native_datafusion => q14b}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/{q44.native_datafusion => q44}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/{q54.native_datafusion => q54}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/{q58.native_datafusion => q58}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/{q6.native_datafusion => q6}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/{q83.ansi.native_datafusion => q83.ansi}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q33.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/{q33.native_datafusion => q33}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q49.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/{q49.native_datafusion => q49}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q56.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/{q56.native_datafusion => q56}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q60.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/{q60.native_datafusion => q60}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/q66.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_1/{q66.native_datafusion => q66}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q2.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/{q2.native_datafusion => q2}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q5.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/{q5.native_datafusion => q5}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/q54.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_2/{q54.native_datafusion => q54}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q1.native_datafusion => q1}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q10.native_datafusion => q10}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q11.native_datafusion => q11}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q12.native_datafusion => q12}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q13.native_datafusion => q13}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q14a.native_datafusion => q14a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q14b.native_datafusion => q14b}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q15.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q15.native_datafusion => q15}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q16.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q16.native_datafusion => q16}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q17.native_datafusion => q17}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q18.native_datafusion => q18}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q19.native_datafusion => q19}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q2.native_datafusion => q2}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q20.native_datafusion => q20}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q21.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q21.native_datafusion => q21}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q22.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q22.native_datafusion => q22}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q23a.native_datafusion => q23a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q23b.native_datafusion => q23b}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q24a.native_datafusion => q24a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q24b.native_datafusion => q24b}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q25.native_datafusion => q25}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q26.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q26.native_datafusion => q26}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q27.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q27.native_datafusion => q27}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q28.native_datafusion => q28}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q29.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q29.native_datafusion => q29}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q3.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q3.native_datafusion => q3}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q30.native_datafusion => q30}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q31.native_datafusion => q31}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q32.native_datafusion => q32}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q33.native_datafusion => q33}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q34.native_datafusion => q34}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q35.native_datafusion => q35}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q36.native_datafusion => q36}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q37.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q37.native_datafusion => q37}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q38.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q38.native_datafusion => q38}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q39a.native_datafusion => q39a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q39b.native_datafusion => q39b}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q4.native_datafusion => q4}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q40.native_datafusion => q40}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q41.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q41.native_datafusion => q41}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q42.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q42.native_datafusion => q42}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q43.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q43.native_datafusion => q43}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q44.native_datafusion => q44}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q45.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q45.native_datafusion => q45}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q46.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q46.native_datafusion => q46}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q47.native_datafusion => q47}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q48.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q48.native_datafusion => q48}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q49.native_datafusion => q49}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q5.native_datafusion => q5}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q50.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q50.native_datafusion => q50}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q51.native_datafusion => q51}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q52.native_datafusion => q52}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q53.native_datafusion => q53}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q54.native_datafusion => q54}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q55.native_datafusion => q55}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q56.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q56.native_datafusion => q56}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q57.native_datafusion => q57}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q58.native_datafusion => q58}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q59.native_datafusion => q59}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q6.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q6.native_datafusion => q6}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q60.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q60.native_datafusion => q60}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q61.native_datafusion => q61}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q62.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q62.native_datafusion => q62}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q63.native_datafusion => q63}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q64.native_datafusion => q64}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q65.native_datafusion => q65}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q66.native_datafusion => q66}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q67.native_datafusion => q67}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q68.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q68.native_datafusion => q68}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q69.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q69.native_datafusion => q69}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q7.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q7.native_datafusion => q7}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q70.native_datafusion => q70}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q71.native_datafusion => q71}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q72.native_datafusion => q72}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q73.native_datafusion => q73}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q74.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q74.native_datafusion => q74}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q75.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q75.native_datafusion => q75}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q76.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q76.native_datafusion => q76}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q77.native_datafusion => q77}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q78.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q78.native_datafusion => q78}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q79.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q79.native_datafusion => q79}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q8.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q8.native_datafusion => q8}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q80.native_datafusion => q80}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q81.native_datafusion => q81}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q82.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q82.native_datafusion => q82}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q83.native_datafusion => q83}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q84.native_datafusion => q84}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q85.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q85.native_datafusion => q85}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q86.native_datafusion => q86}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q87.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q87.native_datafusion => q87}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q88.native_datafusion => q88}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q89.native_datafusion => q89}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q9.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q9.native_datafusion => q9}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q90.native_datafusion => q90}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q91.native_datafusion => q91}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q92.native_datafusion => q92}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q93.native_datafusion => q93}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q94.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q94.native_datafusion => q94}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q95.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q95.native_datafusion => q95}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q96.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q96.native_datafusion => q96}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q97.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q97.native_datafusion => q97}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q98.native_datafusion => q98}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q99.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/{q99.native_datafusion => q99}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q67a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/{q67a.native_datafusion => q67a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/{q70a.native_datafusion => q70a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/{q14.native_datafusion => q14}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/{q6.native_datafusion => q6}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q14a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/{q14a.native_datafusion => q14a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/q49.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_1/{q49.native_datafusion => q49}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/q5a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_2/{q5a.native_datafusion => q5a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q10a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q10a.native_datafusion => q10a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q11.native_datafusion => q11}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q12.native_datafusion => q12}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q14.native_datafusion => q14}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q14a.native_datafusion => q14a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q18a.native_datafusion => q18a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q20.native_datafusion => q20}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q22.native_datafusion => q22}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q22a.native_datafusion => q22a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q24.native_datafusion => q24}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q27a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q27a.native_datafusion => q27a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q34.native_datafusion => q34}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q35.native_datafusion => q35}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q35a.native_datafusion => q35a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q36a.native_datafusion => q36a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q47.native_datafusion => q47}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q49.native_datafusion => q49}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q51a.native_datafusion => q51a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q57.native_datafusion => q57}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q5a.native_datafusion => q5a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q6.native_datafusion => q6}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q64.native_datafusion => q64}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q67a.native_datafusion => q67a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q70a.native_datafusion => q70a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q72.native_datafusion => q72}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q74.native_datafusion => q74}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q75.native_datafusion => q75}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q77a.native_datafusion => q77a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q78.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q78.native_datafusion => q78}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q80a.native_datafusion => q80a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q86a.native_datafusion => q86a}/extended.txt (100%) delete mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt rename spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/{q98.native_datafusion => q98}/extended.txt (100%) 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 100% 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 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-v1_4/q10.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q10/extended.txt similarity index 100% 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/q10/extended.txt 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 100% 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 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 100% 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 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/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q35/extended.txt similarity index 100% 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-v1_4/q35/extended.txt 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 100% 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 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 100% 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 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 100% 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 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 100% 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 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 100% 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 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 100% 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 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-v2_7/q35.native_datafusion/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q35/extended.txt similarity index 100% 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-v2_7/q35/extended.txt 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 100% 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 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 100% 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 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 100% 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 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 From d454495f40018cd1d8a9bcb6ab77c9552066c9c1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 16:24:44 -0600 Subject: [PATCH 03/14] test: restore primitive-type filter pushdown test for native_datafusion --- .../comet/parquet/ParquetReadSuite.scala | 62 ++++++++++++++++++- 1 file changed, 61 insertions(+), 1 deletion(-) 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 a49b976a94..e44379c284 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -20,7 +20,7 @@ package org.apache.comet.parquet import java.io.File -import java.math.BigDecimal +import java.math.{BigDecimal, BigInteger} import java.time.{ZoneId, ZoneOffset} import scala.reflect.ClassTag @@ -1730,6 +1730,66 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { } } + 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_PARQUET_UNSIGNED_SMALL_INT_CHECK.key -> "true") { + makeParquetFileAllPrimitiveTypes( + path, + dictionaryEnabled = false, + 0, + rows, + nullEnabled = false) + } + Seq(true, false).foreach { pushDown => + withSQLConf( + CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, + 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) + } + } + } + } + } + } + test("read basic complex types") { Seq(true, false).foreach(dictionaryEnabled => { withTempPath { dir => From 02226af4ce349742247b318bac234fc93c9ee574 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 18:38:10 -0600 Subject: [PATCH 04/14] chore: remove COMET_NATIVE_SCAN_IMPL and related scan-impl constants Removes the four deprecated symbols from CometConf along with all references in main code, tests, and benchmarks: - COMET_NATIVE_SCAN_IMPL (spark.comet.scan.impl) - SCAN_NATIVE_DATAFUSION - SCAN_NATIVE_ICEBERG_COMPAT - SCAN_AUTO With a single Parquet scan implementation, the scanImpl field on CometScanExec is dropped, CometScanTypeChecker becomes parameterless, and per-impl conditionals in CometScanRule and CometExecRule collapse. --- benchmarks/tpc/engines/comet-hashjoin.toml | 1 - benchmarks/tpc/engines/comet.toml | 1 - .../scala/org/apache/comet/CometConf.scala | 19 -- .../apache/comet/rules/CometExecRule.scala | 2 +- .../apache/comet/rules/CometScanRule.scala | 42 ++-- .../spark/sql/comet/CometScanExec.scala | 41 ++-- .../comet/CometArrayExpressionSuite.scala | 22 +-- .../org/apache/comet/CometCastSuite.scala | 9 +- .../comet/CometCsvExpressionSuite.scala | 4 +- .../apache/comet/CometExpressionSuite.scala | 15 -- .../comet/CometHashExpressionSuite.scala | 12 -- .../apache/comet/CometSqlFileTestSuite.scala | 12 -- .../apache/comet/exec/CometExecSuite.scala | 11 +- .../exec/CometNativeColumnarToRowSuite.scala | 4 +- .../comet/exec/CometNativeReaderSuite.scala | 5 +- .../comet/exec/CometNativeShuffleSuite.scala | 5 +- .../comet/exec/CometWindowExecSuite.scala | 3 +- .../conditional/CometCaseWhenSuite.scala | 14 -- .../conditional/CometCoalesceSuite.scala | 14 -- .../conditional/CometIfSuite.scala | 14 -- .../parquet/CometParquetWriterSuite.scala | 29 ++- .../ParquetReadFromFakeHadoopFsSuite.scala | 10 +- .../comet/parquet/ParquetReadSuite.scala | 184 +++++++----------- .../ParquetTimestampLtzAsNtzSuite.scala | 42 +--- .../comet/rules/CometScanRuleSuite.scala | 4 +- .../CometArrayExpressionBenchmark.scala | 12 +- .../sql/benchmark/CometBenchmarkBase.scala | 6 +- .../sql/comet/CometTaskMetricsSuite.scala | 17 +- .../sql/comet/ParquetEncryptionITCase.scala | 27 +-- 29 files changed, 155 insertions(+), 426 deletions(-) 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/spark/src/main/scala/org/apache/comet/CometConf.scala b/spark/src/main/scala/org/apache/comet/CometConf.scala index 1581abccff..1a4eb3a8c8 100644 --- a/spark/src/main/scala/org/apache/comet/CometConf.scala +++ b/spark/src/main/scala/org/apache/comet/CometConf.scala @@ -111,25 +111,6 @@ object CometConf extends ShimCometConf { .booleanConf .createWithEnvVarOrDefault("ENABLE_COMET_WRITE", false) - @deprecated - val SCAN_NATIVE_DATAFUSION = "native_datafusion" - - @deprecated - val SCAN_NATIVE_ICEBERG_COMPAT = "native_iceberg_compat" - - @deprecated - val SCAN_AUTO = "auto" - - @deprecated - val COMET_NATIVE_SCAN_IMPL: ConfigEntry[String] = conf("spark.comet.scan.impl") - .category(CATEGORY_TESTING) - .internal() - .doc("This configuration option is deprecated and has no effect on Comet behavior.") - .stringConf - .transform(_.toLowerCase(Locale.ROOT)) - .checkValues(Set(SCAN_NATIVE_DATAFUSION, 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) 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..3c79bf7e9e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -258,7 +258,7 @@ case class CometExecRule(session: SparkSession) 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 => + case scan: CometScanExec => convertToComet(scan, CometNativeScan).getOrElse(scan) // Fully native Iceberg scan for V2 (iceberg-rust path) 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 519295cc6b..835925904b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -197,9 +197,7 @@ case class CometScanRule(session: SparkSession) r: HadoopFsRelation, hadoopConf: Configuration): Option[SparkPlan] = { if (!COMET_EXEC_ENABLED.get()) { - withInfo( - scanExec, - s"$SCAN_NATIVE_DATAFUSION scan requires ${COMET_EXEC_ENABLED.key} to be enabled") + 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 @@ -210,7 +208,7 @@ case class CometScanRule(session: SparkSession) !COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER.get()) { withInfo( scanExec, - s"$SCAN_NATIVE_DATAFUSION scan is incompatible with " + + s"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 @@ -219,7 +217,7 @@ case class CometScanRule(session: SparkSession) 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) { @@ -244,10 +242,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)) + Some(CometScanExec(scanExec, session)) } private def transformV2Scan(scanExec: BatchScanExec): SparkPlan = { @@ -313,7 +311,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) @@ -670,19 +668,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 = @@ -690,19 +684,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, @@ -710,8 +700,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 += s"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 @@ -722,9 +712,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 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..4efd35fb3e 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 @@ -42,22 +42,15 @@ 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.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. Most of the code here follows Spark's + * [[FileSourceScanExec]]. After CometScanRule runs, this node is replaced by a fully native scan + * by CometExecRule; it does not survive to execution time. */ case class CometScanExec( - scanImpl: String, @transient relation: HadoopFsRelation, output: Seq[Attribute], requiredSchema: StructType, @@ -72,10 +65,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 +145,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) } /** @@ -516,7 +502,6 @@ case class CometScanExec( override def doCanonicalize(): CometScanExec = { CometScanExec( - scanImpl, relation, output.map(QueryPlan.normalizeExpressions(_, output)), requiredSchema, @@ -534,10 +519,7 @@ case class CometScanExec( object CometScanExec { - def apply( - scanExec: FileSourceScanExec, - session: SparkSession, - scanImpl: String): CometScanExec = { + def apply(scanExec: FileSourceScanExec, session: SparkSession): CometScanExec = { // TreeNode.mapProductIterator is protected method. def mapProductIterator[B: ClassTag](product: Product, f: Any => B): Array[B] = { val arr = Array.ofDim[B](product.productArity) @@ -563,7 +545,6 @@ object CometScanExec { val newArgs = mapProductIterator(scanExec, transform) val wrapped = scanExec.makeCopy(newArgs).asInstanceOf[FileSourceScanExec] val batchScanExec = CometScanExec( - scanImpl, wrapped.relation, wrapped.output, wrapped.requiredSchema, diff --git a/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala index 66f40e7a8a..63d8f44ebd 100644 --- a/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometArrayExpressionSuite.scala @@ -985,18 +985,16 @@ class CometArrayExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelp test("size - respect to legacySizeOfNull") { val table = "t1" - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { - 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..697ada3c23 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -1540,13 +1540,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, diff --git a/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometCsvExpressionSuite.scala index c035abef9b..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_DATAFUSION, - 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/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 9b5e85be3b..a1afac4ee1 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 = @@ -2516,7 +2504,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 +2533,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 +3000,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 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/CometSqlFileTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala index 5a0b34e056..037e50ca70 100644 --- a/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometSqlFileTestSuite.scala @@ -21,23 +21,11 @@ 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 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) 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 dbf60ce90a..86c08fb6b8 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,9 +2267,7 @@ class CometExecSuite extends CometTestBase { } test("Comet native metrics: scan") { - withSQLConf( - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { + withSQLConf(CometConf.COMET_EXEC_ENABLED.key -> "true") { withTempDir { dir => val path = new Path(dir.toURI.toString, "native-scan.parquet") makeParquetFileAllPrimitiveTypes(path, dictionaryEnabled = true, 10000) @@ -3936,8 +3932,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/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 b616254983..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,13 +42,12 @@ class CometNativeReaderSuite extends CometTestBase with AdaptiveSparkPlanHelper override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { - super.test(s"$testName - ${CometConf.SCAN_NATIVE_DATAFUSION}", testTags: _*) { + 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", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { + CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "false") { testFun } } 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/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 e44379c284..81e88a9a20 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -993,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) @@ -1013,9 +1011,7 @@ 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( - 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 // CAST('1.2' AS BINARY) writes BYTE_ARRAY with no decimal annotation. @@ -1044,9 +1040,7 @@ abstract class ParquetReadSuite extends CometTestBase { // Regression guard for #4089 and #4343. Spark's `isDecimalTypeMatched` // accepts decimal-to-decimal only when `scaleIncrease >= 0` AND // `precisionIncrease >= scaleIncrease`. - 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") { val cases = Seq( // (file_p, file_s, read_p, read_s) (10, 2, 5, 0), // #4089: scale narrows. @@ -1072,9 +1066,7 @@ abstract class ParquetReadSuite extends CometTestBase { 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( - 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") { // INT32 source (Byte/Short/Int all written as INT32 by Spark). Seq("byte", "short", "int").foreach { writeType => withTempPath { dir => @@ -1097,9 +1089,7 @@ abstract class ParquetReadSuite extends CometTestBase { 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( - 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") { val cases = Seq( ("bigint", "8589934592", "int"), ("double", "1e40", "float"), @@ -1410,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") @@ -1504,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)), @@ -1518,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) @@ -1534,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", @@ -1581,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) @@ -1596,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() @@ -1633,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() @@ -1702,30 +1678,22 @@ 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")).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) - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { - 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) } } } @@ -1743,9 +1711,7 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { nullEnabled = false) } Seq(true, false).foreach { pushDown => - withSQLConf( - CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION, - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushDown.toString) { + 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 @@ -1796,39 +1762,37 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { val path = new Path(dir.toURI.toString, "complex_types.parquet") makeParquetFileComplexTypes(path, dictionaryEnabled, 10) withParquetTable(path.toUri.toString, "complex_types") { - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { - 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")) - } + 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")) } } }) @@ -1841,20 +1805,18 @@ class ParquetReadV1Suite extends ParquetReadSuite with AdaptiveSparkPlanHelper { val file = getResourceParquetFilePath("test-data/before_1582_date_v3_2_0.snappy.parquet") - withSQLConf(CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_DATAFUSION) { - val df = spark.read.parquet(file) + val df = spark.read.parquet(file) - // Verify Comet scan is in the plan - val plan = df.queryExecution.executedPlan - checkCometOperators(plan) + // 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") - } + // 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/CometScanRuleSuite.scala b/spark/src/test/scala/org/apache/comet/rules/CometScanRuleSuite.scala index cb560c9d91..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_DATAFUSION, - 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/sql/benchmark/CometArrayExpressionBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometArrayExpressionBenchmark.scala index 44ef1a4735..b674906bae 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 @@ -112,14 +112,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 +140,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 2e8dd97a78..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 import org.apache.comet.CometSparkSessionExtensions trait CometBenchmarkBase @@ -172,12 +171,11 @@ trait CometBenchmarkBase } } - benchmark.addCase(s"SQL Parquet - Comet ($SCAN_NATIVE_DATAFUSION)$suffix") { _ => + benchmark.addCase(s"SQL Parquet - Comet$suffix") { _ => withSQLConf( (extraConf ++ Map( CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION)).toSeq: _*) { + CometConf.COMET_EXEC_ENABLED.key -> "true")).toSeq: _*) { spark.sql(query).noop() } } 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 74a9d01096..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 /** * A integration test suite that tests parquet modular encryption usage. @@ -455,26 +454,12 @@ class ParquetEncryptionITCase extends CometTestBase with SQLTestUtils { pos: Position): Unit = { Seq("true", "false").foreach { cometEnabled => - if (cometEnabled == "true") { - super.test( - testName + s" Comet($cometEnabled)" + s" Scan($SCAN_NATIVE_DATAFUSION)", - testTags: _*) { - withSQLConf( - CometConf.COMET_ENABLED.key -> cometEnabled, - CometConf.COMET_EXEC_ENABLED.key -> "true", - SQLConf.ANSI_ENABLED.key -> "false", - CometConf.COMET_NATIVE_SCAN_IMPL.key -> SCAN_NATIVE_DATAFUSION) { - 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 } } } From d03d99a7d93d07f23a129ba3bb692f4e08d3764a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 19:42:46 -0600 Subject: [PATCH 05/14] chore: remove dead native_iceberg_compat JVM Parquet reader chain MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Now that native_iceberg_compat is gone, the JVM-mediated V1 Parquet read path (CometScanExec → CometScanWrapper → CometParquetFileFormat → NativeBatchReader / column-level Native APIs) is unreachable. Iceberg also no longer integrates against Comet's @IcebergApi surface. - CometExecRule: V1 CometScanExec fallback now reverts to wrapped FileSourceScanExec (Spark) instead of CometScanWrapper. - CometScanExec: keep as planning intermediate, throw on doExecute/doExecuteColumnar/executeCollect/inputRDDs. - Drop file-format swap to CometParquetFileFormat in both CometScanExec.apply and CometNativeScanExec.apply. - Delete CometParquetFileFormat and the JVM Parquet reader chain (NativeBatchReader, NativeColumnReader, ColumnReader, RowGroupReader, FileReader, plus all supporting page/index/options/utility classes). - Delete IcebergCometNativeBatchReader and CometLazyVector. - Trim Native.java to the Arrow-native scan APIs. - Delete IcebergApi annotation, CometSchemaImporter, AbstractCometSchemaImporter; strip @IcebergApi from CometVector. - Delete now-orphaned tests (TestFileReader, TestUtils, TestCometInputFile); drop closeColumnReader NPE assertion in CometNativeSuite. --- .../arrow/c/AbstractCometSchemaImporter.java | 76 - .../org/apache/comet/CometSchemaImporter.java | 32 - .../java/org/apache/comet/IcebergApi.java | 44 - .../comet/parquet/AbstractColumnReader.java | 138 -- .../parquet/ArrowConstantColumnReader.java | 255 ---- .../parquet/ArrowRowIndexColumnReader.java | 109 -- .../comet/parquet/BloomFilterReader.java | 253 ---- .../comet/parquet/ColumnIndexReader.java | 230 --- .../comet/parquet/ColumnPageReader.java | 252 ---- .../apache/comet/parquet/ColumnReader.java | 314 ---- .../apache/comet/parquet/CometInputFile.java | 157 -- .../comet/parquet/DictionaryPageReader.java | 190 --- .../org/apache/comet/parquet/FileReader.java | 1308 ----------------- .../apache/comet/parquet/FooterReader.java | 56 - .../IcebergCometNativeBatchReader.java | 85 -- .../org/apache/comet/parquet/IndexFilter.java | 145 -- .../comet/parquet/LazyColumnReader.java | 188 --- .../java/org/apache/comet/parquet/Native.java | 180 --- .../comet/parquet/NativeBatchReader.java | 1101 -------------- .../comet/parquet/NativeColumnReader.java | 159 -- .../comet/parquet/ParquetColumnSpec.java | 112 -- .../parquet/ParquetMetadataSerializer.java | 74 - .../org/apache/comet/parquet/ReadOptions.java | 216 --- .../apache/comet/parquet/RowGroupFilter.java | 125 -- .../apache/comet/parquet/RowGroupReader.java | 88 -- .../org/apache/comet/parquet/TypeUtil.java | 339 ----- .../java/org/apache/comet/parquet/Utils.java | 525 ------- .../comet/parquet/WrappedInputFile.java | 71 - .../parquet/WrappedSeekableInputStream.java | 64 - .../apache/comet/vector/CometLazyVector.java | 82 -- .../org/apache/comet/vector/CometVector.java | 9 - .../parquet/CometParquetFileFormat.scala | 260 ---- .../apache/comet/rules/CometExecRule.scala | 5 +- .../spark/sql/comet/CometNativeScanExec.scala | 48 +- .../spark/sql/comet/CometScanExec.scala | 201 +-- .../comet/parquet/TestCometInputFile.java | 44 - .../apache/comet/parquet/TestFileReader.java | 835 ----------- .../org/apache/comet/parquet/TestUtils.java | 149 -- .../org/apache/comet/CometNativeSuite.scala | 5 - 39 files changed, 42 insertions(+), 8482 deletions(-) delete mode 100644 spark/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java delete mode 100644 spark/src/main/java/org/apache/comet/CometSchemaImporter.java delete mode 100644 spark/src/main/java/org/apache/comet/IcebergApi.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/BloomFilterReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ColumnPageReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ColumnReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/CometInputFile.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/FileReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/FooterReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/IcebergCometNativeBatchReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/IndexFilter.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/LazyColumnReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/NativeBatchReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/NativeColumnReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ParquetMetadataSerializer.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/ReadOptions.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/RowGroupFilter.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/RowGroupReader.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/TypeUtil.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/Utils.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/WrappedInputFile.java delete mode 100644 spark/src/main/java/org/apache/comet/parquet/WrappedSeekableInputStream.java delete mode 100644 spark/src/main/java/org/apache/comet/vector/CometLazyVector.java delete mode 100644 spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala delete mode 100644 spark/src/test/java/org/apache/comet/parquet/TestCometInputFile.java delete mode 100644 spark/src/test/java/org/apache/comet/parquet/TestFileReader.java delete mode 100644 spark/src/test/java/org/apache/comet/parquet/TestUtils.java diff --git a/spark/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java b/spark/src/main/java/org/apache/arrow/c/AbstractCometSchemaImporter.java deleted file mode 100644 index 49ce92a408..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/CometSchemaImporter.java b/spark/src/main/java/org/apache/comet/CometSchemaImporter.java deleted file mode 100644 index 4841f16f19..0000000000 --- a/spark/src/main/java/org/apache/comet/CometSchemaImporter.java +++ /dev/null @@ -1,32 +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 org.apache.arrow.c.*; -import org.apache.arrow.memory.BufferAllocator; - -/** 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); - } -} diff --git a/spark/src/main/java/org/apache/comet/IcebergApi.java b/spark/src/main/java/org/apache/comet/IcebergApi.java deleted file mode 100644 index 915fd87848..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java b/spark/src/main/java/org/apache/comet/parquet/AbstractColumnReader.java deleted file mode 100644 index bb5f33fa68..0000000000 --- a/spark/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 = CometConf.COMET_SCHEMA_EVOLUTION_ENABLED() ? type : null; - boolean useLegacyDateTimestampOrNTZ = - useLegacyDateTimestamp || type == TimestampNTZType$.MODULE$; - nativeHandle = - Utils.initColumnReader( - descriptor, readType, batchSize, useDecimal128, useLegacyDateTimestampOrNTZ); - } -} diff --git a/spark/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java b/spark/src/main/java/org/apache/comet/parquet/ArrowConstantColumnReader.java deleted file mode 100644 index 521eb4aa5b..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java b/spark/src/main/java/org/apache/comet/parquet/ArrowRowIndexColumnReader.java deleted file mode 100644 index 7d17e551df..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/BloomFilterReader.java b/spark/src/main/java/org/apache/comet/parquet/BloomFilterReader.java deleted file mode 100644 index a23216c7f5..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java b/spark/src/main/java/org/apache/comet/parquet/ColumnIndexReader.java deleted file mode 100644 index 4e4f6ba0db..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ColumnPageReader.java b/spark/src/main/java/org/apache/comet/parquet/ColumnPageReader.java deleted file mode 100644 index 744d128304..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ColumnReader.java b/spark/src/main/java/org/apache/comet/parquet/ColumnReader.java deleted file mode 100644 index 6ef3bbf73c..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/CometInputFile.java b/spark/src/main/java/org/apache/comet/parquet/CometInputFile.java deleted file mode 100644 index eb54d1a724..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java b/spark/src/main/java/org/apache/comet/parquet/DictionaryPageReader.java deleted file mode 100644 index b4b5a8fc96..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/FileReader.java b/spark/src/main/java/org/apache/comet/parquet/FileReader.java deleted file mode 100644 index 80c214fc7c..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/FooterReader.java b/spark/src/main/java/org/apache/comet/parquet/FooterReader.java deleted file mode 100644 index 092eb442ce..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/IcebergCometNativeBatchReader.java b/spark/src/main/java/org/apache/comet/parquet/IcebergCometNativeBatchReader.java deleted file mode 100644 index 7748fbbe29..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/IndexFilter.java b/spark/src/main/java/org/apache/comet/parquet/IndexFilter.java deleted file mode 100644 index afa5687cae..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/LazyColumnReader.java b/spark/src/main/java/org/apache/comet/parquet/LazyColumnReader.java deleted file mode 100644 index f2772908b9..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/Native.java b/spark/src/main/java/org/apache/comet/parquet/Native.java index fbe7f23875..13762ec11f 100644 --- a/spark/src/main/java/org/apache/comet/parquet/Native.java +++ b/spark/src/main/java/org/apache/comet/parquet/Native.java @@ -21,184 +21,11 @@ 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 @@ -231,8 +58,6 @@ public static native long initRecordBatchReader( CometFileKeyUnwrapper keyUnwrapper, Object metricsNode); - // arrow native version of read batch - /** * Read the next batch of data into memory on native side * @@ -241,9 +66,6 @@ public static native long initRecordBatchReader( */ 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 * @@ -255,8 +77,6 @@ public static native long initRecordBatchReader( 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 * diff --git a/spark/src/main/java/org/apache/comet/parquet/NativeBatchReader.java b/spark/src/main/java/org/apache/comet/parquet/NativeBatchReader.java deleted file mode 100644 index a2ee4963d9..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/NativeColumnReader.java b/spark/src/main/java/org/apache/comet/parquet/NativeColumnReader.java deleted file mode 100644 index b170ae5830..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java b/spark/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java deleted file mode 100644 index 95fed362d3..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ParquetMetadataSerializer.java b/spark/src/main/java/org/apache/comet/parquet/ParquetMetadataSerializer.java deleted file mode 100644 index 32b40940a6..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/ReadOptions.java b/spark/src/main/java/org/apache/comet/parquet/ReadOptions.java deleted file mode 100644 index ec5c16ce8e..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/RowGroupFilter.java b/spark/src/main/java/org/apache/comet/parquet/RowGroupFilter.java deleted file mode 100644 index 60bbc4d19c..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/RowGroupReader.java b/spark/src/main/java/org/apache/comet/parquet/RowGroupReader.java deleted file mode 100644 index 0ca7478b7b..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/TypeUtil.java b/spark/src/main/java/org/apache/comet/parquet/TypeUtil.java deleted file mode 100644 index eaa1ecb3ce..0000000000 --- a/spark/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 = CometConf.COMET_SCHEMA_EVOLUTION_ENABLED(); - - 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) { - // INT32 -> LONG widening is allowed when Comet's per-Spark-version - // type-promotion default permits it (Spark 4.x). See ShimCometConf. - 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; - // FLOAT -> DOUBLE widening is allowed when Comet's per-Spark-version - // type-promotion default permits it (Spark 4.x). See ShimCometConf. - 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/spark/src/main/java/org/apache/comet/parquet/Utils.java b/spark/src/main/java/org/apache/comet/parquet/Utils.java deleted file mode 100644 index 87845ae760..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/WrappedInputFile.java b/spark/src/main/java/org/apache/comet/parquet/WrappedInputFile.java deleted file mode 100644 index 9b5e50ddb4..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/parquet/WrappedSeekableInputStream.java b/spark/src/main/java/org/apache/comet/parquet/WrappedSeekableInputStream.java deleted file mode 100644 index c463617bd6..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/vector/CometLazyVector.java b/spark/src/main/java/org/apache/comet/vector/CometLazyVector.java deleted file mode 100644 index 17b8d7e712..0000000000 --- a/spark/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/spark/src/main/java/org/apache/comet/vector/CometVector.java b/spark/src/main/java/org/apache/comet/vector/CometVector.java index 6dda765d51..f922f2281c 100644 --- a/spark/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/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/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 3c79bf7e9e..c4fad7dde3 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -257,9 +257,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 + // 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) + 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 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 4efd35fb3e..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,24 +30,21 @@ 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.MetricsSupport -import org.apache.comet.parquet.CometParquetFileFormat - /** - * Comet physical scan node for DataSource V1. Most of the code here follows Spark's - * [[FileSourceScanExec]]. After CometScanRule runs, this node is replaced by a fully native scan - * by CometExecRule; it does not survive to 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( @transient relation: HadoopFsRelation, @@ -182,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 @@ -217,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( @@ -267,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 @@ -442,64 +379,6 @@ 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( relation, @@ -520,41 +399,17 @@ case class CometScanExec( object CometScanExec { def apply(scanExec: FileSourceScanExec, session: SparkSession): 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] val batchScanExec = CometScanExec( - 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/spark/src/test/java/org/apache/comet/parquet/TestCometInputFile.java b/spark/src/test/java/org/apache/comet/parquet/TestCometInputFile.java deleted file mode 100644 index 63bb65d5c8..0000000000 --- a/spark/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/spark/src/test/java/org/apache/comet/parquet/TestFileReader.java b/spark/src/test/java/org/apache/comet/parquet/TestFileReader.java deleted file mode 100644 index d380fc16a9..0000000000 --- a/spark/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/spark/src/test/java/org/apache/comet/parquet/TestUtils.java b/spark/src/test/java/org/apache/comet/parquet/TestUtils.java deleted file mode 100644 index 6daa9a254b..0000000000 --- a/spark/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/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala index fd6d3ef535..51678c3591 100644 --- a/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala @@ -60,11 +60,6 @@ class CometNativeSuite extends CometTestBase { 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") { From 4bf543517e7c2a86173b8157c54f1bf96a894108 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 19:54:17 -0600 Subject: [PATCH 06/14] chore: remove dead Rust column-reader code The JVM-side Parquet column readers were deleted in the previous commit; the Rust JNI methods and supporting modules they called are now dead. - parquet/mod.rs: drop JNI methods initColumnReader, setDictionaryPage, setPageV1, setPageV2, resetBatch, readBatch, skipBatch, currentBatch, closeColumnReader, plus the Context/get_reader helpers. - Delete parquet/read/ (column, levels, values, mod), mutable_vector.rs, data_type.rs, and the util sub-modules bit_packing, buffer, memory, test_common that only existed for the column readers. - Trim parquet/util/jni.rs to deserialize_schema (the only helper used by the surviving Arrow-native scan path). - Delete native/core/src/common/ entirely; bit.rs and buffer.rs were only consumed by the deleted parquet column-reader chain. - Drop the parquet_read, bit_util, and parquet_decode benches that exercised the removed code; deregister them from Cargo.toml. --- native/core/Cargo.toml | 12 - native/core/benches/bit_util.rs | 216 - native/core/benches/parquet_decode.rs | 56 - native/core/benches/parquet_read.rs | 216 - native/core/src/common/bit.rs | 1602 -------- native/core/src/common/buffer.rs | 366 -- native/core/src/common/mod.rs | 21 - native/core/src/lib.rs | 2 - native/core/src/parquet/data_type.rs | 116 - native/core/src/parquet/mod.rs | 280 +- native/core/src/parquet/mutable_vector.rs | 243 -- native/core/src/parquet/read/column.rs | 839 ---- native/core/src/parquet/read/levels.rs | 229 -- native/core/src/parquet/read/mod.rs | 108 - native/core/src/parquet/read/values.rs | 1131 ----- native/core/src/parquet/util/bit_packing.rs | 3658 ----------------- native/core/src/parquet/util/buffer.rs | 128 - native/core/src/parquet/util/jni.rs | 228 - native/core/src/parquet/util/memory.rs | 557 --- native/core/src/parquet/util/mod.rs | 7 - .../core/src/parquet/util/test_common/mod.rs | 23 - .../src/parquet/util/test_common/page_util.rs | 309 -- .../src/parquet/util/test_common/rand_gen.rs | 57 - 23 files changed, 6 insertions(+), 10398 deletions(-) delete mode 100644 native/core/benches/bit_util.rs delete mode 100644 native/core/benches/parquet_decode.rs delete mode 100644 native/core/benches/parquet_read.rs delete mode 100644 native/core/src/common/bit.rs delete mode 100644 native/core/src/common/buffer.rs delete mode 100644 native/core/src/common/mod.rs delete mode 100644 native/core/src/parquet/data_type.rs delete mode 100644 native/core/src/parquet/mutable_vector.rs delete mode 100644 native/core/src/parquet/read/column.rs delete mode 100644 native/core/src/parquet/read/levels.rs delete mode 100644 native/core/src/parquet/read/mod.rs delete mode 100644 native/core/src/parquet/read/values.rs delete mode 100644 native/core/src/parquet/util/bit_packing.rs delete mode 100644 native/core/src/parquet/util/buffer.rs delete mode 100644 native/core/src/parquet/util/memory.rs delete mode 100644 native/core/src/parquet/util/test_common/mod.rs delete mode 100644 native/core/src/parquet/util/test_common/page_util.rs delete mode 100644 native/core/src/parquet/util/test_common/rand_gen.rs diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 4fb3ed4c5d..c58d446917 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -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/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/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 c522b3a14e..975c61b503 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 { @@ -460,10 +192,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)?; 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/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/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 c90630d6a3..0000000000 --- a/native/core/src/parquet/util/test_common/page_util.rs +++ /dev/null @@ -1,309 +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::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 mut level_encoder = LevelEncoder::v1_streaming(max_level); - level_encoder.put_with_observer(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)); - } -} From 7f7c414ab0504aab5186ea9a6f8d5348baecb785 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 20:03:06 -0600 Subject: [PATCH 07/14] fix: lint --- .../src/main/scala/org/apache/comet/rules/CometScanRule.scala | 4 ++-- .../spark/sql/benchmark/CometArrayExpressionBenchmark.scala | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) 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 835925904b..6895729cc5 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -208,7 +208,7 @@ case class CometScanRule(session: SparkSession) !COMET_SCAN_ALLOW_DISABLED_PARQUET_VECTORIZED_READER.get()) { withInfo( scanExec, - s"Native Parquet scan is incompatible with " + + "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 @@ -700,7 +700,7 @@ case class CometScanTypeChecker() extends DataTypeSupport with CometTypeShim { fallbackReasons: ListBuffer[String]): Boolean = { dt match { case ShortType if CometConf.COMET_PARQUET_UNSIGNED_SMALL_INT_CHECK.get() => - fallbackReasons += s"Native Parquet scan may not handle unsigned UINT_8 correctly for " + + 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 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 b674906bae..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: From 4c62d359d03ec7489a2d683f6a79b8b2378eecd3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 22:00:33 -0600 Subject: [PATCH 08/14] chore: update Spark diffs after scan-impl constant removal Updates the Spark test diffs to compile against the trimmed CometConf API and the new 10-arg CometScanExec signature: - Drop references to COMET_NATIVE_SCAN_IMPL, SCAN_NATIVE_DATAFUSION, SCAN_NATIVE_ICEBERG_COMPAT, and SCAN_AUTO in patched Spark sources. - Fix the CometScanExec extractor pattern in SubquerySuite to use 10 placeholders now that scanImpl is gone. - Collapse the SQLTestUtils per-impl branching to a single check for IgnoreCometNativeDataFusion / IgnoreCometNativeScan, since native_datafusion is the only Parquet scan impl. - Remove the unused IgnoreCometNativeIcebergCompat tag. --- dev/diffs/3.4.3.diff | 54 ++++++++-------------------- dev/diffs/3.5.8.diff | 86 +++++++++++++------------------------------- dev/diffs/4.0.2.diff | 78 +++++++++++----------------------------- dev/diffs/4.1.1.diff | 78 +++++++++++----------------------------- 4 files changed, 80 insertions(+), 216 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 79a945add3..925ae32132 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -608,10 +608,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..c528360742a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -640,7 +640,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") + @@ -1040,7 +1039,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 +1060,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( @@ -2864,18 +2863,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..cd18a23d4de 100644 +index dd55fcfe42c..d4f94b1d608 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._ @@ -2883,7 +2874,7 @@ index dd55fcfe42c..cd18a23d4de 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,17 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -2892,33 +2883,16 @@ index dd55fcfe42c..cd18a23d4de 100644 + 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 (isCometEnabled && ++ testTags.exists(t => ++ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", 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 +254,11 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2930,7 +2904,7 @@ index dd55fcfe42c..cd18a23d4de 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -434,6 +469,8 @@ private[sql] trait SQLTestUtilsBase +@@ -434,6 +451,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index a72e44fc4f..4ccabe6c88 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -503,18 +503,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 +514,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 +556,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 +564,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 +572,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 +580,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 +592,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..903e6b1342a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -636,7 +624,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") + @@ -992,7 +979,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 +1000,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( @@ -2824,18 +2811,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..3134078a122 100644 +index e937173a590..5fa7207a3d9 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._ @@ -2843,7 +2822,7 @@ index e937173a590..3134078a122 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,17 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -2852,33 +2831,16 @@ index e937173a590..3134078a122 100644 + 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 (isCometEnabled && ++ testTags.exists(t => ++ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", 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 +254,11 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2890,7 +2852,7 @@ index e937173a590..3134078a122 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +470,8 @@ private[sql] trait SQLTestUtilsBase +@@ -435,6 +452,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index 2614abc979..a7e0d7867b 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -739,10 +739,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..c528360742a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -771,7 +771,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") + @@ -1199,7 +1198,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 +1219,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( @@ -2773,18 +2772,10 @@ 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..f27ffc7ae1b 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 @@ -2792,7 +2783,7 @@ index 08fd8a9ecb5..27aee839b8c 100644 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,17 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { case f: FileSourceScanExec => numPartitions += f.inputRDD.partitions.length numOutputRows += f.metrics("numOutputRows").value @@ -2810,16 +2801,13 @@ index 08fd8a9ecb5..27aee839b8c 100644 case _ => } assert(numPartitions > 0) -@@ -303,6 +316,12 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -303,6 +315,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") @@ -3460,26 +3448,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..be5e113c3ed 100644 +index f0f3f94b811..f132dfc0f1d 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, IgnoreCometNativeDataFusion, IgnoreCometNativeScan, 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} @@ -3487,7 +3468,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,17 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -3496,33 +3477,16 @@ index f0f3f94b811..be5e113c3ed 100644 + 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 (isCometEnabled && ++ testTags.exists(t => ++ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", 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 +260,15 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3538,7 +3502,7 @@ index f0f3f94b811..be5e113c3ed 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +488,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +470,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child diff --git a/dev/diffs/4.1.1.diff b/dev/diffs/4.1.1.diff index 2ed8a5a32f..4b2b9fd192 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.diff @@ -800,10 +800,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..c528360742a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ +@@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -832,7 +832,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") + @@ -1292,7 +1291,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 +1312,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( @@ -2932,18 +2931,10 @@ 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..f27ffc7ae1b 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 @@ -2951,7 +2942,7 @@ index 08fd8a9ecb5..27aee839b8c 100644 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,17 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { case f: FileSourceScanExec => numPartitions += f.inputRDD.partitions.length numOutputRows += f.metrics("numOutputRows").value @@ -2969,16 +2960,13 @@ index 08fd8a9ecb5..27aee839b8c 100644 case _ => } assert(numPartitions > 0) -@@ -303,6 +316,12 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { +@@ -303,6 +315,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") @@ -3724,26 +3712,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..f132dfc0f1d 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, IgnoreCometNativeDataFusion, IgnoreCometNativeScan, 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} @@ -3751,7 +3732,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,17 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with override protected def test(testName: String, testTags: Tag*)(testFun: => Any) (implicit pos: Position): Unit = { @@ -3760,33 +3741,16 @@ index f0f3f94b811..be5e113c3ed 100644 + 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 (isCometEnabled && ++ testTags.exists(t => ++ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", 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 +260,15 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3802,7 +3766,7 @@ index f0f3f94b811..be5e113c3ed 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +488,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +470,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child From 7b725b57a96f73ec852e8a6377698e2ec4a40d50 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 18 May 2026 22:08:10 -0600 Subject: [PATCH 09/14] chore: collapse IgnoreCometNativeDataFusion/Scan into IgnoreComet With native_datafusion as the only Parquet scan implementation, the per-impl tag variants are equivalent to the plain IgnoreComet tag. - Replace all IgnoreCometNativeDataFusion and IgnoreCometNativeScan call sites with IgnoreComet. - Drop the corresponding case-class definitions from IgnoreComet.scala. - Remove the now-redundant secondary check in SQLTestUtils; the existing IgnoreComet check handles all usages. --- dev/diffs/3.4.3.diff | 50 +++++++++++++++++++------------------------- dev/diffs/3.5.8.diff | 46 +++++++++++++++++----------------------- dev/diffs/4.0.2.diff | 40 ++++++++++++++--------------------- dev/diffs/4.1.1.diff | 40 ++++++++++++++--------------------- 4 files changed, 72 insertions(+), 104 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 925ae32132..e8ca1b6ac8 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -608,10 +608,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..c528360742a +index 00000000000..4b31bea33de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,44 @@ +@@ -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,8 +640,6 @@ index 00000000000..c528360742a + * 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 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. @@ -1968,14 +1966,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..4adb273170a 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 @@ -1998,7 +1996,7 @@ index 104b4e416cd..4adb273170a 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", @@ -2034,7 +2032,7 @@ index 104b4e416cd..4adb273170a 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( @@ -2044,7 +2042,7 @@ index 104b4e416cd..4adb273170a 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) )) @@ -2089,7 +2087,7 @@ index 104b4e416cd..4adb273170a 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") @@ -2134,14 +2132,14 @@ index 8670d95c65e..b624c3811dd 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..dcb8a0e9bef 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 @@ -2151,7 +2149,7 @@ index 29cb224c878..dcb8a0e9bef 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) @@ -2161,7 +2159,7 @@ index 29cb224c878..dcb8a0e9bef 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. @@ -2171,7 +2169,7 @@ index 29cb224c878..dcb8a0e9bef 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/4354")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4354")) { def readParquet(schema: String, path: File): DataFrame = { spark.read.schema(schema).parquet(path.toString) } @@ -2201,7 +2199,7 @@ index 29cb224c878..dcb8a0e9bef 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/4354")) { ++ 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 @@ -2290,14 +2288,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..7e143a0e0f9 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 @@ -2317,7 +2315,7 @@ index bf5c51b89bb..7e143a0e0f9 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/4316")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SparkException]) @@ -2863,7 +2861,7 @@ 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..d4f94b1d608 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 @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -2874,7 +2872,7 @@ index dd55fcfe42c..d4f94b1d608 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 +120,17 @@ 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 = { @@ -2882,17 +2880,11 @@ index dd55fcfe42c..d4f94b1d608 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled && -+ testTags.exists(t => -+ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -242,6 +254,11 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +248,11 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2904,7 +2896,7 @@ index dd55fcfe42c..d4f94b1d608 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -434,6 +451,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 diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 4ccabe6c88..edaa2af45d 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -592,10 +592,10 @@ index 93275487f29..a5208b8d54b 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..903e6b1342a +index 00000000000..a42533c7c84 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,44 @@ +@@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with @@ -624,8 +624,6 @@ index 00000000000..903e6b1342a + * 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 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. @@ -1945,7 +1943,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..097c518a19a 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 @@ -1967,7 +1965,7 @@ index 8e88049f51e..097c518a19a 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", @@ -2002,7 +2000,7 @@ index 8e88049f51e..097c518a19a 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( @@ -2012,7 +2010,7 @@ index 8e88049f51e..097c518a19a 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) )) @@ -2060,7 +2058,7 @@ index 8e88049f51e..097c518a19a 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") @@ -2105,7 +2103,7 @@ index 8ed9ef1630e..eed2a6f5ad5 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..b62ff2975e6 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 @@ -2114,7 +2112,7 @@ index f6472ba3d9d..b62ff2975e6 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) @@ -2124,7 +2122,7 @@ index f6472ba3d9d..b62ff2975e6 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. @@ -2134,7 +2132,7 @@ index f6472ba3d9d..b62ff2975e6 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/4354")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4354")) { def readParquet(schema: String, path: File): DataFrame = { spark.read.schema(schema).parquet(path.toString) } @@ -2164,7 +2162,7 @@ index f6472ba3d9d..b62ff2975e6 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/4354")) { ++ 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 @@ -2253,14 +2251,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..80b7ef6c46a 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 @@ -2280,7 +2278,7 @@ index 3f47c5e506f..80b7ef6c46a 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/4316")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SparkException]) @@ -2811,7 +2809,7 @@ 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..5fa7207a3d9 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 @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -2822,7 +2820,7 @@ index e937173a590..5fa7207a3d9 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 +120,17 @@ 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 = { @@ -2830,17 +2828,11 @@ index e937173a590..5fa7207a3d9 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled && -+ testTags.exists(t => -+ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -242,6 +254,11 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +248,11 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2852,7 +2844,7 @@ index e937173a590..5fa7207a3d9 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +452,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 diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index a7e0d7867b..6da20cb802 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -739,10 +739,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..c528360742a +index 00000000000..4b31bea33de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,44 @@ +@@ -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,8 +771,6 @@ index 00000000000..c528360742a + * 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 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. @@ -2543,14 +2541,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..f5dadef89ae 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 @@ -2573,7 +2571,7 @@ index 6080a5e8e4b..f5dadef89ae 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", @@ -2608,7 +2606,7 @@ index 6080a5e8e4b..f5dadef89ae 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( @@ -2618,7 +2616,7 @@ index 6080a5e8e4b..f5dadef89ae 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) )) @@ -2665,7 +2663,7 @@ index 6080a5e8e4b..f5dadef89ae 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") @@ -2832,7 +2830,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..e7c65429119 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 @@ -2840,7 +2838,7 @@ index 0acb21f3e6f..e7c65429119 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 @@ -2860,7 +2858,7 @@ index 0acb21f3e6f..e7c65429119 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/4316")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) @@ -3448,7 +3446,7 @@ 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..f132dfc0f1d 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 @@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, Suite, Tag} @@ -3456,7 +3454,7 @@ index f0f3f94b811..f132dfc0f1d 100644 import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, 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 @@ -3468,7 +3466,7 @@ index f0f3f94b811..f132dfc0f1d 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 +122,17 @@ 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 = { @@ -3476,17 +3474,11 @@ index f0f3f94b811..f132dfc0f1d 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled && -+ testTags.exists(t => -+ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -248,8 +260,15 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +254,15 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3502,7 +3494,7 @@ index f0f3f94b811..f132dfc0f1d 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +470,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/diffs/4.1.1.diff b/dev/diffs/4.1.1.diff index 4b2b9fd192..96751c0960 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.diff @@ -800,10 +800,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..c528360742a +index 00000000000..4b31bea33de --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,44 @@ +@@ -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,8 +832,6 @@ index 00000000000..c528360742a + * 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 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. @@ -2684,14 +2682,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 @@ -2714,7 +2712,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", @@ -2749,7 +2747,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( @@ -2759,7 +2757,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) )) @@ -2806,7 +2804,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") @@ -2991,7 +2989,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..5872d9962cc 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 @@ -2999,7 +2997,7 @@ index 56076175d60..5872d9962cc 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 @@ -3019,7 +3017,7 @@ index 56076175d60..5872d9962cc 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/4316")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/4316")) { withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) assert(e.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) @@ -3712,7 +3710,7 @@ 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..f132dfc0f1d 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 @@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, Suite, Tag} @@ -3720,7 +3718,7 @@ index f0f3f94b811..f132dfc0f1d 100644 import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{AnalysisException, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, IgnoreCometNativeDataFusion, 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 @@ -3732,7 +3730,7 @@ index f0f3f94b811..f132dfc0f1d 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 +122,17 @@ 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 = { @@ -3740,17 +3738,11 @@ index f0f3f94b811..f132dfc0f1d 100644 + if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { + ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) + return -+ } -+ if (isCometEnabled && -+ testTags.exists(t => -+ t.isInstanceOf[IgnoreCometNativeDataFusion] || t.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return + } if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -248,8 +260,15 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +254,15 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3766,7 +3758,7 @@ index f0f3f94b811..f132dfc0f1d 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +470,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 From 20cdc4796ccaf1f6a40c1d909c21dd434f958514 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 19 May 2026 07:42:00 -0600 Subject: [PATCH 10/14] chore: drop CometScanExec case from ParquetRowIndexSuite Spark diffs CometScanExec is a planning-only intermediate after the native_iceberg_compat removal, so it never reaches the executed plan inspected by ParquetRowIndexSuite. Its inputRDD lazy val is also gone, breaking the sql_core/sql_hive test compile. --- dev/diffs/3.4.3.diff | 9 +++------ dev/diffs/3.5.8.diff | 9 +++------ dev/diffs/4.0.2.diff | 11 ++++------- dev/diffs/4.1.1.diff | 11 ++++------- 4 files changed, 14 insertions(+), 26 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index e8ca1b6ac8..7f9ce51ed2 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2238,24 +2238,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 diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index edaa2af45d..a86fa2a914 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -2201,24 +2201,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 diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index 6da20cb802..2d0d5cca35 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -2770,24 +2770,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..f27ffc7ae1b 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 @@ -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 +246,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 @@ -2799,7 +2796,7 @@ index 08fd8a9ecb5..f27ffc7ae1b 100644 case _ => } assert(numPartitions > 0) -@@ -303,6 +315,9 @@ 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}") { diff --git a/dev/diffs/4.1.1.diff b/dev/diffs/4.1.1.diff index 96751c0960..960e421da9 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.diff @@ -2929,24 +2929,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..f27ffc7ae1b 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 @@ -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 +246,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 @@ -2958,7 +2955,7 @@ index 08fd8a9ecb5..f27ffc7ae1b 100644 case _ => } assert(numPartitions > 0) -@@ -303,6 +315,9 @@ 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}") { From 2a02f3e2b2ce969abe38c5e2732986b455e152b6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 19 May 2026 08:16:01 -0600 Subject: [PATCH 11/14] remove more unused methods --- .../java/org/apache/comet/parquet/Native.java | 51 ------------------- .../org/apache/comet/CometNativeSuite.scala | 8 --- 2 files changed, 59 deletions(-) diff --git a/spark/src/main/java/org/apache/comet/parquet/Native.java b/spark/src/main/java/org/apache/comet/parquet/Native.java index 13762ec11f..c412c4fa5f 100644 --- a/spark/src/main/java/org/apache/comet/parquet/Native.java +++ b/spark/src/main/java/org/apache/comet/parquet/Native.java @@ -25,8 +25,6 @@ public final class Native extends NativeBase { - ///////////// Arrow Native Parquet Reader APIs - /** * Verify that object store options are valid. An exception will be thrown if the provided options * are not valid. @@ -34,53 +32,4 @@ public final class Native extends NativeBase { 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); - - /** - * 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); - - /** - * 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); - - /** - * Close the record batch reader. Free the resources - * - * @param handle - */ - public static native void closeRecordBatchReader(long handle); } diff --git a/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala index 51678c3591..9c34b3a3ce 100644 --- a/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometNativeSuite.scala @@ -54,14 +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") - } - 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 From 1cc640964b55fddb62048e46a510693e858832ec Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 19 May 2026 08:20:07 -0600 Subject: [PATCH 12/14] spotless --- spark/src/main/java/org/apache/comet/parquet/Native.java | 1 - 1 file changed, 1 deletion(-) diff --git a/spark/src/main/java/org/apache/comet/parquet/Native.java b/spark/src/main/java/org/apache/comet/parquet/Native.java index c412c4fa5f..8bf82dd5ed 100644 --- a/spark/src/main/java/org/apache/comet/parquet/Native.java +++ b/spark/src/main/java/org/apache/comet/parquet/Native.java @@ -31,5 +31,4 @@ public final class Native extends NativeBase { */ public static native void validateObjectStoreConfig( String filePath, Map objectStoreOptions); - } From 257c31c9a7252111226ec3849f5a7fdd5d9dfa03 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 19 May 2026 08:48:54 -0600 Subject: [PATCH 13/14] chore: remove dead validateObjectStoreConfig path CometScanRule.validateObjectStoreConfig had no production caller; only NativeConfigSuite invoked it. Drop the Scala wrapper, its configValidity cache, the Native.java JNI declaration (which leaves the file empty), the backing Rust JNI impl, and the tests that exercised the wrapper. Keep the extractObjectStoreOptions tests since that utility is still used by the native scan paths. --- native/core/src/parquet/mod.rs | 25 ------- .../java/org/apache/comet/parquet/Native.java | 34 ---------- .../apache/comet/rules/CometScanRule.scala | 58 +--------------- .../comet/objectstore/NativeConfigSuite.scala | 68 +------------------ 4 files changed, 2 insertions(+), 183 deletions(-) delete mode 100644 spark/src/main/java/org/apache/comet/parquet/Native.java diff --git a/native/core/src/parquet/mod.rs b/native/core/src/parquet/mod.rs index 975c61b503..806800cd59 100644 --- a/native/core/src/parquet/mod.rs +++ b/native/core/src/parquet/mod.rs @@ -129,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] diff --git a/spark/src/main/java/org/apache/comet/parquet/Native.java b/spark/src/main/java/org/apache/comet/parquet/Native.java deleted file mode 100644 index 8bf82dd5ed..0000000000 --- a/spark/src/main/java/org/apache/comet/parquet/Native.java +++ /dev/null @@ -1,34 +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.NativeBase; - -public final class Native extends NativeBase { - - /** - * 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); -} 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 6895729cc5..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} @@ -735,61 +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 = s"Object store config not supported: ${e.getMessage}" - fallbackReasons += reason - configValidityMap.put(cacheKey, Some(reason)) - } - } - - } - /** * Single-pass validation of Iceberg FileScanTasks. * 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 - } } From 30b9d0639f31f6575b1492f21b31b18d875712fe Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 19 May 2026 08:49:01 -0600 Subject: [PATCH 14/14] chore: remove unused ParquetFilters and CometFileReaderThreadPool ParquetFilters and its helper SourceFilterSerde have no instantiation sites left after the scan-impl cleanups; CometReaderThreadPool / CometFileReaderThreadPool likewise has no callers. Drop all three. --- .../comet/parquet/CometReaderThreadPool.scala | 60 - .../apache/comet/parquet/ParquetFilters.scala | 1044 ----------------- .../comet/parquet/SourceFilterSerde.scala | 179 --- 3 files changed, 1283 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala delete mode 100644 spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala delete mode 100644 spark/src/main/scala/org/apache/comet/parquet/SourceFilterSerde.scala diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala b/spark/src/main/scala/org/apache/comet/parquet/CometReaderThreadPool.scala deleted file mode 100644 index 1759ea2765..0000000000 --- a/spark/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/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() - } - -}