RussellSpitzer commented on a change in pull request #1397:
URL: https://github.com/apache/iceberg/pull/1397#discussion_r478703612
##########
File path: spark/src/main/java/org/apache/iceberg/actions/BaseAction.java
##########
@@ -94,9 +102,30 @@ protected String metadataTableName(String tableName,
MetadataTableType type) {
return buildValidDataFileDF(spark, table().toString());
}
+ private static class ReadManifest implements
FlatMapFunction<ManifestFileBean, String> {
+ private final Broadcast<FileIO> io;
+
+ ReadManifest(Broadcast<FileIO> io) {
+ this.io = io;
+ }
+
+ @Override
+ public Iterator<String> call(ManifestFileBean manifest) {
+ return new ClosingIterator<>(ManifestFiles.readPaths(manifest,
io.getValue()).iterator());
+ }
+ }
+
protected Dataset<Row> buildValidDataFileDF(SparkSession spark, String
tableName) {
- String allDataFilesMetadataTable = metadataTableName(tableName,
MetadataTableType.ALL_DATA_FILES);
- return
spark.read().format("iceberg").load(allDataFilesMetadataTable).select("file_path");
+ JavaSparkContext context = new JavaSparkContext(spark.sparkContext());
+ Broadcast<FileIO> ioBroadcast =
context.broadcast(SparkUtil.serializableFileIO(table()));
+ String allManifestsMetadataTable = metadataTableName(tableName,
MetadataTableType.ALL_MANIFESTS);
+
+ Dataset<ManifestFileBean> allManifests =
spark.read().format("iceberg").load(allManifestsMetadataTable)
+ .selectExpr("path", "length", "partition_spec_id as partitionSpecId",
"added_snapshot_id as addedSnapshotId")
+ .dropDuplicates("path")
+ .as(Encoders.bean(ManifestFileBean.class));
Review comment:
As we discussed a bit, it may make sense to log a warning here for users
with "adaptive query on" that they will lose the ability to control the
parallelism of the manifest read stage with that parameter enabled. I'm a
little torn on whether that is too technical a detail or whether it will trip
up lots of real users ...
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]