-
Notifications
You must be signed in to change notification settings - Fork 2.8k
Spark: Add flag to handle missing files for importSparkTable
#12212
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Spark: Add flag to handle missing files for importSparkTable
#12212
Conversation
TableMigrationUtil
TableMigrationUtil
5b7ff64
to
658010c
Compare
@jshmchenxi Thanks for the fix. Can you add a test? |
There was a problem hiding this 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.
36e2b7b
to
f3c2e11
Compare
@manuzhang @RussellSpitzer Thanks for the suggestion! I've added test cases to cover this change. |
60ef66a
to
23a1b11
Compare
data/src/test/java/org/apache/iceberg/data/TestTableMigrationUtil.java
Outdated
Show resolved
Hide resolved
data/src/test/java/org/apache/iceberg/data/TestTableMigrationUtil.java
Outdated
Show resolved
Hide resolved
data/src/test/java/org/apache/iceberg/data/TestTableMigrationUtil.java
Outdated
Show resolved
Hide resolved
data/src/test/java/org/apache/iceberg/data/TestTableMigrationUtil.java
Outdated
Show resolved
Hide resolved
data/src/test/java/org/apache/iceberg/data/TestTableMigrationUtil.java
Outdated
Show resolved
Hide resolved
23a1b11
to
6b96233
Compare
@jshmchenxi Can we add an end-to-end test in |
6b96233
to
2aacf49
Compare
@manuzhang I've added the end-to-end test. Please take a look. |
Kindly ping @manuzhang @RussellSpitzer @stevenzwu |
data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java
Outdated
Show resolved
Hide resolved
5e58ba9
to
e0c9f62
Compare
@RussellSpitzer Hi, does anything else need to be updated for this PR? |
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! |
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. |
@Fokko Hi! When you get a chance, could you help review this PR? It’s been open for a little while. Appreciate it! |
Let's wait for more time, since folks are busy with Iceberg Summit now. |
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! |
e0c9f62
to
30efd1c
Compare
Tests are currently failing, Could we get those fixed please? |
.filter(FileStatus::isFile) | ||
.collect(Collectors.toList()); | ||
List<FileStatus> fileStatus; | ||
if (fs.exists(partitionDir)) { |
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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
}
```
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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( |
There was a problem hiding this comment.
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
00a7c87
to
084bc85
Compare
084bc85
to
909f0c2
Compare
TableMigrationUtil
importSparkTable
@@ -20,6 +20,7 @@ | |||
|
|||
import static org.apache.spark.sql.functions.col; | |||
|
|||
import java.io.FileNotFoundException; |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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 { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
testImportSparkTableWith...
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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"); |
There was a problem hiding this comment.
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"?
There was a problem hiding this comment.
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") |
There was a problem hiding this comment.
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
There was a problem hiding this comment.
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()); |
There was a problem hiding this comment.
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
There was a problem hiding this 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
@RussellSpitzer Thanks for the review! I've pushed an update to address the comments. |
Thanks @jshmchenxi for the PR! Thank you to @manuzhang , @rohanag12 and @ebyhr for reviews! |
…rkTable` (apache#12212) (apache#1591) Co-authored-by: Xi Chen <[email protected]>
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: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:
Solution:
ignoreMissingFiles
, in theimportSparkPartitions
andimportSparkTable
methods.Next Steps:
Extend
ignoreMissingFiles
support to the Spark proceduressnapshot
andmigrate
in a follow-up PR.