Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Conversation

jshmchenxi
Copy link
Contributor

@jshmchenxi jshmchenxi commented Feb 10, 2025

When importing a partitioned Spark table into Iceberg via SparkTableUtil, any partition whose directory is missing on the underlying filesystem will cause the migration to fail:

Caused by: java.lang.RuntimeException: Unable to list files in partition: s3://bucket/table/partition=foo
	at org.apache.iceberg.data.TableMigrationUtil.listPartition(TableMigrationUtil.java:206)
	at org.apache.iceberg.spark.SparkTableUtil.listPartition(SparkTableUtil.java:309)
	at org.apache.iceberg.spark.SparkTableUtil.lambda$importSparkPartitions$37333fc7$1(SparkTableUtil.java:767)
	at org.apache.spark.sql.Dataset.$anonfun$flatMap$2(Dataset.scala:3484)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:486)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:492)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)
	at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:225)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$.$anonfun$prepareShuffleDependency$10(ShuffleExchangeExec.scala:375)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:893)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:893)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:367)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:331)
	at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:104)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:54)
	at org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:166)
	at org.apache.spark.scheduler.Task.run(Task.scala:141)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:620)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64)
	at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:94)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:623)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
	at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: java.io.FileNotFoundException: No such file or directory: s3://bucket/table/partition=foo
	at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3799)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3650)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.innerListStatus(S3AFileSystem.java:3373)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$null$22(S3AFileSystem.java:3344)
	at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$listStatus$23(S3AFileSystem.java:3343)
	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:547)
	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:528)
	at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:449)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2478)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2497)
	at org.apache.hadoop.fs.s3a.S3AFileSystem.listStatus(S3AFileSystem.java:3342)
	at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:2078)
	at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:2122)
	at org.apache.iceberg.data.TableMigrationUtil.listPartition(TableMigrationUtil.java:167)
	... 32 more

This typically happens when a partition directory has been deleted—either manually or by an automated cleanup—while its metadata still exists in the metastore.

Problem:

  • Migrating Spark tables to Iceberg crashes on partitions whose directories don’t exist.
  • Users must manually recreate directories or clean up metadata before retrying.

Solution:

  • Introduce a new boolean flag, ignoreMissingFiles, in the importSparkPartitions and importSparkTable methods.
  • When enabled, partitions whose locations cannot be listed will be skipped rather than causing a hard failure.

Next Steps:

Extend ignoreMissingFiles support to the Spark procedures snapshot and migrate in a follow-up PR.

@github-actions github-actions bot added the data label Feb 10, 2025
@jshmchenxi jshmchenxi changed the title Handle case where partition location is missing from the file system in TableMigrationUtil Data: Handle case where partition location is missing for TableMigrationUtil Feb 10, 2025
@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch 2 times, most recently from 5b7ff64 to 658010c Compare February 10, 2025 02:31
@manuzhang
Copy link
Member

@jshmchenxi Thanks for the fix. Can you add a test?

Copy link
Member

@RussellSpitzer RussellSpitzer left a comment

Choose a reason for hiding this comment

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

Looks good although I agree we need a test to check that this is working as expected.

@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch 2 times, most recently from 36e2b7b to f3c2e11 Compare February 11, 2025 06:52
@jshmchenxi
Copy link
Contributor Author

@manuzhang @RussellSpitzer Thanks for the suggestion! I've added test cases to cover this change.

@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch 2 times, most recently from 60ef66a to 23a1b11 Compare February 11, 2025 06:56
@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch from 23a1b11 to 6b96233 Compare February 11, 2025 07:04
@manuzhang
Copy link
Member

@jshmchenxi Can we add an end-to-end test in TestSnapshotTableAction?

@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch from 6b96233 to 2aacf49 Compare February 16, 2025 02:34
@github-actions github-actions bot added the spark label Feb 16, 2025
@jshmchenxi
Copy link
Contributor Author

@jshmchenxi Can we add an end-to-end test in TestSnapshotTableAction?

@manuzhang I've added the end-to-end test. Please take a look.

@jshmchenxi
Copy link
Contributor Author

Kindly ping @manuzhang @RussellSpitzer @stevenzwu

@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch 2 times, most recently from 5e58ba9 to e0c9f62 Compare February 21, 2025 01:31
@jshmchenxi
Copy link
Contributor Author

@RussellSpitzer Hi, does anything else need to be updated for this PR?

@jshmchenxi
Copy link
Contributor Author

Hi @manuzhang @RussellSpitzer @ebyhr, just checking in on this PR. All feedback from the previous rounds has been addressed, and I believe it’s ready for the final review. I know you’re busy, but I’d really appreciate it if you could take a look when you have some time. Thanks so much!

Copy link

This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Mar 31, 2025
@github-actions github-actions bot removed the stale label Apr 1, 2025
@jshmchenxi
Copy link
Contributor Author

