Skip to content

Commit 2b8ae22

Browse files
yuxiqianlinjc13
andcommitted
[FLINK-37578][cdc-runtime] Fix distributed schema registry exposes bad internal state accidentally
This closes apache#3972 Co-authored-by: linjc13 <linjc13@chinatelecom.cn>
1 parent d504357 commit 2b8ae22

File tree

4 files changed

+102
-30
lines changed

4 files changed

+102
-30
lines changed

flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaCoordinator.java

Lines changed: 62 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -55,12 +55,14 @@
5555
import java.time.Duration;
5656
import java.util.ArrayList;
5757
import java.util.Collection;
58+
import java.util.HashMap;
5859
import java.util.List;
5960
import java.util.Map;
6061
import java.util.Set;
6162
import java.util.concurrent.CompletableFuture;
6263
import java.util.concurrent.ConcurrentHashMap;
6364
import java.util.concurrent.ExecutorService;
65+
import java.util.concurrent.Executors;
6466
import java.util.concurrent.TimeoutException;
6567
import java.util.concurrent.atomic.AtomicReference;
6668
import java.util.stream.Collectors;
@@ -98,6 +100,9 @@ public class SchemaCoordinator extends SchemaRegistry {
98100
private transient Multimap<Tuple2<Integer, SchemaChangeEvent>, Integer>
99101
alreadyHandledSchemaChangeEvents;
100102

103+
/** Executor service to execute schema change. */
104+
private final ExecutorService schemaChangeThreadPool;
105+
101106
public SchemaCoordinator(
102107
String operatorName,
103108
OperatorCoordinator.Context context,
@@ -114,6 +119,7 @@ public SchemaCoordinator(
114119
routingRules,
115120
schemaChangeBehavior,
116121
rpcTimeout);
122+
this.schemaChangeThreadPool = Executors.newSingleThreadExecutor();
117123
}
118124

119125
// -----------------
@@ -131,6 +137,14 @@ public void start() throws Exception {
131137
"Started SchemaRegistry for {}. Parallelism: {}", operatorName, currentParallelism);
132138
}
133139

140+
@Override
141+
public void close() throws Exception {
142+
super.close();
143+
if (schemaChangeThreadPool != null && !schemaChangeThreadPool.isShutdown()) {
144+
schemaChangeThreadPool.shutdownNow();
145+
}
146+
}
147+
134148
// --------------------------
135149
// Checkpoint related methods
136150
// --------------------------
@@ -268,7 +282,20 @@ private void handleSchemaEvolveRequest(
268282
LOG.info(
269283
"Received the last required schema change request {}. Switching from WAITING_FOR_FLUSH to EVOLVING.",
270284
request);
271-
startSchemaChange();
285+
286+
schemaChangeThreadPool.submit(
287+
() -> {
288+
try {
289+
startSchemaChange();
290+
} catch (Throwable t) {
291+
failJob(
292+
"Schema change applying task",
293+
new FlinkRuntimeException(
294+
"Failed to apply schema change event.", t));
295+
throw new FlinkRuntimeException(
296+
"Failed to apply schema change event.", t);
297+
}
298+
});
272299
}
273300
}
274301

