Skip to content

Commit 6df6164

Browse files
[HUDI-4152] Provider UT & IT for compact multi compaction plan
1 parent 538fdfb commit 6df6164

2 files changed

Lines changed: 270 additions & 0 deletions

File tree

hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java

Lines changed: 125 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -18,13 +18,19 @@
1818

1919
package org.apache.hudi.sink.compact;
2020

21+
import org.apache.flink.streaming.api.datastream.DataStream;
22+
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
23+
import org.apache.flink.streaming.api.functions.ProcessFunction;
24+
import org.apache.flink.util.Collector;
25+
import org.apache.flink.util.OutputTag;
2126
import org.apache.hudi.avro.model.HoodieCompactionPlan;
2227
import org.apache.hudi.client.HoodieFlinkWriteClient;
2328
import org.apache.hudi.common.table.HoodieTableMetaClient;
2429
import org.apache.hudi.common.table.timeline.HoodieInstant;
2530
import org.apache.hudi.common.table.timeline.HoodieTimeline;
2631
import org.apache.hudi.common.util.CompactionUtils;
2732
import org.apache.hudi.common.util.Option;
33+
import org.apache.hudi.common.util.collection.Pair;
2834
import org.apache.hudi.configuration.FlinkOptions;
2935
import org.apache.hudi.table.HoodieFlinkTable;
3036
import org.apache.hudi.util.CompactionUtil;
@@ -49,6 +55,7 @@
4955

