RussellSpitzer commented on code in PR #15239:
URL: https://github.com/apache/iceberg/pull/15239#discussion_r2848359084
##########
spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java:
##########
@@ -253,9 +255,15 @@ private static class InitialOffsetStore {
private final String initialOffsetLocation;
private final Long fromTimestamp;
- InitialOffsetStore(Table table, String checkpointLocation, Long
fromTimestamp) {
+ InitialOffsetStore(
+ Table table, String checkpointLocation, Long fromTimestamp,
JavaSparkContext sparkContext) {
this.table = table;
- this.io = table.io();
+ // Use ResolvingFileIO with empty properties to get defaults for the
checkpoint location
+ this.io =
+ CatalogUtil.loadFileIO(
+ "org.apache.iceberg.io.ResolvingFileIO",
Review Comment:
Using `Collections.emptyMap()` discards all user configuration. If the
checkpoint location is on S3 with a custom endpoint, custom credential
provider, or regional configuration, this won't pick any of that up. The empty
properties are the main concern here — this is essentially creating an
unconfigured FileIO and hoping the Hadoop config alone is enough.
Also, hard-coding `"org.apache.iceberg.io.ResolvingFileIO"` creates a tight
coupling to a specific implementation. `ResolvingFileIO` is currently not used
anywhere in the Spark module, and its fallback is `HadoopFileIO`, which means
this is effectively a Hadoop dependency in disguise.
I'd suggest either:
1. Using Hadoop `FileSystem` directly (since we're already calling
`sparkContext.hadoopConfiguration()` here anyway), or
2. If staying within FileIO, using `HadoopFileIO` directly — which is
simpler, more explicit, and what `ResolvingFileIO` would resolve to for most
checkpoint path schemes (local, hdfs).
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]