@@ -301,34 +328,56 @@ private void startSchemaChange() throws TimeoutException {
301328
LOG.info("All flushed. Going to evolve schema for pending requests: {}", pendingRequests);
302329
flushedSinkWriters.clear();
303330

304-
// Deduce what schema change events should be applied to sink table
305-
List<SchemaChangeEvent> deducedSchemaChangeEvents = deduceEvolvedSchemaChanges();
331+
// Deduce what schema change events should be applied to sink table, and affected sink
332+
// tables' schema
333+
Tuple2<Set<TableId>, List<SchemaChangeEvent>> deduceSummary = deduceEvolvedSchemaChanges();
306334

307335
// And tries to apply it to external system
308336
List<SchemaChangeEvent> successfullyAppliedSchemaChangeEvents = new ArrayList<>();
309-
for (SchemaChangeEvent appliedSchemaChangeEvent : deducedSchemaChangeEvents) {
337+
for (SchemaChangeEvent appliedSchemaChangeEvent : deduceSummary.f1) {
310338
if (applyAndUpdateEvolvedSchemaChange(appliedSchemaChangeEvent)) {
311339
successfullyAppliedSchemaChangeEvents.add(appliedSchemaChangeEvent);
312340
}
313341
}
314342

315-
// Then, we broadcast affected schema changes to mapper and release upstream
316-
pendingRequests.forEach(
317-
(subTaskId, tuple) -> {
318-
LOG.info("Coordinator finishes pending future from {}", subTaskId);
319-
tuple.f1.complete(
320-
wrap(new SchemaChangeResponse(successfullyAppliedSchemaChangeEvents)));
321-
});
343+
// Fetch refreshed view for affected tables. We can't rely on operator clients to do this
344+
// because it might not have a complete schema view after restoring from previous states.
345+
Set<TableId> affectedTableIds = deduceSummary.f0;
346+
Map<TableId, Schema> evolvedSchemaView = new HashMap<>();
347+
for (TableId tableId : affectedTableIds) {
348+
schemaManager
349+
.getLatestEvolvedSchema(tableId)
350+
.ifPresent(schema -> evolvedSchemaView.put(tableId, schema));
351+
}
352+
353+
List<Tuple2<SchemaChangeRequest, CompletableFuture<CoordinationResponse>>> futures =
354+
new ArrayList<>(pendingRequests.values());
322355

356+
// Restore coordinator internal states first...
323357
pendingRequests.clear();
324358

325359
LOG.info("Finished schema evolving. Switching from EVOLVING to IDLE.");
326360
Preconditions.checkState(
327361
evolvingStatus.compareAndSet(RequestStatus.EVOLVING, RequestStatus.IDLE),
328362
"RequestStatus should be EVOLVING when schema evolving finishes.");
363+
364+
// ... and broadcast affected schema changes to mapper and release upstream then.
365+
// Make sure we've cleaned-up internal state before this, or we may receive new requests in
366+
// a dirty state.
367+
futures.forEach(
368+
tuple -> {
369+
LOG.info(
370+
"Coordinator finishes pending future from {}",
371+
tuple.f0.getSinkSubTaskId());
372+
tuple.f1.complete(
373+
wrap(
374+
new SchemaChangeResponse(
375+
evolvedSchemaView,
376+
successfullyAppliedSchemaChangeEvents)));
377+
});
329378
}
330379

331-
private List<SchemaChangeEvent> deduceEvolvedSchemaChanges() {
380+
private Tuple2<Set<TableId>, List<SchemaChangeEvent>> deduceEvolvedSchemaChanges() {
332381
List<SchemaChangeRequest> validSchemaChangeRequests =
333382
pendingRequests.values().stream()
334383
.map(e -> e.f0)
@@ -408,7 +457,7 @@ private List<SchemaChangeEvent> deduceEvolvedSchemaChanges() {
408457
evolvedSchemaChanges.addAll(normalizedEvents);
409458
}
410459

411-
return evolvedSchemaChanges;
460+
return Tuple2.of(affectedSinkTableIds, evolvedSchemaChanges);
412461
}
413462

414463
private boolean applyAndUpdateEvolvedSchemaChange(SchemaChangeEvent schemaChangeEvent) {

flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java

Lines changed: 3 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -205,17 +205,11 @@ private void requestSchemaChange(
205205
LOG.info("{}> Evolve request response: {}", subTaskId, response);
206206

207207
// Update local evolved schema cache
208-
response.getSchemaEvolveResult()
209-
.forEach(
210-
schemaChangeEvent ->
211-
evolvedSchemaMap.compute(
212-
schemaChangeEvent.tableId(),
213-
(tableId, schema) ->
214-
SchemaUtils.applySchemaChangeEvent(
215-
schema, schemaChangeEvent)));
208+
evolvedSchemaMap.putAll(response.getEvolvedSchemas());
216209

217210
// And emit schema change events to downstream
218-
response.getSchemaEvolveResult().forEach(evt -> output.collect(new StreamRecord<>(evt)));
211+
response.getEvolvedSchemaChangeEvents()
212+
.forEach(evt -> output.collect(new StreamRecord<>(evt)));
219213
LOG.info(
220214
"{}> Successfully updated evolved schema cache. Current state: {}",
221215
subTaskId,

flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/event/SchemaChangeRequest.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,4 +65,16 @@ public SchemaChangeEvent getSchemaChangeEvent() {
6565
!isNoOpRequest(), "Unable to fetch source subTaskId for an align event.");
6666
return schemaChangeEvent;
6767
}
68+
69+
@Override
70+
public String toString() {
71+
return "SchemaChangeRequest{"
72+
+ "sourceSubTaskId="
73+
+ sourceSubTaskId
74+
+ ", sinkSubTaskId="
75+
+ sinkSubTaskId
76+
+ ", schemaChangeEvent="
77+
+ schemaChangeEvent
78+
+ '}';
79+
}
6880
}

flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/event/SchemaChangeResponse.java

Lines changed: 25 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -18,23 +18,34 @@
1818
package org.apache.flink.cdc.runtime.operators.schema.distributed.event;
1919

2020
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
21+
import org.apache.flink.cdc.common.event.TableId;
22+
import org.apache.flink.cdc.common.schema.Schema;
2123
import org.apache.flink.cdc.runtime.operators.schema.distributed.SchemaCoordinator;
2224
import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
2325

2426
import java.util.List;
27+
import java.util.Map;
2528
import java.util.Objects;
2629

2730
/** Response from a {@link SchemaCoordinator} to broadcast a coordination consensus. */
2831
public class SchemaChangeResponse implements CoordinationResponse {
2932

30-
private final List<SchemaChangeEvent> schemaEvolveResult;
33+
private final Map<TableId, Schema> evolvedSchemas;
34+
private final List<SchemaChangeEvent> evolvedSchemaChangeEvents;
3135

32-
public SchemaChangeResponse(List<SchemaChangeEvent> schemaEvolveResult) {
33-
this.schemaEvolveResult = schemaEvolveResult;
36+
public SchemaChangeResponse(
37+
Map<TableId, Schema> evolvedSchemas,
38+
List<SchemaChangeEvent> evolvedSchemaChangeEvents) {
39+
this.evolvedSchemas = evolvedSchemas;
40+
this.evolvedSchemaChangeEvents = evolvedSchemaChangeEvents;
3441
}
3542

36-
public List<SchemaChangeEvent> getSchemaEvolveResult() {
37-
return schemaEvolveResult;
43+
public Map<TableId, Schema> getEvolvedSchemas() {
44+
return evolvedSchemas;
45+
}
46+
47+
public List<SchemaChangeEvent> getEvolvedSchemaChangeEvents() {
48+
return evolvedSchemaChangeEvents;
3849
}
3950

4051
@Override
@@ -43,16 +54,22 @@ public boolean equals(Object o) {
4354
return false;
4455
}
4556
SchemaChangeResponse that = (SchemaChangeResponse) o;
46-
return Objects.equals(schemaEvolveResult, that.schemaEvolveResult);
57+
return Objects.equals(evolvedSchemas, that.evolvedSchemas)
58+
&& Objects.equals(evolvedSchemaChangeEvents, that.evolvedSchemaChangeEvents);
4759
}
4860

4961
@Override
5062
public int hashCode() {
51-
return Objects.hash(schemaEvolveResult);
63+
return Objects.hash(evolvedSchemas, evolvedSchemaChangeEvents);
5264
}
5365

5466
@Override
5567
public String toString() {
56-
return "SchemaChangeResponse{" + "schemaEvolveResult=" + schemaEvolveResult + '}';
68+
return "SchemaChangeResponse{"
69+
+ "evolvedSchemas="
70+
+ evolvedSchemas
71+
+ ", evolvedSchemaChangeEvents="
72+
+ evolvedSchemaChangeEvents
73+
+ '}';
5774
}
5875
}

0 commit comments

Comments
 (0)