-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Spark: Refactor action for expiring snapshots #2314
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 |
|---|---|---|
| @@ -0,0 +1,50 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
|
|
||
| package org.apache.iceberg.actions; | ||
|
|
||
| public class BaseExpireSnapshotsActionResult implements ExpireSnapshots.Result { | ||
|
|
||
| private final long deletedDataFilesCount; | ||
|
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. Seems we don't consider the format v2 in this patch, right ? That's OK if true because we could support v2 in a separate issue. For the RewriteDataFilesAction, now @chenjunjiedada and I have prepared few PRs to support format v2's Rewrite actions.
Contributor
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. Yep, you are correct. I keep the scope of this PR to the existing logic. @RussellSpitzer is also working on redesigning the rewrite data files action to also support sort-based compactions and per partition compactions. Shall 4 of us meet next week to discuss this?
Contributor
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. It would be easy to extend the interface and this implementation to include other counts once we support them. |
||
| private final long deletedManifestsCount; | ||
| private final long deletedManifestListsCount; | ||
|
|
||
| public BaseExpireSnapshotsActionResult(long deletedDataFilesCount, | ||
| long deletedManifestsCount, | ||
| long deletedManifestListsCount) { | ||
| this.deletedDataFilesCount = deletedDataFilesCount; | ||
| this.deletedManifestsCount = deletedManifestsCount; | ||
| this.deletedManifestListsCount = deletedManifestListsCount; | ||
| } | ||
|
|
||
| @Override | ||
| public long deletedDataFilesCount() { | ||
| return deletedDataFilesCount; | ||
| } | ||
|
|
||
| @Override | ||
| public long deletedManifestsCount() { | ||
| return deletedManifestsCount; | ||
| } | ||
|
|
||
| @Override | ||
| public long deletedManifestListsCount() { | ||
| return deletedManifestListsCount; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.common.DynConstructors; | ||
| import org.apache.iceberg.common.DynMethods; | ||
| import org.apache.iceberg.spark.actions.BaseExpireSnapshotsSparkAction; | ||
| import org.apache.spark.sql.SparkSession; | ||
|
|
||
| public class Actions { | ||
|
|
@@ -79,7 +80,8 @@ public RewriteDataFilesAction rewriteDataFiles() { | |
| } | ||
|
|
||
| public ExpireSnapshotsAction expireSnapshots() { | ||
| return new ExpireSnapshotsAction(spark, table); | ||
| BaseExpireSnapshotsSparkAction delegate = new BaseExpireSnapshotsSparkAction(spark, table); | ||
| return new ExpireSnapshotsAction(delegate); | ||
|
Contributor
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. Are we still use ExpireSnapshotsAction since it is deprecated in this PR?
Contributor
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. This is a user-facing API so we cannot break it without deprecating first.
Contributor
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. Not sure about the Iceberg deprecating process. Here is my understanding, we need to create a new method with new return type for this user-facing API, which can be used by user moving forward. Meanwhile we mark this method deprecated so that user won't use it anymore.
Contributor
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. Oh, yeah, we will mark |
||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
|
|
||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.function.Supplier; | ||
| import org.apache.iceberg.BaseTable; | ||
|
|
@@ -36,6 +37,7 @@ | |
| import org.apache.iceberg.io.FileIO; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Maps; | ||
| import org.apache.iceberg.spark.JobGroupInfo; | ||
| import org.apache.iceberg.spark.JobGroupUtils; | ||
| import org.apache.iceberg.spark.SparkUtil; | ||
|
|
@@ -51,12 +53,13 @@ | |
|
|
||
| import static org.apache.iceberg.MetadataTableType.ALL_MANIFESTS; | ||
|
|
||
| abstract class BaseSparkAction<ThisT, R> implements Action<ThisT, R> { | ||
| public abstract class BaseSparkAction<ThisT, R> implements Action<ThisT, R> { | ||
|
Contributor
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. Do we need it to be public? Looks like there is no scope change needed in this PR.
Contributor
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. Temporarily yes since we are using it in another package. Once we get rid of the old actions, we should be able to move this and make it non-public.
Contributor
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. If this is public but will be removed, we should mark it deprecated so people know not to rely on it.
Contributor
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. This class will be used even after refactoring. It may be moved but to another package, though. |
||
|
|
||
| private static final AtomicInteger JOB_COUNTER = new AtomicInteger(); | ||
|
|
||
| private final SparkSession spark; | ||
| private final JavaSparkContext sparkContext; | ||
| private final Map<String, String> options = Maps.newHashMap(); | ||
|
|
||
| protected BaseSparkAction(SparkSession spark) { | ||
| this.spark = spark; | ||
|
|
@@ -71,6 +74,24 @@ protected JavaSparkContext sparkContext() { | |
| return sparkContext; | ||
| } | ||
|
|
||
| protected abstract ThisT self(); | ||
|
|
||
| @Override | ||
| public ThisT option(String name, String value) { | ||
| options.put(name, value); | ||
| return self(); | ||
| } | ||
|
|
||
| @Override | ||
| public ThisT options(Map<String, String> newOptions) { | ||
| options.putAll(newOptions); | ||
| return self(); | ||
| } | ||
|
|
||
| protected Map<String, String> options() { | ||
| return options; | ||
| } | ||
|
|
||
| protected <T> T withJobGroupInfo(JobGroupInfo info, Supplier<T> supplier) { | ||
| SparkContext context = spark().sparkContext(); | ||
| JobGroupInfo previousInfo = JobGroupUtils.getJobGroupInfo(context); | ||
|
|
||
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 grateful that the ExpireSnapshotsAction refactoring work has been considered to work for different compute engines, such as flink. In this way, we flink don't have to abstract the common logic of actions between flink and spark and we could just extend those BaseXXAction to implement our flink own actions. That's really important !
About the
ExpireSnapshotsAction, I think both flink & spark will have the sameResult. So maybe we could just use theExpireSnapshotsActionResultdirectly ( I don't think there will be a reason that flink or spark will extend thisBaseExpireSnapshotsActionResult, so maybe we could just remove theBaseprefix).Uh oh!
There was an error while loading. Please reload this page.
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.
Here, I followed the naming we have in
corelikeBaseFile,BaseTable,BaseRewriteManifests.Also, there is a class called
ExpireSnapshotsActionResultin this package in Spark already. We deprecate it with the introduction of this new one.I agree query engines will most likely use the same result classes (unless they are doing something really specific).
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.
What about making
ExpireSnapshots.Resulta class instead of an interface? Then we would always have the same default implementation.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 am fine converting
Resultinto a class but I am not sure we will gain much by doing that. The current class is in the core module so it is accessible to everyone. MakingResulta class may make inheritance a bit harder. I do like our current pattern of havingBaseXXXclasses in core and interfaces in the API module.