[HUDI-4152] Flink offline compaction support compacting multi compaction plan at once (#5677)
* [HUDI-4152] Flink offline compaction allow compact multi compaction plan at once * [HUDI-4152] Fix exception for duplicated uid when multi compaction plan are compacted * [HUDI-4152] Provider UT & IT for compact multi compaction plan * [HUDI-4152] Put multi compaction plans into one compaction plan source * [HUDI-4152] InstantCompactionPlanSelectStrategy allow multi instant by using comma * [HUDI-4152] Add IT for InstantCompactionPlanSelectStrategy
This commit is contained in:
@@ -18,6 +18,11 @@
|
||||
|
||||
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;
|
||||
@@ -25,6 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
@@ -49,6 +55,7 @@ import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -67,6 +74,8 @@ public class ITTestHoodieFlinkCompactor {
|
||||
|
||||
private static final Map<String, List<String>> EXPECTED2 = new HashMap<>();
|
||||
|
||||
private static final Map<String, List<String>> EXPECTED3 = new HashMap<>();
|
||||
|
||||
static {
|
||||
EXPECTED1.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1"));
|
||||
EXPECTED1.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2"));
|
||||
@@ -77,6 +86,12 @@ public class ITTestHoodieFlinkCompactor {
|
||||
EXPECTED2.put("par2", Arrays.asList("id3,par2,id3,Julian,54,3000,par2", "id4,par2,id4,Fabian,32,4000,par2"));
|
||||
EXPECTED2.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3", "id9,par3,id9,Jane,19,6000,par3"));
|
||||
EXPECTED2.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4", "id10,par4,id10,Ella,38,7000,par4", "id11,par4,id11,Phoebe,52,8000,par4"));
|
||||
|
||||
EXPECTED3.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1"));
|
||||
EXPECTED3.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2"));
|
||||
EXPECTED3.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3"));
|
||||
EXPECTED3.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4"));
|
||||
EXPECTED3.put("par5", Arrays.asList("id12,par5,id12,Tony,27,9000,par5", "id13,par5,id13,Jenny,72,10000,par5"));
|
||||
}
|
||||
|
||||
@TempDir
|
||||
@@ -203,4 +218,106 @@ public class ITTestHoodieFlinkCompactor {
|
||||
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED2);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testHoodieFlinkCompactorWithPlanSelectStrategy(boolean enableChangelog) throws Exception {
|
||||
// Create hoodie table and insert into data.
|
||||
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
|
||||
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
|
||||
tableEnv.getConfig().getConfiguration()
|
||||
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
|
||||
Map<String, String> options = new HashMap<>();
|
||||
options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false");
|
||||
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
|
||||
options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), enableChangelog + "");
|
||||
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
|
||||
tableEnv.executeSql(hoodieTableDDL);
|
||||
tableEnv.executeSql(TestSQL.INSERT_T1).await();
|
||||
|
||||
TimeUnit.SECONDS.sleep(3);
|
||||
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
FlinkCompactionConfig cfg = new FlinkCompactionConfig();
|
||||
cfg.path = tempFile.getAbsolutePath();
|
||||
Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg);
|
||||
conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
|
||||
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf);
|
||||
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
|
||||
CompactionUtil.setAvroSchema(conf, metaClient);
|
||||
CompactionUtil.inferChangelogMode(conf, metaClient);
|
||||
|
||||
List<String> compactionInstantTimeList = new ArrayList<>(2);
|
||||
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
|
||||
compactionInstantTimeList.add(scheduleCompactionPlan(metaClient, writeClient));
|
||||
|
||||
// insert a new record to new partition, so that we can generate a new compaction plan
|
||||
String insertT1ForNewPartition = "insert into t1 values\n"
|
||||
+ "('id12','Tony',27,TIMESTAMP '1970-01-01 00:00:09','par5'),\n"
|
||||
+ "('id13','Jenny',72,TIMESTAMP '1970-01-01 00:00:10','par5')";
|
||||
tableEnv.executeSql(insertT1ForNewPartition).await();
|
||||
|
||||
TimeUnit.SECONDS.sleep(3);
|
||||
|
||||
compactionInstantTimeList.add(scheduleCompactionPlan(metaClient, writeClient));
|
||||
|
||||
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
|
||||
|
||||
List<Pair<String, HoodieCompactionPlan>> compactionPlans = new ArrayList<>(2);
|
||||
for (String compactionInstantTime : compactionInstantTimeList) {
|
||||
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(table.getMetaClient(), compactionInstantTime);
|
||||
compactionPlans.add(Pair.of(compactionInstantTime, plan));
|
||||
}
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
for (String compactionInstantTime : compactionInstantTimeList) {
|
||||
HoodieInstant hoodieInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(hoodieInstant);
|
||||
}
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
DataStream<CompactionPlanEvent> source = env.addSource(new MultiCompactionPlanSourceFunction(compactionPlans))
|
||||
.name("compaction_source")
|
||||
.uid("uid_compaction_source");
|
||||
SingleOutputStreamOperator<Void> operator = 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")
|
||||
.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();
|
||||
TestData.checkWrittenFullData(tempFile, EXPECTED3);
|
||||
}
|
||||
|
||||
private String scheduleCompactionPlan(HoodieTableMetaClient metaClient, HoodieFlinkWriteClient<?> writeClient) {
|
||||
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());
|
||||
}
|
||||
assertTrue(scheduled, "The compaction plan should be scheduled");
|
||||
return compactionInstantTimeOption.get();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,149 @@
|
||||
/*
|
||||
* 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 static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
import java.util.Arrays;
|
||||
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.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
/**
|
||||
* Test case for every {@link CompactionPlanSelectStrategy} implements
|
||||
*/
|
||||
public class TestCompactionPlanSelectStrategy {
|
||||
private HoodieTimeline timeline;
|
||||
private HoodieTimeline emptyTimeline;
|
||||
private HoodieTimeline allCompleteTimeline;
|
||||
|
||||
private static final HoodieInstant INSTANT_001 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
|
||||
private static final HoodieInstant INSTANT_002 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "002");
|
||||
private static final HoodieInstant INSTANT_003 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003");
|
||||
private static final HoodieInstant INSTANT_004 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "004");
|
||||
private static final HoodieInstant INSTANT_005 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMPACTION_ACTION, "005");
|
||||
private static final HoodieInstant INSTANT_006 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006");
|
||||
|
||||
@BeforeEach
|
||||
public void beforeEach() {
|
||||
timeline = new MockHoodieActiveTimeline(INSTANT_001, INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_005, INSTANT_006);
|
||||
emptyTimeline = new MockHoodieActiveTimeline();
|
||||
allCompleteTimeline = new MockHoodieActiveTimeline(INSTANT_001, INSTANT_005);
|
||||
}
|
||||
|
||||
@Test
|
||||
void testSingleCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
|
||||
SingleCompactionPlanSelectStrategy strategy = new SingleCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO;
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
|
||||
|
||||
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testMultiCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
compactionConfig.compactionPlanMaxSelect = 2;
|
||||
|
||||
MultiCompactionPlanSelectStrategy strategy = new MultiCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO;
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
|
||||
|
||||
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testAllPendingCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
|
||||
AllPendingCompactionPlanSelectStrategy strategy = new AllPendingCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_006},
|
||||
strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
|
||||
|
||||
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
|
||||
}
|
||||
|
||||
@Test
|
||||
void testInstantCompactionPlanSelectStrategy() {
|
||||
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
|
||||
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
|
||||
compactionConfig.compactionPlanInstant = "004";
|
||||
|
||||
InstantCompactionPlanSelectStrategy strategy = new InstantCompactionPlanSelectStrategy();
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
compactionConfig.compactionPlanInstant = "002,003";
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
compactionConfig.compactionPlanInstant = "002,005";
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
|
||||
compactionConfig.compactionPlanInstant = "005";
|
||||
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(pendingCompactionTimeline, compactionConfig));
|
||||
}
|
||||
|
||||
private void assertHoodieInstantsEquals(HoodieInstant[] expected, List<HoodieInstant> actual) {
|
||||
assertEquals(expected.length, actual.size());
|
||||
for (int index = 0; index < expected.length; index++) {
|
||||
assertHoodieInstantEquals(expected[index], actual.get(index));
|
||||
}
|
||||
}
|
||||
|
||||
private void assertHoodieInstantEquals(HoodieInstant expected, HoodieInstant actual) {
|
||||
assertEquals(expected.getState(), actual.getState());
|
||||
assertEquals(expected.getAction(), actual.getAction());
|
||||
assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
}
|
||||
|
||||
private static final class MockHoodieActiveTimeline extends HoodieActiveTimeline {
|
||||
public MockHoodieActiveTimeline(HoodieInstant... instants) {
|
||||
super();
|
||||
setInstants(Arrays.asList(instants));
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user