Skip to content

Commit 27569da

Browse files
rkhachatryanpnowojski
authored andcommitted
[FLINK-36710] Log job ID in RecreateOnResetOperatorCoordinator
1 parent 7341b6e commit 27569da

File tree

2 files changed

+11
-7
lines changed

2 files changed

+11
-7
lines changed

flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/RecreateOnResetOperatorCoordinator.java

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ Licensed to the Apache Software Foundation (ASF) under one
2323
import org.apache.flink.metrics.groups.OperatorCoordinatorMetricGroup;
2424
import org.apache.flink.runtime.checkpoint.CheckpointCoordinator;
2525
import org.apache.flink.runtime.jobgraph.OperatorID;
26+
import org.apache.flink.util.MdcUtils;
2627
import org.apache.flink.util.Preconditions;
2728
import org.apache.flink.util.function.ThrowingConsumer;
2829
import org.apache.flink.util.function.ThrowingRunnable;
@@ -33,6 +34,7 @@ Licensed to the Apache Software Foundation (ASF) under one
3334
import javax.annotation.Nullable;
3435

3536
import java.time.Duration;
37+
import java.util.Map;
3638
import java.util.concurrent.BlockingQueue;
3739
import java.util.concurrent.CompletableFuture;
3840
import java.util.concurrent.LinkedBlockingQueue;
@@ -58,7 +60,7 @@ private RecreateOnResetOperatorCoordinator(
5860
throws Exception {
5961
this.context = context;
6062
this.provider = provider;
61-
this.coordinator = new DeferrableCoordinator(context.getOperatorId());
63+
this.coordinator = new DeferrableCoordinator(context.getOperatorId(), context.getJobID());
6264
this.coordinator.createNewInternalCoordinator(context, provider);
6365
this.coordinator.processPendingCalls();
6466
this.closingTimeoutMs = closingTimeoutMs;
@@ -132,7 +134,7 @@ public void resetToCheckpoint(final long checkpointId, @Nullable final byte[] ch
132134
// After this point all the subsequent calls will be made to the new coordinator.
133135
final DeferrableCoordinator oldCoordinator = coordinator;
134136
final DeferrableCoordinator newCoordinator =
135-
new DeferrableCoordinator(context.getOperatorId());
137+
new DeferrableCoordinator(context.getOperatorId(), context.getJobID());
136138
coordinator = newCoordinator;
137139
// Close the old coordinator asynchronously in a separate closing thread.
138140
// The future will be completed when the old coordinator closes.
@@ -327,26 +329,30 @@ private OperatorCoordinator.Context getContext() {
327329
private static class DeferrableCoordinator {
328330
private final OperatorID operatorId;
329331
private final BlockingQueue<NamedCall> pendingCalls;
332+
private final Map<String, String> mdc;
330333
private QuiesceableContext internalQuiesceableContext;
331334
private OperatorCoordinator internalCoordinator;
332335
private boolean hasCaughtUp;
333336
private boolean closed;
334337
private volatile boolean failed;
335338

336-
private DeferrableCoordinator(OperatorID operatorId) {
339+
private DeferrableCoordinator(OperatorID operatorId, JobID jobID) {
337340
this.operatorId = operatorId;
338341
this.pendingCalls = new LinkedBlockingQueue<>();
339342
this.hasCaughtUp = false;
340343
this.closed = false;
341344
this.failed = false;
345+
this.mdc = MdcUtils.asContextData(jobID);
342346
}
343347

344348
synchronized <T extends Exception> void applyCall(
345349
String name, ThrowingConsumer<OperatorCoordinator, T> call) throws T {
346350
synchronized (this) {
347351
if (hasCaughtUp) {
348352
// The new coordinator has caught up.
349-
call.accept(internalCoordinator);
353+
try (MdcUtils.MdcCloseable ignored = MdcUtils.withContext(mdc)) {
354+
call.accept(internalCoordinator);
355+
}
350356
} else {
351357
pendingCalls.add(new NamedCall(name, call));
352358
}

flink-tests/src/test/java/org/apache/flink/test/misc/JobIDLoggingITCase.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -149,9 +149,7 @@ void testJobIDLogging(@InjectClusterClient ClusterClient<?> clusterClient) throw
149149
sourceCoordinatorLogging,
150150
asList(
151151
"Starting split enumerator.*",
152-
"Distributing maxAllowedWatermark.*",
153-
"Source .* registering reader for parallel task.*",
154-
"Closing SourceCoordinator for source .*"));
152+
"Source .* registering reader for parallel task.*"));
155153

156154
assertJobIDPresent(
157155
jobID,

0 commit comments

Comments
 (0)