Skip to content

Conversation

@hsiang-c
Copy link
Contributor

@hsiang-c hsiang-c commented May 5, 2025

Which issue does this PR close?

Closes #. #1685

Rationale for this change

Run Iceberg Spark' tests as part of Comet CI

What changes are included in this PR?

How are these changes tested?

with:
repository: apache/iceberg
path: apache-iceberg
ref: apache-iceberg-${{inputs.iceberg-version}}
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Based on Iceberg's release tag: https://github.com/apache/iceberg/tags

@codecov-commenter
Copy link

codecov-commenter commented May 5, 2025

Codecov Report

All modified and coverable lines are covered by tests ✅

Project coverage is 58.79%. Comparing base (f09f8af) to head (738bebb).
Report is 276 commits behind head on main.

Additional details and impacted files
@@             Coverage Diff              @@
##               main    #1715      +/-   ##
============================================
+ Coverage     56.12%   58.79%   +2.66%     
- Complexity      976     1140     +164     
============================================
  Files           119      130      +11     
  Lines         11743    12836    +1093     
  Branches       2251     2415     +164     
============================================
+ Hits           6591     7547     +956     
- Misses         4012     4065      +53     
- Partials       1140     1224      +84     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.

<exclude>**/build/**</exclude>
<exclude>**/target/**</exclude>
<exclude>**/apache-spark/**</exclude>
<exclude>**/apache-iceberg/**</exclude>
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I forgot to exclude the iceberg repo

Copy link
Contributor

@kazuyukitanimura kazuyukitanimura left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @hsiang-c

run: |
cd apache-iceberg
rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups
ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_PARQUET_SCAN_IMPL=native_datafusion ./gradlew -DsparkVersions=${{ matrix.spark-version.short }} -DscalaVersion=${{ matrix.scala-version }} -DflinkVersions= -DkafkaVersions= \
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmmm do we expect native_datafusion works for iceberg?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Native execution doesn't work for iceberg yet. Iceberg uses ParquetReaderType to control whether to use Comet or not. The default is ParquetReaderType.ICEBERG. We need to set to ParquetReaderType.COMET to turn on Comet, but currently it's for Comet reader only, not for native execution yet.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @kazuyukitanimura @huaxingao, I will remove native_dafafusion and native_iceberg_compact builds for now.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but currently it's for Comet reader only, not for native execution yet.

Correct. I plan to work with @huaxingao once the Spark sql tests pass for native_iceberg_compat.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@parthchandra Please feel free to involve me, happy to help here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's great @hsiang-c ! We can discuss this offline.

// Controls which Parquet reader implementation to use
public static final String PARQUET_READER_TYPE = "spark.sql.iceberg.parquet.reader-type";
- public static final ParquetReaderType PARQUET_READER_TYPE_DEFAULT = ParquetReaderType.ICEBERG;
+ public static final ParquetReaderType PARQUET_READER_TYPE_DEFAULT = ParquetReaderType.COMET;
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@huaxingao I changed the default to COMET.

}

- @TestTemplate
+ @Disabled
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@huaxingao Disabled 2 unit tests b/c they fail with Comet reader.

integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts')
integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts')
integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts')
+ integrationImplementation project(path: ':iceberg-parquet')
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Only in Spark 3.4, I need to include iceberg-parquet otherwise the iceberg-spark-runtime-3.4 tests throw the following errors

org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 4) (17.115.161.202 executor driver): java.lang.NoSuchMethodError: 'org.apache.parquet.column.ParquetProperties$Builder org.apache.parquet.column.ParquetProperties$Builder.withBloomFilterFPP(java.lang.String, double)'
	at org.apache.iceberg.parquet.Parquet$WriteBuilder.build(Parquet.java:389)
	at org.apache.iceberg.parquet.Parquet$DataWriteBuilder.build(Parquet.java:787)
	at org.apache.iceberg.data.BaseFileWriterFactory.newDataWriter(BaseFileWriterFactory.java:131)
	at org.apache.iceberg.io.RollingDataWriter.newWriter(RollingDataWriter.java:52)
	at org.apache.iceberg.io.RollingDataWriter.newWriter(RollingDataWriter.java:32)
	at org.apache.iceberg.io.RollingFileWriter.openCurrentWriter(RollingFileWriter.java:108)
	at org.apache.iceberg.io.RollingDataWriter.<init>(RollingDataWriter.java:47)
	at org.apache.iceberg.spark.source.SparkWrite$UnpartitionedDataWriter.<init>(SparkWrite.java:701)
	at org.apache.iceberg.spark.source.SparkWrite$WriterFactory.createWriter(SparkWrite.java:675)
	at org.apache.iceberg.spark.source.SparkWrite$WriterFactory.createWriter(SparkWrite.java:652)
	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteToDataSourceV2Exec.scala:459)
	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteToDataSourceV2Exec.scala:448)
	at org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2Exec.scala:514)
	at org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.$anonfun$writeWithV2$2(WriteToDataSourceV2Exec.scala:411)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:92)
	at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)
	at org.apache.spark.scheduler.Task.run(Task.scala:139)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:554)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1529)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:557)

run: |
cd apache-iceberg
rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups
ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true ./gradlew -DsparkVersions=${{ matrix.spark-version.short }} -DscalaVersion=${{ matrix.scala-version }} -DflinkVersions= -DkafkaVersions= \
Copy link
Contributor

@kazuyukitanimura kazuyukitanimura May 8, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ENABLE_COMET is available only for the patched Spark (with the diff) through setup-spark-builder
Which Spark is combined with this Iceberg test?
Also I just realized ENABLE_COMET_SHUFFLE is not used at all looks like

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kazuyukitanimura You're right, I don't read both env vars with the diff I made.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we still need to update the Spark referred in Iceberg?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kazuyukitanimura

Sorry I don't get it.

Do you mean the build comamnd -DsparkVersions=${{ matrix.spark-version.short }} or in the diff?

In the diff, I modified the Iceberg Spark Gradle module according to the doc

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, the spark version provided by -DsparkVersions= is OSS Spark. do we need to let it load comet library?
Not sure if this test automatically load the Comet library in Spark referred by Iceberg...
@huaxingao ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@kazuyukitanimura @huaxingao Thanks for the feedback

  1. In the iceberg_spark_test workflow, I added an additional step to build a Spark Snapshot version w/ Comet enabled.
  2. In Iceberg Spark module, I made Iceberg depends on the Spark Snapshot version
  3. Finally, ENABLE_COMET=true is added to test Iceberg Spark module

./dev/change-scala-version.sh ${{inputs.scala-version}
./build/mvn versions:set -DnewVersion=${{inputs.spark-version}}-SNAPSHOT
# Might need to skip enforcer b/c comet is snapshot
./build/mvn -Pscala-${{inputs.scala-version}} -Phive -Phive-thriftserver -DskipTests -Denforcer.skip=true clean install
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Publish a SNAPSHOT version for Iceberg Spark module.

spark-hive33 = "3.3.4"
spark-hive34 = "3.4.4"
-spark-hive35 = "3.5.4"
+spark-hive35 = "3.5.6-SNAPSHOT"
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Leverage the local Spark SNAPSHOT version.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Iceberg 1.8.1 has mavenLocal() configured

Copy link
Contributor

@kazuyukitanimura kazuyukitanimura left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall looks good
Pending with CI

os: [ubuntu-24.04]
java-version: [11, 17]
iceberg-version: [{short: '1.8', full: '1.8.1'}]
spark-version: [{short: '3.5', full: '3.5.6'}]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We have iceberg diff for 3.4 but here we only have 3.5

@kazuyukitanimura kazuyukitanimura merged commit 4b8782a into apache:main Jun 24, 2025
83 of 84 checks passed
coderfender pushed a commit to coderfender/datafusion-comet that referenced this pull request Dec 13, 2025
## Which issue does this PR close?

Closes #. apache#1685

## Rationale for this change

Run Iceberg Spark' tests as part of Comet CI

## What changes are included in this PR?

 - Added a new action `setup-spark-local-jar/action.yaml` that locally publishes Spark w/ Comet jars
 - Produced a git diff for Iceberg based on this [instruction](https://github.com/apache/datafusion-comet/blob/main/docs/source/user-guide/iceberg.md) 
    - Make Iceberg Spark module depend on the local Spark SNAPSHOT jar
 - Target Iceberg version 1.8.1, Spark 3.5.6 and Scala 2.13
 - Run Iceberg Spark's tests, based on Iceberg's GitHub workflow: https://github.com/apache/iceberg/blob/main/.github/workflows/spark-ci.yml
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants