[minor] following 4152, refactor the clazz about plan selection strategy (#6060)
This commit is contained in:
@@ -18,11 +18,6 @@
|
||||
|
||||
package org.apache.hudi.sink.compact;
|
||||
|
||||
import org.apache.flink.streaming.api.datastream.DataStream;
|
||||
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
|
||||
import org.apache.flink.streaming.api.functions.ProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.apache.flink.util.OutputTag;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
@@ -54,8 +49,9 @@ import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -138,17 +134,7 @@ public class ITTestHoodieFlinkCompactor {
|
||||
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
|
||||
boolean scheduled = false;
|
||||
// judge whether have operation
|
||||
// To compute the compaction instant time and do compaction.
|
||||
Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
if (compactionInstantTimeOption.isPresent()) {
|
||||
scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
|
||||
}
|
||||
|
||||
String compactionInstantTime = compactionInstantTimeOption.get();
|
||||
|
||||
assertTrue(scheduled, "The compaction plan should be scheduled");
|
||||
String compactionInstantTime = scheduleCompactionPlan(metaClient, writeClient);
|
||||
|
||||
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
|
||||
// generate compaction plan
|
||||
@@ -160,7 +146,7 @@ public class ITTestHoodieFlinkCompactor {
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
|
||||
env.addSource(new CompactionPlanSourceFunction(compactionPlan, compactionInstantTime))
|
||||
env.addSource(new CompactionPlanSourceFunction(Collections.singletonList(Pair.of(compactionInstantTime, compactionPlan))))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
@@ -280,29 +266,18 @@ public class ITTestHoodieFlinkCompactor {
|
||||
}
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
DataStream<CompactionPlanEvent> source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans))
|
||||
env.addSource(new CompactionPlanSourceFunction(compactionPlans))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source");
|
||||
SingleOutputStreamOperator<Void> operator = source.rebalance()
|
||||
.uid("uid_compaction_source")
|
||||
.rebalance()
|
||||
.transform("compact_task",
|
||||
TypeInformation.of(CompactionCommitEvent.class),
|
||||
new ProcessOperator<>(new CompactFunction(conf)))
|
||||
.setParallelism(1)
|
||||
.process(new ProcessFunction<CompactionCommitEvent, Void>() {
|
||||
@Override
|
||||
public void processElement(CompactionCommitEvent event, ProcessFunction<CompactionCommitEvent, Void>.Context context, Collector<Void> out) {
|
||||
context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event);
|
||||
}
|
||||
})
|
||||
.name("group_by_compaction_plan")
|
||||
.uid("uid_group_by_compaction_plan")
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("compaction_commit")
|
||||
.uid("uid_compaction_commit")
|
||||
.setParallelism(1);
|
||||
compactionPlans.forEach(pair ->
|
||||
operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class)))
|
||||
.addSink(new CompactionCommitSink(conf))
|
||||
.name("clean_commits " + pair.getLeft())
|
||||
.uid("uid_clean_commits_" + pair.getLeft())
|
||||
.setParallelism(1));
|
||||
|
||||
env.execute("flink_hudi_compaction");
|
||||
writeClient.close();
|
||||
@@ -311,8 +286,7 @@ public class ITTestHoodieFlinkCompactor {
|
||||
|
||||
private String scheduleCompactionPlan(HoodieTableMetaClient metaClient, HoodieFlinkWriteClient<?> writeClient) {
|
||||
boolean scheduled = false;
|
||||
// judge whether have operation
|
||||
// To compute the compaction instant time and do compaction.
|
||||
// judge whether there are any compaction operations.
|
||||
Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
|
||||
if (compactionInstantTimeOption.isPresent()) {
|
||||
scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
|
||||
|
||||
@@ -25,18 +25,16 @@ import java.util.List;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.sink.compact.strategy.AllPendingCompactionPlanSelectStrategy;
|
||||
import org.apache.hudi.sink.compact.strategy.CompactionPlanSelectStrategy;
|
||||
import org.apache.hudi.sink.compact.strategy.InstantCompactionPlanSelectStrategy;
|
||||
import org.apache.hudi.sink.compact.strategy.MultiCompactionPlanSelectStrategy;
|
||||
import org.apache.hudi.sink.compact.strategy.SingleCompactionPlanSelectStrategy;
|
||||
import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies;
|
||||
import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategy;
|
||||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/**
|
||||
* Test case for every {@link CompactionPlanSelectStrategy} implements
|
||||
* Test case for every {@link CompactionPlanStrategy} implements
|
||||
*/
|
||||
public class TestCompactionPlanSelectStrategy {
|
||||
public class TestCompactionPlanStrategy {
|
||||
private HoodieTimeline timeline;
|
||||
private HoodieTimeline emptyTimeline;
|
||||
private HoodieTimeline allCompleteTimeline;
|
||||
@@ -59,72 +57,75 @@ public class TestCompactionPlanSelectStrategy {
|
||||
void testSingleCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig);
|
||||
|
||||
SingleCompactionPlanSelectStrategy strategy = new SingleCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO;
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_LIFO;
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline));
|
||||
|
||||
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMultiCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
compactionConfig.compactionPlanMaxSelect = 2;
|
||||
compactionConfig.maxNumCompactionPlans = 2;
|
||||
|
||||
MultiCompactionPlanSelectStrategy strategy = new MultiCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig);
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO;
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_LIFO;
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline));
|
||||
|
||||
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAllPendingCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
compactionConfig.compactionPlanSelectStrategy = CompactionPlanStrategy.ALL;
|
||||
CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig);
|
||||
|
||||
AllPendingCompactionPlanSelectStrategy strategy = new AllPendingCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_006},
|
||||
strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
strategy.select(pendingCompactionTimeline));
|
||||
|
||||
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline));
|
||||
|
||||
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInstantCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
|
||||
compactionConfig.compactionPlanSelectStrategy = CompactionPlanStrategy.INSTANTS;
|
||||
CompactionPlanStrategy strategy = CompactionPlanStrategies.getStrategy(compactionConfig);
|
||||
compactionConfig.compactionPlanInstant = "004";
|
||||
|
||||
InstantCompactionPlanSelectStrategy strategy = new InstantCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
compactionConfig.compactionPlanInstant = "002,003";
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
compactionConfig.compactionPlanInstant = "002,005";
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline));
|
||||
|
||||
compactionConfig.compactionPlanInstant = "005";
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline));
|
||||
}
|
||||
|
||||
private void assertHoodieInstantsEquals(HoodieInstant[] expected, List<HoodieInstant> actual) {
|
||||
Reference in New Issue
Block a user