5056
import java.io.File;
5157
import java.util.Arrays;
58+
import java.util.ArrayList;
5259
import java.util.HashMap;
5360
import java.util.List;
5461
import java.util.Map;
@@ -67,6 +74,8 @@ public class ITTestHoodieFlinkCompactor {
6774

6875
private static final Map<String, List<String>> EXPECTED2 = new HashMap<>();
6976

77+
private static final Map<String, List<String>> EXPECTED3 = new HashMap<>();
78+
7079
static {
7180
EXPECTED1.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1"));
7281
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 {
7786
EXPECTED2.put("par2", Arrays.asList("id3,par2,id3,Julian,54,3000,par2", "id4,par2,id4,Fabian,32,4000,par2"));
7887
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"));
7988
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"));
89+
90+
EXPECTED3.put("par1", Arrays.asList("id1,par1,id1,Danny,23,1000,par1", "id2,par1,id2,Stephen,33,2000,par1"));
91+
EXPECTED3.put("par2", Arrays.asList("id3,par2,id3,Julian,53,3000,par2", "id4,par2,id4,Fabian,31,4000,par2"));
92+
EXPECTED3.put("par3", Arrays.asList("id5,par3,id5,Sophia,18,5000,par3", "id6,par3,id6,Emma,20,6000,par3"));
93+
EXPECTED3.put("par4", Arrays.asList("id7,par4,id7,Bob,44,7000,par4", "id8,par4,id8,Han,56,8000,par4"));
94+
EXPECTED3.put("par5", Arrays.asList("id12,par5,id12,Tony,27,9000,par5", "id13,par5,id13,Jenny,72,10000,par5"));
8095
}
8196

8297
@TempDir
@@ -203,4 +218,114 @@ public void testHoodieFlinkCompactorService(boolean enableChangelog) throws Exce
203218

204219
TestData.checkWrittenFullData(tempFile, EXPECTED2);
205220
}
221+
222+
@ParameterizedTest
223+
@ValueSource(booleans = {true, false})
224+
public void testHoodieFlinkCompactorWithPlanSelectStrategy(boolean enableChangelog) throws Exception {
225+
// Create hoodie table and insert into data.
226+
EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
227+
TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
228+
tableEnv.getConfig().getConfiguration()
229+
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
230+
Map<String, String> options = new HashMap<>();
231+
options.put(FlinkOptions.COMPACTION_ASYNC_ENABLED.key(), "false");
232+
options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
233+
options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
234+
options.put(FlinkOptions.CHANGELOG_ENABLED.key(), enableChangelog + "");
235+
String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", options);
236+
tableEnv.executeSql(hoodieTableDDL);
237+
tableEnv.executeSql(TestSQL.INSERT_T1).await();
238+
239+
TimeUnit.SECONDS.sleep(3);
240+
241+
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
242+
FlinkCompactionConfig cfg = new FlinkCompactionConfig();
243+
cfg.path = tempFile.getAbsolutePath();
244+
Configuration conf = FlinkCompactionConfig.toFlinkConfig(cfg);
245+
conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
246+
247+
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(conf);
248+
conf.setString(FlinkOptions.TABLE_NAME, metaClient.getTableConfig().getTableName());
249+
CompactionUtil.setAvroSchema(conf, metaClient);
250+
CompactionUtil.inferChangelogMode(conf, metaClient);
251+
252+
List<String> compactionInstantTimeList = new ArrayList<>(2);
253+
254+
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
255+
256+
compactionInstantTimeList.add(scheduleCompactionPlan(metaClient, writeClient));
257+
258+
// insert a new record to new partition, so that we can generate a new compaction plan
259+
String insertT1ForNewPartition = "insert into t1 values\n"
260+
+ "('id12','Tony',27,TIMESTAMP '1970-01-01 00:00:09','par5'),\n"
261+
+ "('id13','Jenny',72,TIMESTAMP '1970-01-01 00:00:10','par5')";
262+
tableEnv.executeSql(insertT1ForNewPartition).await();
263+
264+
TimeUnit.SECONDS.sleep(3);
265+
266+
compactionInstantTimeList.add(scheduleCompactionPlan(metaClient, writeClient));
267+
268+
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
269+
270+
List<Pair<String, HoodieCompactionPlan>> compactionPlans = new ArrayList<>(2);
271+
for (String compactionInstantTime : compactionInstantTimeList) {
272+
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(table.getMetaClient(), compactionInstantTime);
273+
compactionPlans.add(Pair.of(compactionInstantTime, plan));
274+
}
275+
276+
// Mark instant as compaction inflight
277+
for (String compactionInstantTime : compactionInstantTimeList) {
278+
HoodieInstant hoodieInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
279+
table.getActiveTimeline().transitionCompactionRequestedToInflight(hoodieInstant);
280+
}
281+
table.getMetaClient().reloadActiveTimeline();
282+
283+
Pair<String, HoodieCompactionPlan> firstPlan = compactionPlans.get(0);
284+
DataStream<CompactionPlanEvent> source = env.addSource(new CompactionPlanSourceFunction(firstPlan.getRight(), firstPlan.getLeft()))
285+
.name("compaction_source " + firstPlan.getLeft())
286+
.uid("uid_compaction_source " + firstPlan.getLeft());
287+
if (compactionPlans.size() > 1) {
288+
for (Pair<String, HoodieCompactionPlan> pair : compactionPlans.subList(1, compactionPlans.size())) {
289+
source = source.union(env.addSource(new CompactionPlanSourceFunction(pair.getRight(), pair.getLeft()))
290+
.name("compaction_source " + pair.getLeft())
291+
.uid("uid_compaction_source " + pair.getLeft()));
292+
}
293+
}
294+
SingleOutputStreamOperator<Void> operator = source.rebalance()
295+
.transform("compact_task",
296+
TypeInformation.of(CompactionCommitEvent.class),
297+
new ProcessOperator<>(new CompactFunction(conf)))
298+
.setParallelism(1)
299+
.process(new ProcessFunction<CompactionCommitEvent, Void>() {
300+
@Override
301+
public void processElement(CompactionCommitEvent event, ProcessFunction<CompactionCommitEvent, Void>.Context context, Collector<Void> out) {
302+
context.output(new OutputTag<>(event.getInstant(), TypeInformation.of(CompactionCommitEvent.class)), event);
303+
}
304+
})
305+
.name("group_by_compaction_plan")
306+
.uid("uid_group_by_compaction_plan")
307+
.setParallelism(1);
308+
compactionPlans.forEach(pair ->
309+
operator.getSideOutput(new OutputTag<>(pair.getLeft(), TypeInformation.of(CompactionCommitEvent.class)))
310+
.addSink(new CompactionCommitSink(conf))
311+
.name("clean_commits " + pair.getLeft())
312+
.uid("uid_clean_commits_" + pair.getLeft())
313+
.setParallelism(1));
314+
315+
env.execute("flink_hudi_compaction");
316+
writeClient.close();
317+
TestData.checkWrittenFullData(tempFile, EXPECTED3);
318+
}
319+
320+
private String scheduleCompactionPlan(HoodieTableMetaClient metaClient, HoodieFlinkWriteClient<?> writeClient) {
321+
boolean scheduled = false;
322+
// judge whether have operation
323+
// To compute the compaction instant time and do compaction.
324+
Option<String> compactionInstantTimeOption = CompactionUtil.getCompactionInstantTime(metaClient);
325+
if (compactionInstantTimeOption.isPresent()) {
326+
scheduled = writeClient.scheduleCompactionAtInstant(compactionInstantTimeOption.get(), Option.empty());
327+
}
328+
assertTrue(scheduled, "The compaction plan should be scheduled");
329+
return compactionInstantTimeOption.get();
330+
}
206331
}
Lines changed: 145 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,145 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hudi.sink.compact;
20+
21+
import static org.junit.jupiter.api.Assertions.assertEquals;
22+
import static org.junit.jupiter.api.Assertions.assertThrows;
23+
24+
import java.util.Arrays;
25+
import java.util.List;
26+
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
27+
import org.apache.hudi.common.table.timeline.HoodieInstant;
28+
import org.apache.hudi.common.table.timeline.HoodieTimeline;
29+
import org.apache.hudi.exception.HoodieException;
30+
import org.apache.hudi.sink.compact.strategy.AllPendingCompactionPlanSelectStrategy;
31+
import org.apache.hudi.sink.compact.strategy.CompactionPlanSelectStrategy;
32+
import org.apache.hudi.sink.compact.strategy.InstantCompactionPlanSelectStrategy;
33+
import org.apache.hudi.sink.compact.strategy.MultiCompactionPlanSelectStrategy;
34+
import org.apache.hudi.sink.compact.strategy.SingleCompactionPlanSelectStrategy;
35+
import org.junit.jupiter.api.BeforeEach;
36+
import org.junit.jupiter.api.Test;
37+
38+
/**
39+
* Test case for every {@link CompactionPlanSelectStrategy} implements
40+
*/
41+
public class TestCompactionPlanSelectStrategy {
42+
private HoodieTimeline timeline;
43+
private HoodieTimeline emptyTimeline;
44+
private HoodieTimeline allCompleteTimeline;
45+
46+
private static final HoodieInstant INSTANT_001 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001");
47+
private static final HoodieInstant INSTANT_002 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "002");
48+
private static final HoodieInstant INSTANT_003 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003");
49+
private static final HoodieInstant INSTANT_004 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "004");
50+
private static final HoodieInstant INSTANT_005 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMPACTION_ACTION, "005");
51+
private static final HoodieInstant INSTANT_006 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006");
52+
53+
@BeforeEach
54+
public void beforeEach() {
55+
timeline = new MockHoodieActiveTimeline(INSTANT_001, INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_005, INSTANT_006);
56+
emptyTimeline = new MockHoodieActiveTimeline();
57+
allCompleteTimeline = new MockHoodieActiveTimeline(INSTANT_001, INSTANT_005);
58+
}
59+
60+
@Test
61+
void testSingleCompactionPlanSelectStrategy() {
62+
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
63+
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
64+
65+
SingleCompactionPlanSelectStrategy strategy = new SingleCompactionPlanSelectStrategy();
66+
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006}, strategy.select(pendingCompactionTimeline, compactionConfig));
67+
68+
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO;
69+
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002}, strategy.select(pendingCompactionTimeline, compactionConfig));
70+
71+
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
72+
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
73+
74+
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
75+
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
76+
}
77+
78+
@Test
79+
void testMultiCompactionPlanSelectStrategy() {
80+
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
81+
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
82+
compactionConfig.compactionPlanMaxSelect = 2;
83+
84+
MultiCompactionPlanSelectStrategy strategy = new MultiCompactionPlanSelectStrategy();
85+
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_006, INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig));
86+
87+
compactionConfig.compactionSeq = FlinkCompactionConfig.SEQ_FIFO;
88+
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003}, strategy.select(pendingCompactionTimeline, compactionConfig));
89+
90+
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
91+
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
92+
93+
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
94+
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
95+
}
96+
97+
@Test
98+
void testAllPendingCompactionPlanSelectStrategy() {
99+
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
100+
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
101+
102+
AllPendingCompactionPlanSelectStrategy strategy = new AllPendingCompactionPlanSelectStrategy();
103+
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_002, INSTANT_003, INSTANT_004, INSTANT_006},
104+
strategy.select(pendingCompactionTimeline, compactionConfig));
105+
106+
HoodieTimeline emptyPendingCompactionTimeline = emptyTimeline.filterPendingCompactionTimeline();
107+
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(emptyPendingCompactionTimeline, compactionConfig));
108+
109+
HoodieTimeline allCompleteCompactionTimeline = allCompleteTimeline.filterPendingCompactionTimeline();
110+
assertHoodieInstantsEquals(new HoodieInstant[]{}, strategy.select(allCompleteCompactionTimeline, compactionConfig));
111+
}
112+
113+
@Test
114+
void testInstantCompactionPlanSelectStrategy() {
115+
HoodieTimeline pendingCompactionTimeline = this.timeline.filterPendingCompactionTimeline();
116+
FlinkCompactionConfig compactionConfig = new FlinkCompactionConfig();
117+
compactionConfig.compactionPlanInstant = "004";
118+
119+
InstantCompactionPlanSelectStrategy strategy = new InstantCompactionPlanSelectStrategy();
120+
assertHoodieInstantsEquals(new HoodieInstant[]{INSTANT_004}, strategy.select(pendingCompactionTimeline, compactionConfig));
121+
122+
compactionConfig.compactionPlanInstant = "005";
123+
assertThrows(HoodieException.class, () -> strategy.select(pendingCompactionTimeline, compactionConfig));
124+
}
125+
126+
private void assertHoodieInstantsEquals(HoodieInstant[] expected, List<HoodieInstant> actual) {
127+
assertEquals(expected.length, actual.size());
128+
for (int index = 0; index < expected.length; index++) {
129+
assertHoodieInstantEquals(expected[index], actual.get(index));
130+
}
131+
}
132+
133+
private void assertHoodieInstantEquals(HoodieInstant expected, HoodieInstant actual) {
134+
assertEquals(expected.getState(), actual.getState());
135+
assertEquals(expected.getAction(), actual.getAction());
136+
assertEquals(expected.getTimestamp(), actual.getTimestamp());
137+
}
138+
139+
private static final class MockHoodieActiveTimeline extends HoodieActiveTimeline {
140+
public MockHoodieActiveTimeline(HoodieInstant... instants) {
141+
super();
142+
setInstants(Arrays.asList(instants));
143+
}
144+
}
145+
}

0 commit comments

Comments
 (0)