[HUDI-2654] Schedules the compaction from earliest for flink (#3891)
This commit is contained in:
@@ -20,7 +20,6 @@ package org.apache.hudi.utils;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -30,38 +29,90 @@ import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.CompactionUtil;
|
||||
import org.apache.hudi.util.FlinkTables;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test cases for {@link org.apache.hudi.util.CompactionUtil}.
|
||||
*/
|
||||
public class TestCompactionUtil {
|
||||
|
||||
private HoodieFlinkTable<?> table;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private Configuration conf;
|
||||
|
||||
@TempDir
|
||||
File tempFile;
|
||||
|
||||
@Test
|
||||
void rollbackCompaction() throws IOException {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
conf.setInteger(FlinkOptions.COMPACTION_DELTA_SECONDS, 0);
|
||||
|
||||
@BeforeEach
|
||||
void beforeEach() throws IOException {
|
||||
this.conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
this.table = FlinkTables.createTable(conf);
|
||||
this.metaClient = table.getMetaClient();
|
||||
}
|
||||
|
||||
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
|
||||
HoodieFlinkTable table = writeClient.getHoodieTable();
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
@Test
|
||||
void rollbackCompaction() {
|
||||
conf.setInteger(FlinkOptions.COMPACTION_TIMEOUT_SECONDS, 0);
|
||||
List<String> oriInstants = IntStream.range(0, 3)
|
||||
.mapToObj(i -> generateCompactionPlan()).collect(Collectors.toList());
|
||||
List<HoodieInstant> instants = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT)
|
||||
.getInstants()
|
||||
.collect(Collectors.toList());
|
||||
assertThat("all the instants should be in pending state", instants.size(), is(3));
|
||||
CompactionUtil.rollbackCompaction(table, conf);
|
||||
boolean allRolledBack = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.allMatch(instant -> instant.getState() == HoodieInstant.State.REQUESTED);
|
||||
assertTrue(allRolledBack, "all the instants should be rolled back");
|
||||
List<String> actualInstants = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
assertThat(actualInstants, is(oriInstants));
|
||||
}
|
||||
|
||||
@Test
|
||||
void rollbackEarliestCompaction() {
|
||||
List<String> oriInstants = IntStream.range(0, 3)
|
||||
.mapToObj(i -> generateCompactionPlan()).collect(Collectors.toList());
|
||||
List<HoodieInstant> instants = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.INFLIGHT)
|
||||
.getInstants()
|
||||
.collect(Collectors.toList());
|
||||
assertThat("all the instants should be in pending state", instants.size(), is(3));
|
||||
CompactionUtil.rollbackEarliestCompaction(table);
|
||||
long requestedCnt = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED).count();
|
||||
assertThat("Only the first instant expects to be rolled back", requestedCnt, is(1L));
|
||||
|
||||
String instantTime = metaClient.getActiveTimeline()
|
||||
.filterPendingCompactionTimeline().filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED)
|
||||
.firstInstant().get().getTimestamp();
|
||||
assertThat(instantTime, is(oriInstants.get(0)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a compaction plan on the timeline and returns its instant time.
|
||||
*/
|
||||
private String generateCompactionPlan() {
|
||||
HoodieCompactionOperation operation = new HoodieCompactionOperation();
|
||||
HoodieCompactionPlan plan = new HoodieCompactionPlan(Collections.singletonList(operation), Collections.emptyMap(), 1);
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
@@ -75,13 +126,7 @@ public class TestCompactionUtil {
|
||||
throw new HoodieIOException("Exception scheduling compaction", ioe);
|
||||
}
|
||||
metaClient.reloadActiveTimeline();
|
||||
HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().orElse(null);
|
||||
assertThat(instant.getTimestamp(), is(instantTime));
|
||||
|
||||
CompactionUtil.rollbackCompaction(table, conf);
|
||||
HoodieInstant rollbackInstant = table.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get();
|
||||
assertThat(rollbackInstant.getState(), is(HoodieInstant.State.REQUESTED));
|
||||
assertThat(rollbackInstant.getTimestamp(), is(instantTime));
|
||||
return instantTime;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user