-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Spark-3.2: Avoid duplicate computation of ALL_MANIFESTS metadata table for spark actions #4674
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -42,6 +42,7 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import static org.apache.iceberg.MetadataTableType.ALL_MANIFESTS; | ||
| import static org.apache.iceberg.TableProperties.GC_ENABLED; | ||
| import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; | ||
|
|
||
|
|
@@ -69,6 +70,11 @@ public void accept(String file) { | |
| private Consumer<String> deleteFunc = defaultDelete; | ||
| private ExecutorService deleteExecutorService = null; | ||
| private FileIO io = new HadoopFileIO(spark().sessionState().newHadoopConf()); | ||
| private Dataset<Row> allManifests = null; | ||
| private boolean useCaching = true; | ||
|
|
||
| private static final String USE_CACHING = "use-caching"; | ||
| private static final boolean USE_CACHING_DEFAULT = true; | ||
|
|
||
| public BaseDeleteReachableFilesSparkAction(SparkSession spark, String metadataFileLocation) { | ||
| super(spark); | ||
|
|
@@ -116,17 +122,27 @@ private Result doExecute() { | |
| Dataset<Row> reachableFileDF = buildReachableFileDF(metadata).distinct(); | ||
|
|
||
| boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, STREAM_RESULTS_DEFAULT); | ||
| Result result; | ||
| if (streamResults) { | ||
| return deleteFiles(reachableFileDF.toLocalIterator()); | ||
| result = deleteFiles(reachableFileDF.toLocalIterator()); | ||
| } else { | ||
| return deleteFiles(reachableFileDF.collectAsList().iterator()); | ||
| result = deleteFiles(reachableFileDF.collectAsList().iterator()); | ||
| } | ||
| if (useCaching) { | ||
| allManifests.unpersist(); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Needs to be in a finally block
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea if we go for this approach, can we do something like in https://github.com/apache/iceberg/blob/master/spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteManifestsSparkAction.java#L237. (withReusableDS) ? |
||
| } | ||
| return result; | ||
| } | ||
|
|
||
| private Dataset<Row> buildReachableFileDF(TableMetadata metadata) { | ||
| Table staticTable = newStaticTable(metadata, io); | ||
| return withFileType(buildValidContentFileDF(staticTable), CONTENT_FILE) | ||
| .union(withFileType(buildManifestFileDF(staticTable), MANIFEST)) | ||
| allManifests = loadMetadataTable(staticTable, ALL_MANIFESTS); | ||
| useCaching = PropertyUtil.propertyAsBoolean(options(), USE_CACHING, USE_CACHING_DEFAULT); | ||
| if (useCaching) { | ||
| allManifests.persist(); | ||
| } | ||
| return withFileType(buildValidContentFileDF(staticTable, allManifests), CONTENT_FILE) | ||
| .union(withFileType(buildManifestFileDF(allManifests), MANIFEST)) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. How does this change the performance? Don't we have to compute all manifests in both locations here? Or Does changing to an object let Spark cache the relation?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I thought compute of dataset will happen in the first action and the results are reused for both the steps.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @RussellSpitzer : I got what you meant now. I will fix by adding persist(), it will cache and reuse it. Finally it will avoid two time reading the manifest list.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fixed and verified the scanning by adding and checking logs.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ah yes I should have been more clear, I was referring to the fact that the dataset would be recomputed on both lines. The "loadMetadataTable" function should be very fast but the actual planning should be expensive and that would require a cache of some kind. I'm a little worried in general about persisting things since I want to make sure we clean up our caches asap.
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. One other thing to worry about here is what is the additional cost of the persist operation here. Running a persist is not free and we need to check for sure that doing this cache is cost effective. In my experience 3 uses of a persisted df is most of the time worth it, but 2 sometimes is not (Very much dependent on the computation leading up to the cached df)
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Because it involves the IO operation, it will definitely help when there are hundreds or thousands of snapshots.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @RussellSpitzer , which is better? a cache or manually calling dataset.collect() on that allManifest df and building new dataset on top of those collected rows and reusing it on both the location?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @ajantha-bhat The problem is that persisting does another IO serialization of what should be basically the same amount of information but hopefully in a more readable form. Persist by default is a Memory and Disk based cache. You really need to test it out to be sure. For Cache vs Collect. Cache is probably much better, pieces would be stored on executors and the IO would hopefully be mostly local, doing a Collect and and building a DF from it would essentially bring all data back to the driver serialize, then deserialize and send everything back out. This is always worse than cache/persist |
||
| .union(withFileType(buildManifestListDF(staticTable), MANIFEST_LIST)) | ||
| .union(withFileType(buildAllReachableOtherMetadataFileDF(staticTable), OTHERS)); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,6 @@ | |
| import java.util.function.Consumer; | ||
| import org.apache.iceberg.HasTableOperations; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.TableMetadata; | ||
| import org.apache.iceberg.TableOperations; | ||
| import org.apache.iceberg.actions.BaseExpireSnapshotsActionResult; | ||
| import org.apache.iceberg.actions.ExpireSnapshots; | ||
|
|
@@ -46,6 +45,7 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import static org.apache.iceberg.MetadataTableType.ALL_MANIFESTS; | ||
| import static org.apache.iceberg.TableProperties.GC_ENABLED; | ||
| import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; | ||
|
|
||
|
|
@@ -87,6 +87,12 @@ public void accept(String file) { | |
| private Consumer<String> deleteFunc = defaultDelete; | ||
| private ExecutorService deleteExecutorService = null; | ||
| private Dataset<Row> expiredFiles = null; | ||
| private Dataset<Row> allManifestsBefore = null; | ||
| private Dataset<Row> allManifestsAfter = null; | ||
| private boolean useCaching = true; | ||
|
|
||
| private static final String USE_CACHING = "use-caching"; | ||
| private static final boolean USE_CACHING_DEFAULT = true; | ||
|
|
||
| public BaseExpireSnapshotsSparkAction(SparkSession spark, Table table) { | ||
| super(spark); | ||
|
|
@@ -147,7 +153,13 @@ public BaseExpireSnapshotsSparkAction deleteWith(Consumer<String> newDeleteFunc) | |
| public Dataset<Row> expire() { | ||
| if (expiredFiles == null) { | ||
| // fetch metadata before expiration | ||
| Dataset<Row> originalFiles = buildValidFileDF(ops.current()); | ||
| Table staticTableBefore = newStaticTable(ops.current(), table.io()); | ||
| allManifestsBefore = loadMetadataTable(staticTableBefore, ALL_MANIFESTS); | ||
| useCaching = PropertyUtil.propertyAsBoolean(options(), USE_CACHING, USE_CACHING_DEFAULT); | ||
| if (useCaching) { | ||
| allManifestsBefore.persist(); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we would want to do the "memory only" cache here (and in all the other usages) ... but I'm not sure |
||
| } | ||
| Dataset<Row> originalFiles = buildValidFileDF(staticTableBefore, allManifestsBefore); | ||
|
|
||
| // perform expiration | ||
| org.apache.iceberg.ExpireSnapshots expireSnapshots = table.expireSnapshots().cleanExpiredFiles(false); | ||
|
|
@@ -166,7 +178,12 @@ public Dataset<Row> expire() { | |
| expireSnapshots.commit(); | ||
|
|
||
| // fetch metadata after expiration | ||
| Dataset<Row> validFiles = buildValidFileDF(ops.refresh()); | ||
| Table staticTableAfter = newStaticTable(ops.refresh(), table.io()); | ||
| allManifestsAfter = loadMetadataTable(staticTableAfter, ALL_MANIFESTS); | ||
| if (useCaching) { | ||
| allManifestsAfter.persist(); | ||
| } | ||
| Dataset<Row> validFiles = buildValidFileDF(staticTableAfter, allManifestsAfter); | ||
|
|
||
| // determine expired files | ||
| this.expiredFiles = originalFiles.except(validFiles); | ||
|
|
@@ -206,17 +223,22 @@ private String jobDesc() { | |
|
|
||
| private ExpireSnapshots.Result doExecute() { | ||
| boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, STREAM_RESULTS_DEFAULT); | ||
| BaseExpireSnapshotsActionResult result; | ||
| if (streamResults) { | ||
| return deleteFiles(expire().toLocalIterator()); | ||
| result = deleteFiles(expire().toLocalIterator()); | ||
| } else { | ||
| return deleteFiles(expire().collectAsList().iterator()); | ||
| result = deleteFiles(expire().collectAsList().iterator()); | ||
| } | ||
| if (useCaching) { | ||
| allManifestsBefore.unpersist(); | ||
| allManifestsAfter.unpersist(); | ||
| } | ||
| return result; | ||
| } | ||
|
|
||
| private Dataset<Row> buildValidFileDF(TableMetadata metadata) { | ||
| Table staticTable = newStaticTable(metadata, table.io()); | ||
| return withFileType(buildValidContentFileDF(staticTable), CONTENT_FILE) | ||
| .union(withFileType(buildManifestFileDF(staticTable), MANIFEST)) | ||
| private Dataset<Row> buildValidFileDF(Table staticTable, Dataset<Row> allManifests) { | ||
| return withFileType(buildValidContentFileDF(staticTable, allManifests), CONTENT_FILE) | ||
| .union(withFileType(buildManifestFileDF(allManifests), MANIFEST)) | ||
| .union(withFileType(buildManifestListDF(staticTable), MANIFEST_LIST)); | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -47,7 +47,8 @@ public class ExpireSnapshotsProcedure extends BaseProcedure { | |
| ProcedureParameter.optional("older_than", DataTypes.TimestampType), | ||
| ProcedureParameter.optional("retain_last", DataTypes.IntegerType), | ||
| ProcedureParameter.optional("max_concurrent_deletes", DataTypes.IntegerType), | ||
| ProcedureParameter.optional("stream_results", DataTypes.BooleanType) | ||
| ProcedureParameter.optional("stream_results", DataTypes.BooleanType), | ||
| ProcedureParameter.optional("use_caching", DataTypes.BooleanType) | ||
| }; | ||
|
|
||
| private static final StructType OUTPUT_TYPE = new StructType(new StructField[]{ | ||
|
|
@@ -86,6 +87,7 @@ public InternalRow[] call(InternalRow args) { | |
| Integer retainLastNum = args.isNullAt(2) ? null : args.getInt(2); | ||
| Integer maxConcurrentDeletes = args.isNullAt(3) ? null : args.getInt(3); | ||
| Boolean streamResult = args.isNullAt(4) ? null : args.getBoolean(4); | ||
| Boolean useCaching = args.isNullAt(5) ? null : args.getBoolean(5); | ||
|
|
||
| Preconditions.checkArgument(maxConcurrentDeletes == null || maxConcurrentDeletes > 0, | ||
| "max_concurrent_deletes should have value > 0, value: " + maxConcurrentDeletes); | ||
|
|
@@ -101,14 +103,18 @@ public InternalRow[] call(InternalRow args) { | |
| action.retainLast(retainLastNum); | ||
| } | ||
|
|
||
| if (maxConcurrentDeletes != null && maxConcurrentDeletes > 0) { | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. maxConcurrentDeletes > 0 is already checked in the
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's best to leave changes like this for a separate cleanup pr |
||
| if (maxConcurrentDeletes != null) { | ||
| action.executeDeleteWith(executorService(maxConcurrentDeletes, "expire-snapshots")); | ||
| } | ||
|
|
||
| if (streamResult != null) { | ||
| action.option(BaseExpireSnapshotsSparkAction.STREAM_RESULTS, Boolean.toString(streamResult)); | ||
| } | ||
|
|
||
| if (useCaching != null) { | ||
| action.option("use_caching", useCaching.toString()); | ||
| } | ||
|
|
||
| ExpireSnapshots.Result result = action.execute(); | ||
|
|
||
| return toOutputRows(result); | ||
|
|
||
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 needs to be in a finally block, otherwise if we have an error in the collect as list there is a possibility that we do not uncache. For services running these actions that would be a problem.