Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 9 additions & 3 deletions .github/workflows/pr_build_linux.yml
Original file line number Diff line number Diff line change
Expand Up @@ -97,9 +97,10 @@ jobs:
- name: "Spark 4.0, JDK 21"
java_version: "21"
maven_opts: "-Pspark-4.0"
# Spark 4.1 is intentionally absent: the lint job invokes -Psemanticdb,
# but semanticdb-scalac_2.13.17 is not yet published, so we cannot
# currently run scalafix against the spark-4.1 profile.
# Spark 4.1 and 4.2 are intentionally absent: the lint job invokes -Psemanticdb,
# but semanticdb-scalac for those Scala patch versions (2.13.17 / 2.13.18) is not
# yet published, so we cannot currently run scalafix against the spark-4.1 or
# spark-4.2 profiles.
fail-fast: false
steps:
- uses: runs-on/action@742bf56072eb4845a0f94b3394673e4903c90ff0 # v2.1.0
Expand Down Expand Up @@ -305,6 +306,11 @@ jobs:
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: |
Expand Down
6 changes: 6 additions & 0 deletions .github/workflows/pr_build_macos.yml
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,12 @@ jobs:
# runtime; the scala-2.13 profile would override it back to 2.13.16 and break.
maven_opts: "-Pspark-4.1"

- name: "Spark 4.2, JDK 17, Scala 2.13"
java_version: "17"
# The spark-4.2 profile pins Scala to 2.13.18 to match Spark 4.2.0-preview4's
# runtime; the scala-2.13 profile would override it back to 2.13.16 and break.
maven_opts: "-Pspark-4.2"

suite:
- name: "fuzz"
value: |
Expand Down
23 changes: 22 additions & 1 deletion spark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,28 @@ under the License.
</profile>
<profile>
<id>spark-4.2</id>
<!-- 4.2 preview profile is build-only; no Iceberg or Jetty test dependencies are wired up. -->
<dependencies>
<!-- iceberg-spark-runtime-4.2 is not yet published; reuse the 4.0 runtime -->
<dependency>
<groupId>org.apache.iceberg</groupId>
<artifactId>iceberg-spark-runtime-4.0_${scala.binary.version}</artifactId>
<version>1.10.0</version>
<scope>test</scope>
</dependency>
<!-- Jetty 11.x for Spark 4.2 (jakarta.servlet); matches Spark 4.2.0-preview4's jetty.version -->
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
<version>11.0.26</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-servlet</artifactId>
<version>11.0.26</version>
<scope>test</scope>
</dependency>
</dependencies>
</profile>
<profile>
<id>generate-docs</id>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,10 @@ object CometSparkSessionExtensions extends Logging {
org.apache.spark.SPARK_VERSION >= "4.1"
}

def isSpark42Plus: Boolean = {
org.apache.spark.SPARK_VERSION >= "4.2"
}

/**
* Whether we should override Spark memory configuration for Comet. This only returns true when
* Comet native execution is enabled and/or Comet shuffle is enabled and Comet doesn't use
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
CometNativeColumnarToRow
+- CometTakeOrderedAndProject
+- CometProject
+- CometBroadcastHashJoin
:- CometProject
: +- CometBroadcastHashJoin
: :- CometProject
: : +- CometBroadcastHashJoin
: : :- CometFilter
: : : +- CometHashAggregate
: : : +- CometExchange
: : : +- CometHashAggregate
: : : +- CometProject
: : : +- CometBroadcastHashJoin
: : : :- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.store_returns
: : : : +- CometSubqueryBroadcast
: : : : +- CometBroadcastExchange
: : : : +- CometProject
: : : : +- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- CometBroadcastExchange
: : : +- CometProject
: : : +- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : +- CometBroadcastExchange
: : +- CometFilter
: : +- CometHashAggregate
: : +- CometExchange
: : +- CometHashAggregate
: : +- CometHashAggregate
: : +- CometExchange
: : +- CometHashAggregate
: : +- CometProject
: : +- CometBroadcastHashJoin
: : :- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.store_returns
: : : +- ReusedSubquery
: : +- CometBroadcastExchange
: : +- CometProject
: : +- CometFilter
: : +- CometNativeScan parquet spark_catalog.default.date_dim
: +- CometBroadcastExchange
: +- CometProject
: +- CometFilter
: +- CometNativeScan parquet spark_catalog.default.store
+- CometBroadcastExchange
+- CometProject
+- CometFilter
+- CometNativeScan parquet spark_catalog.default.customer

Comet accelerated 47 out of 49 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet.
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
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.
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
TakeOrderedAndProject
+- HashAggregate
+- Exchange
+- HashAggregate
+- Project
+- BroadcastHashJoin
:- Project
: +- BroadcastHashJoin
: :- Project
: : +- Filter
: : +- BroadcastHashJoin
: : :- BroadcastHashJoin [COMET: Unsupported join type ExistenceJoin(exists#1)]
: : : :- CometNativeColumnarToRow
: : : : +- CometBroadcastHashJoin
: : : : :- CometFilter
: : : : : +- CometNativeScan parquet spark_catalog.default.customer
: : : : +- CometBroadcastExchange
: : : : +- CometProject
: : : : +- CometBroadcastHashJoin
: : : : :- CometNativeScan parquet spark_catalog.default.store_sales
: : : : : +- CometSubqueryBroadcast
: : : : : +- CometBroadcastExchange
: : : : : +- CometProject
: : : : : +- CometFilter
: : : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : : +- CometBroadcastExchange
: : : : +- CometProject
: : : : +- CometFilter
: : : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : : +- BroadcastExchange
: : : +- CometNativeColumnarToRow
: : : +- CometProject
: : : +- CometBroadcastHashJoin
: : : :- CometNativeScan parquet spark_catalog.default.web_sales
: : : : +- ReusedSubquery
: : : +- CometBroadcastExchange
: : : +- CometProject
: : : +- CometFilter
: : : +- CometNativeScan parquet spark_catalog.default.date_dim
: : +- BroadcastExchange
: : +- CometNativeColumnarToRow
: : +- CometProject
: : +- CometBroadcastHashJoin
: : :- CometNativeScan parquet spark_catalog.default.catalog_sales
: : : +- ReusedSubquery
: : +- CometBroadcastExchange
: : +- CometProject
: : +- CometFilter
: : +- CometNativeScan parquet spark_catalog.default.date_dim
: +- BroadcastExchange
: +- CometNativeColumnarToRow
: +- CometProject
: +- CometFilter
: +- CometNativeScan parquet spark_catalog.default.customer_address
+- BroadcastExchange
+- CometNativeColumnarToRow
+- CometProject
+- CometFilter
+- CometNativeScan parquet spark_catalog.default.customer_demographics

Comet accelerated 35 out of 54 eligible operators (64%). Final plan contains 5 transitions between Spark and Comet.
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
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.
Loading
Loading