1
0

[HUDI-4258] Fix when HoodieTable removes data file before the end of Flink job (#5876)

* [HUDI-4258] Fix when HoodieTable removes data file before the end of Flink job
This commit is contained in:
Alexander Trushev
2022-06-20 16:07:49 +07:00
committed by GitHub
parent 7c6bedff25
commit f1103281d2
4 changed files with 102 additions and 19 deletions

View File

@@ -285,7 +285,8 @@ public class StreamWriteOperatorCoordinator
if (event.isEndInput()) {
// handle end input event synchronously
handleEndInputEvent(event);
// wrap handleEndInputEvent in executeSync to preserve the order of events
executor.executeSync(() -> handleEndInputEvent(event), "handle end input event for instant %s", this.instant);
} else {
executor.execute(
() -> {

View File

@@ -163,6 +163,18 @@ public class WriteMetadataEvent implements OperatorEvent {
return lastBatch && this.instantTime.equals(currentInstant);
}
@Override
public String toString() {
return "WriteMetadataEvent{"
+ "writeStatusesSize=" + writeStatuses.size()
+ ", taskID=" + taskID
+ ", instantTime='" + instantTime + '\''
+ ", lastBatch=" + lastBatch
+ ", endInput=" + endInput
+ ", bootstrap=" + bootstrap
+ '}';
}
// -------------------------------------------------------------------------
// Utilities
// -------------------------------------------------------------------------

View File

@@ -26,9 +26,11 @@ import org.slf4j.Logger;
import javax.annotation.Nullable;
import java.util.Objects;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
/**
* An executor service that catches all the throwable with logging.
@@ -85,25 +87,21 @@ public class NonThrownExecutor implements AutoCloseable {
final ExceptionHook hook,
final String actionName,
final Object... actionParams) {
executor.execute(wrapAction(action, hook, actionName, actionParams));
}
executor.execute(
() -> {
final String actionString = String.format(actionName, actionParams);
try {
action.run();
logger.info("Executor executes action [{}] success!", actionString);
} 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
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
final String errMsg = String.format("Executor executes action [%s] error", actionString);
logger.error(errMsg, t);
if (hook != null) {
hook.apply(errMsg, t);
}
}
});
/**
* Run the action in a loop and wait for completion.
*/
public void executeSync(ThrowingRunnable<Throwable> action, String actionName, Object... actionParams) {
try {
executor.submit(wrapAction(action, this.exceptionHook, actionName, actionParams)).get();
} catch (InterruptedException e) {
handleException(e, this.exceptionHook, getActionString(actionName, actionParams));
} catch (ExecutionException e) {
// nonfatal exceptions are handled by wrapAction
ExceptionUtils.rethrowIfFatalErrorOrOOM(e.getCause());
}
}
@Override
@@ -120,6 +118,40 @@ public class NonThrownExecutor implements AutoCloseable {
}
}
private <E extends Throwable> Runnable wrapAction(
final ThrowingRunnable<E> action,
final ExceptionHook hook,
final String actionName,
final Object... actionParams) {
return () -> {
final Supplier<String> actionString = getActionString(actionName, actionParams);
try {
action.run();
logger.info("Executor executes action [{}] success!", actionString.get());
} catch (Throwable t) {
handleException(t, hook, actionString);
}
};
}
private void handleException(Throwable t, ExceptionHook hook, Supplier<String> actionString) {
// 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
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
final String errMsg = String.format("Executor executes action [%s] error", actionString.get());
logger.error(errMsg, t);
if (hook != null) {
hook.apply(errMsg, t);
}
}
private Supplier<String> getActionString(String actionName, Object... actionParams) {
// avoid String.format before OOM rethrown
return () -> String.format(actionName, actionParams);
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------