[HUDI-2654] Add compaction failed event(part2) (#3896)
This commit is contained in:
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.sink.utils;
|
||||
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
|
||||
import org.apache.flink.util.ExceptionUtils;
|
||||
import org.apache.flink.util.function.ThrowingRunnable;
|
||||
@@ -25,17 +27,21 @@ import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* A mock {@link CoordinatorExecutor} that executes the actions synchronously.
|
||||
* A mock {@link NonThrownExecutor} that executes the actions synchronously.
|
||||
*/
|
||||
public class MockCoordinatorExecutor extends CoordinatorExecutor {
|
||||
public class MockCoordinatorExecutor extends NonThrownExecutor {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(MockCoordinatorExecutor.class);
|
||||
|
||||
public MockCoordinatorExecutor(OperatorCoordinator.Context context) {
|
||||
super(context, LOG);
|
||||
super(LOG, (errMsg, t) -> context.failJob(new HoodieException(errMsg, t)), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(ThrowingRunnable<Throwable> action, String actionName, Object... actionParams) {
|
||||
public void execute(
|
||||
ThrowingRunnable<Throwable> action,
|
||||
ExceptionHook hook,
|
||||
String actionName,
|
||||
Object... actionParams) {
|
||||
final String actionString = String.format(actionName, actionParams);
|
||||
try {
|
||||
action.run();
|
||||
@@ -43,9 +49,12 @@ public class MockCoordinatorExecutor extends CoordinatorExecutor {
|
||||
} catch (Throwable t) {
|
||||
// if we have a JVM critical error, promote it immediately, there is a good
|
||||
// chance the
|
||||
// logging or job failing will not succeed any more
|
||||
// logging or job failing will not succeed anymore
|
||||
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
|
||||
exceptionHook(actionString, t);
|
||||
final String errMsg = String.format("Executor executes action [%s] error", actionString);
|
||||
if (hook != null) {
|
||||
hook.apply(errMsg, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -70,7 +70,6 @@ public class TestCompactionUtil {
|
||||
|
||||
@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()
|
||||
@@ -79,7 +78,7 @@ public class TestCompactionUtil {
|
||||
.getInstants()
|
||||
.collect(Collectors.toList());
|
||||
assertThat("all the instants should be in pending state", instants.size(), is(3));
|
||||
CompactionUtil.rollbackCompaction(table, conf);
|
||||
CompactionUtil.rollbackCompaction(table);
|
||||
boolean allRolledBack = metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants()
|
||||
.allMatch(instant -> instant.getState() == HoodieInstant.State.REQUESTED);
|
||||
assertTrue(allRolledBack, "all the instants should be rolled back");
|
||||
@@ -90,6 +89,7 @@ public class TestCompactionUtil {
|
||||
|
||||
@Test
|
||||
void rollbackEarliestCompaction() {
|
||||
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()
|
||||
@@ -98,7 +98,7 @@ public class TestCompactionUtil {
|
||||
.getInstants()
|
||||
.collect(Collectors.toList());
|
||||
assertThat("all the instants should be in pending state", instants.size(), is(3));
|
||||
CompactionUtil.rollbackEarliestCompaction(table);
|
||||
CompactionUtil.rollbackEarliestCompaction(table, conf);
|
||||
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));
|
||||
|
||||
Reference in New Issue
Block a user