Skip to content

Commit

Permalink
[FLINK-37110][cdc-runtime] Fixed the problem of using the same execut…
Browse files Browse the repository at this point in the history
…or to process schema change requests and flush event requests, resulting in blocking timeout.
  • Loading branch information
linjc13 committed Jan 14, 2025
1 parent a130718 commit 032e560
Showing 1 changed file with 14 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeoutException;
import java.util.function.BooleanSupplier;

Expand Down Expand Up @@ -96,6 +97,9 @@ public abstract class SchemaRegistry implements OperatorCoordinator, Coordinatio
protected transient SchemaManager schemaManager;
protected transient TableIdRouter router;

/** Executor service to execute handle event from operator. */
private final ExecutorService runInEventFromOperatorExecutor;

protected SchemaRegistry(
OperatorCoordinator.Context context,
String operatorName,
Expand All @@ -111,6 +115,7 @@ protected SchemaRegistry(
this.routingRules = routingRules;
this.rpcTimeout = rpcTimeout;
this.behavior = schemaChangeBehavior;
this.runInEventFromOperatorExecutor = Executors.newSingleThreadExecutor();
}

// ---------------
Expand All @@ -130,6 +135,7 @@ public void start() throws Exception {
public void close() throws Exception {
LOG.info("Closing SchemaRegistry - {}.", operatorName);
coordinatorExecutor.shutdown();
runInEventFromOperatorExecutor.shutdown();
}

// ------------------------------
Expand Down Expand Up @@ -235,6 +241,7 @@ public final CompletableFuture<CoordinationResponse> handleCoordinationRequest(
CoordinationRequest request) {
CompletableFuture<CoordinationResponse> future = new CompletableFuture<>();
runInEventLoop(
coordinatorExecutor,
() -> {
if (request instanceof GetEvolvedSchemaRequest) {
handleGetEvolvedSchemaRequest((GetEvolvedSchemaRequest) request, future);
Expand All @@ -253,6 +260,7 @@ public final CompletableFuture<CoordinationResponse> handleCoordinationRequest(
public final void handleEventFromOperator(
int subTaskId, int attemptNumber, OperatorEvent event) {
runInEventLoop(
runInEventFromOperatorExecutor,
() -> {
if (event instanceof FlushSuccessEvent) {
handleFlushSuccessEvent((FlushSuccessEvent) event);
Expand Down Expand Up @@ -297,7 +305,11 @@ public final void executionAttemptReady(
public final void checkpointCoordinator(
long checkpointId, CompletableFuture<byte[]> completableFuture) throws Exception {
LOG.info("Going to start checkpoint No.{}", checkpointId);
runInEventLoop(() -> snapshot(completableFuture), "Taking checkpoint - %d", checkpointId);
runInEventLoop(
coordinatorExecutor,
() -> snapshot(completableFuture),
"Taking checkpoint - %d",
checkpointId);
}

@Override
Expand Down Expand Up @@ -325,6 +337,7 @@ public final void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpoi
* directly, make sure you're running heavy logics inside, or the entire job might hang!
*/
protected void runInEventLoop(
final ExecutorService coordinatorExecutor,
final ThrowingRunnable<Throwable> action,
final String actionName,
final Object... actionNameFormatParameters) {
Expand Down

0 comments on commit 032e560

Please sign in to comment.