-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-5049] HoodieCatalog supports the implementation of dropPartition #6991
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
Merged
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
eb8629f
[HUDI-5049] HoodieCatalog supports the implementation of dropPartition
SteNicholas 4d799a4
[HUDI-5049] HoodieHiveCatalog supports the implementation of dropPart…
SteNicholas 5c24459
[HUDI-5049] HoodieHiveCatalog supports the implementation of dropPart…
SteNicholas fe64bb4
[HUDI-5049] HoodieHiveCatalog supports the implementation of dropPart…
SteNicholas File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
101 changes: 101 additions & 0 deletions
101
...in/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,101 @@ | ||
| /* | ||
| * 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.hudi.table.action.commit; | ||
|
|
||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; | ||
| import org.apache.hudi.client.WriteStatus; | ||
| import org.apache.hudi.common.engine.HoodieEngineContext; | ||
| import org.apache.hudi.common.model.FileSlice; | ||
| import org.apache.hudi.common.model.HoodieRecordPayload; | ||
| import org.apache.hudi.common.model.WriteOperationType; | ||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
| import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; | ||
| import org.apache.hudi.common.util.HoodieTimer; | ||
| import org.apache.hudi.common.util.collection.Pair; | ||
| import org.apache.hudi.config.HoodieWriteConfig; | ||
| import org.apache.hudi.exception.HoodieDeletePartitionException; | ||
| import org.apache.hudi.table.HoodieTable; | ||
| import org.apache.hudi.table.WorkloadProfile; | ||
| import org.apache.hudi.table.WorkloadStat; | ||
| import org.apache.hudi.table.action.HoodieWriteMetadata; | ||
|
|
||
| import java.time.Duration; | ||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; | ||
| import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; | ||
|
|
||
| public class FlinkDeletePartitionCommitActionExecutor<T extends HoodieRecordPayload<T>> | ||
| extends FlinkInsertOverwriteCommitActionExecutor<T> { | ||
|
|
||
| private final List<String> partitions; | ||
|
|
||
| public FlinkDeletePartitionCommitActionExecutor(HoodieEngineContext context, | ||
| HoodieWriteConfig config, | ||
| HoodieTable<?, ?, ?, ?> table, | ||
| String instantTime, | ||
| List<String> partitions) { | ||
| super(context, null, config, table, instantTime, null, WriteOperationType.DELETE_PARTITION); | ||
| this.partitions = partitions; | ||
| } | ||
|
|
||
| @Override | ||
| public HoodieWriteMetadata<List<WriteStatus>> execute() { | ||
| try { | ||
| HoodieTimer timer = new HoodieTimer().startTimer(); | ||
| context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions."); | ||
| Map<String, List<String>> partitionToReplaceFileIds = | ||
| context.parallelize(partitions).distinct().collectAsList() | ||
| .stream().collect(Collectors.toMap(partitionPath -> partitionPath, this::getAllExistingFileIds)); | ||
| HoodieWriteMetadata<List<WriteStatus>> result = new HoodieWriteMetadata<>(); | ||
| result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); | ||
| result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); | ||
| result.setWriteStatuses(Collections.emptyList()); | ||
|
|
||
| // created requested | ||
| HoodieInstant dropPartitionsInstant = new HoodieInstant(REQUESTED, REPLACE_COMMIT_ACTION, instantTime); | ||
| if (!table.getMetaClient().getFs().exists(new Path(table.getMetaClient().getMetaPath(), | ||
| dropPartitionsInstant.getFileName()))) { | ||
| HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() | ||
| .setOperationType(WriteOperationType.DELETE_PARTITION.name()) | ||
| .setExtraMetadata(extraMetadata.orElse(Collections.emptyMap())) | ||
| .build(); | ||
| table.getMetaClient().getActiveTimeline().saveToPendingReplaceCommit(dropPartitionsInstant, | ||
| TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); | ||
| } | ||
|
|
||
| this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), | ||
| instantTime); | ||
| this.commitOnAutoCommit(result); | ||
| return result; | ||
| } catch (Exception e) { | ||
| throw new HoodieDeletePartitionException("Failed to drop partitions for commit time " + instantTime, e); | ||
| } | ||
| } | ||
|
|
||
| private List<String> getAllExistingFileIds(String partitionPath) { | ||
| // because new commit is not complete. it is safe to mark all existing file Ids as old files | ||
| return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList()); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Hi
May I ask why we need to do
fs.deletehere? Will it cause any problem?I refer the code in
HoodieSparkSqlWriter.scalaandSparkDeletePartitionCommitActionExecutor.java.There is no path deletion operation in spark side.
https://github.com/apache/hudi/blob/master/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala#L270
https://github.com/apache/hudi/blob/master/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java#L62
cc @voonhous
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.
@TengHuo, like
HoodieHiveCatalog, thedropPartitionoperation needs to drop the partition meta and directory on the filesystem. Otherwise, after thedropPartitionoperation, as you mentioned, this will cause the unvalid data files in the dropped partition if there is insert operation and no cleaner to clean the data files.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.
So, if I understand correctly, in spark side, there will be the invalid data files in the dropped partition if there is insert operation and no cleaner to clean the data files. Am I right?
And may I ask what the invalid data files issue is exactly? Do you have a ticket about it?
Voon and me are checking the code about
drop partitions, we may fix it if there is any.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.
@TengHuo, the invalid data files after dropping partition mean that the data files is dirty data. Meanwhile, why should the
dropPartitionbehavior keeps the consistency between Flink and Spark? If should, IMO, this should provider unified interface to drop partition.cc @voonhous