-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1984] Support independent flink hudi compaction function #3046
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
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
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
111 changes: 111 additions & 0 deletions
111
hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanSourceFunction.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,111 @@ | ||
| /* | ||
| * 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.sink.compact; | ||
|
|
||
| import org.apache.hudi.avro.model.HoodieCompactionPlan; | ||
| import org.apache.hudi.common.model.CompactionOperation; | ||
| import org.apache.hudi.common.table.timeline.HoodieInstant; | ||
| import org.apache.hudi.table.HoodieFlinkTable; | ||
|
|
||
| import org.apache.flink.api.common.functions.AbstractRichFunction; | ||
| import org.apache.flink.configuration.Configuration; | ||
| import org.apache.flink.streaming.api.functions.source.SourceFunction; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.util.List; | ||
|
|
||
| import static java.util.stream.Collectors.toList; | ||
|
|
||
| /** | ||
| * Flink hudi compaction source function. | ||
| * | ||
| * <P>This function read the compaction plan as {@link CompactionOperation}s then assign the compaction task | ||
| * event {@link CompactionPlanEvent} to downstream operators. | ||
| * | ||
| * <p>The compaction instant time is specified explicitly with strategies: | ||
| * | ||
| * <ul> | ||
| * <li>If the timeline has no inflight instants, | ||
| * use {@link org.apache.hudi.common.table.timeline.HoodieActiveTimeline#createNewInstantTime()} | ||
| * as the instant time;</li> | ||
| * <li>If the timeline has inflight instants, | ||
| * use the {earliest inflight instant time - 1ms} as the instant time.</li> | ||
| * </ul> | ||
| */ | ||
| public class CompactionPlanSourceFunction extends AbstractRichFunction implements SourceFunction<CompactionPlanEvent> { | ||
|
|
||
| protected static final Logger LOG = LoggerFactory.getLogger(CompactionPlanSourceFunction.class); | ||
|
|
||
| /** | ||
| * Compaction instant time. | ||
| */ | ||
| private String compactionInstantTime; | ||
|
|
||
| /** | ||
| * Hoodie flink table. | ||
| */ | ||
| private HoodieFlinkTable<?> table; | ||
|
|
||
| /** | ||
| * The compaction plan. | ||
| */ | ||
| private HoodieCompactionPlan compactionPlan; | ||
|
|
||
| /** | ||
| * Hoodie instant. | ||
| */ | ||
| private HoodieInstant instant; | ||
|
|
||
| public CompactionPlanSourceFunction(HoodieFlinkTable<?> table, HoodieInstant instant, HoodieCompactionPlan compactionPlan, String compactionInstantTime) { | ||
| this.table = table; | ||
| this.instant = instant; | ||
| this.compactionPlan = compactionPlan; | ||
| this.compactionInstantTime = compactionInstantTime; | ||
| } | ||
|
|
||
| @Override | ||
| public void open(Configuration parameters) throws Exception { | ||
| // no operation | ||
| } | ||
|
|
||
| @Override | ||
| public void run(SourceContext sourceContext) throws Exception { | ||
| // Mark instant as compaction inflight | ||
| table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); | ||
| table.getMetaClient().reloadActiveTimeline(); | ||
|
|
||
| List<CompactionOperation> operations = this.compactionPlan.getOperations().stream() | ||
| .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList()); | ||
| LOG.info("CompactionPlanFunction compacting " + operations + " files"); | ||
| for (CompactionOperation operation : operations) { | ||
| sourceContext.collect(new CompactionPlanEvent(compactionInstantTime, operation)); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws Exception { | ||
| // no operation | ||
| } | ||
|
|
||
| @Override | ||
| public void cancel() { | ||
| // no operation | ||
| } | ||
| } |
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.
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.
should we move these validation checks to a commonplace? Looks like this was being used elsewhere too.
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.
There is a bug in BaseScheduleCompactionActionExecutor$execute as the PR #3025 want to fix, this bug will let independence hudi compaction can not run. But this PR need to discuss, so we choose to modify the subclass FlinkScheduleCompactionActionExecutor to finish compaction.