@Fokko Hi! When you get a chance, could you help review this PR? It’s been open for a little while. Appreciate it!

@manuzhang
Copy link
Member

Let's wait for more time, since folks are busy with Iceberg Summit now.

@jshmchenxi
Copy link
Contributor Author

jshmchenxi commented Apr 22, 2025

Kindly pinging @RussellSpitzer @huaxingao — when you have a moment, could you please help review this change? Looking forward to getting it merged. Thanks in advance!

@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch from e0c9f62 to 30efd1c Compare April 22, 2025 15:52
@RussellSpitzer
Copy link
Member

Tests are currently failing, Could we get those fixed please?

.filter(FileStatus::isFile)
.collect(Collectors.toList());
List<FileStatus> fileStatus;
if (fs.exists(partitionDir)) {
Copy link
Member

Choose a reason for hiding this comment

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

I'm actually a little worried now that we are going to silently error out on a permissions issue here. Should we have this behavior behind a flag?

Copy link
Member

@RussellSpitzer RussellSpitzer Apr 22, 2025

Choose a reason for hiding this comment

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

We added a flag "checkDuplicateFiles" to Spark Table util.

I'm afraid we don't have a clean way of adding more parameters to this method method but i'm still not confident we should go from failure to warning for everyone all at once.

I think it may be best to have a flag in the caller so it looks something like

  try {
     listPartition(ex)
     } Catch (FileNotFound ...) {
     if (ignoreMissingFiles) 
        log.warn
      else
        throw FNF
     }
     ```

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That makes sense. Should we add the flag to the Spark migrate and snapshot procedures as well?

Copy link
Member

Choose a reason for hiding this comment

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

That makes sense to me, but you can do that in a followup if you don't want this PR to get too large

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 for the suggestion! I've added the new flag to callers in SparkTableUtil.

.filter(FileStatus::isFile)
.collect(Collectors.toList());
} else {
LOG.info(
Copy link
Member

Choose a reason for hiding this comment

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

This should probably be a "warn" at least

@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch 3 times, most recently from 00a7c87 to 084bc85 Compare May 2, 2025 09:52
@jshmchenxi jshmchenxi force-pushed the bugfix/spark-util-partition-location-nonexist branch from 084bc85 to 909f0c2 Compare May 2, 2025 11:55
@jshmchenxi jshmchenxi changed the title Data: Handle case where partition location is missing for TableMigrationUtil Spark: Add flag to handle missing files for importSparkTable May 2, 2025
@@ -20,6 +20,7 @@

import static org.apache.spark.sql.functions.col;

import java.io.FileNotFoundException;
Copy link
Member

Choose a reason for hiding this comment

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

Not a big deal since this change set is small, but usually we only do a change in one Spark version at a time, review and then backport.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Got it, I’ll follow this approach in the next PRs.

@@ -2170,6 +2173,99 @@ public void testTableWithInt96Timestamp() throws IOException {
}
}

@Test
public void testSparkTableWithMissingFilesFailure() throws IOException {
Copy link
Member

Choose a reason for hiding this comment

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

testImportSparkTableWith...

Copy link
Member

Choose a reason for hiding this comment

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

Just want to make sure we note what's going on

Copy link
Contributor Author

Choose a reason for hiding this comment

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

updated

Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b"));

Dataset<Row> inputDF = spark.createDataFrame(records, SimpleRecord.class);
inputDF.select("data", "id").write().mode("overwrite").insertInto("parquet_table");
Copy link
Member

Choose a reason for hiding this comment

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

minor question here, do we need to "select"?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

No, we don't. It was copied from other tests. I'll remove this and the name mapping part as well.

new org.apache.spark.sql.catalyst.TableIdentifier("parquet_table"),
table,
stagingLocation))
.hasMessageContaining("Unable to list files in partition")
Copy link
Member

Choose a reason for hiding this comment

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

we can also check that the missing string contains partitionLocaltionPath

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That makes sense!

Path partitionLocationPath = parquetTablePath.toPath().resolve("id=1234");
java.nio.file.Files.delete(partitionLocationPath);

NameMapping mapping = MappingUtil.create(table.schema());
Copy link
Member

Choose a reason for hiding this comment

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

I don't think you actually need the name mapping for this test

Copy link
Member

@RussellSpitzer RussellSpitzer left a comment

Choose a reason for hiding this comment

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

I have a few nits here, but I think we are pretty close! Let's clean those up and we can get this merged

@jshmchenxi
Copy link
Contributor Author

@RussellSpitzer Thanks for the review! I've pushed an update to address the comments.

@RussellSpitzer
Copy link
Member

Thanks @jshmchenxi for the PR! Thank you to @manuzhang , @rohanag12 and @ebyhr for reviews!

@RussellSpitzer RussellSpitzer merged commit af32a07 into apache:main May 5, 2025
42 checks passed
anuragmantri added a commit to anuragmantri/iceberg that referenced this pull request Jul 25, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants