Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.flink.cdc.runtime.operators.schema.regular;

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.cdc.common.event.CreateTableEvent;
import org.apache.flink.cdc.common.event.SchemaChangeEvent;
import org.apache.flink.cdc.common.event.TableId;
Expand Down Expand Up @@ -62,6 +63,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;

import static org.apache.flink.cdc.runtime.operators.schema.common.CoordinationResponseUtils.wrap;
Expand All @@ -74,15 +76,22 @@ public class SchemaCoordinator extends SchemaRegistry {
private final ExecutorService schemaChangeThreadPool;

/**
* Atomic flag indicating if current RequestHandler could accept more schema changes for now.
* Sink writers which have sent flush success events for the request.<br>
* {@code MapEntry.Key} is an {@code Integer}, indicating the upstream subTaskId that initiates
* this schema change request.<br>
* {@code MapEntry.Value} is a {@code Set<Integer>}, containing downstream subTasks' ID that
* have acknowledged and successfully flushed pending events for this schema change event.
*/
private transient RequestStatus schemaChangeStatus;

/** Sink writers which have sent flush success events for the request. */
private transient ConcurrentHashMap<Integer, Set<Integer>> flushedSinkWriters;

/** Currently handling request's completable future. */
private transient CompletableFuture<CoordinationResponse> pendingResponseFuture;
/**
* Schema evolution requests that we're currently handling.<br>
* For each subTask executing in parallelism, they may initiate requests simultaneously, so we
* use each task's unique subTaskId as Map key to distinguish each of them.
*/
private transient Map<
Integer, Tuple2<SchemaChangeRequest, CompletableFuture<CoordinationResponse>>>
pendingRequests;

// Static fields
public SchemaCoordinator(
Expand All @@ -108,7 +117,7 @@ public SchemaCoordinator(
public void start() throws Exception {
super.start();
this.flushedSinkWriters = new ConcurrentHashMap<>();
this.schemaChangeStatus = RequestStatus.IDLE;
this.pendingRequests = new ConcurrentHashMap<>();
}

@Override
Expand Down Expand Up @@ -185,7 +194,7 @@ protected void handleCustomCoordinationRequest(
}

@Override
protected void handleFlushSuccessEvent(FlushSuccessEvent event) {
protected void handleFlushSuccessEvent(FlushSuccessEvent event) throws TimeoutException {
int sinkSubtask = event.getSinkSubTaskId();
int sourceSubtask = event.getSourceSubTaskId();
LOG.info(
Expand All @@ -200,16 +209,25 @@ protected void handleFlushSuccessEvent(FlushSuccessEvent event) {
"Currently flushed sink writers for source task {} are: {}",
sourceSubtask,
flushedSinkWriters.get(sourceSubtask));

if (flushedSinkWriters.get(sourceSubtask).size() >= currentParallelism) {
LOG.info(
"Source SubTask {} have collected enough flush success event. Will start evolving schema changes...",
sourceSubtask);
flushedSinkWriters.remove(sourceSubtask);
startSchemaChangesEvolve(sourceSubtask);
}
}

@Override
protected void handleUnrecoverableError(String taskDescription, Throwable t) {
super.handleUnrecoverableError(taskDescription, t);

// There's a pending future, release it exceptionally before quitting
if (pendingResponseFuture != null) {
pendingResponseFuture.completeExceptionally(t);
}
// For each pending future, release it exceptionally before quitting
pendingRequests.forEach(
(index, tuple) -> {
tuple.f1.completeExceptionally(t);
});
}

/**
Expand All @@ -219,73 +237,14 @@ protected void handleUnrecoverableError(String taskDescription, Throwable t) {
*/
public void handleSchemaChangeRequest(
SchemaChangeRequest request, CompletableFuture<CoordinationResponse> responseFuture) {

// We use subTaskId to identify each schema change request
int subTaskId = request.getSubTaskId();

if (schemaChangeStatus == RequestStatus.IDLE) {
if (activeSinkWriters.size() < currentParallelism) {
LOG.info(
"Not all active sink writers have been registered. Current {}, expected {}.",
activeSinkWriters.size(),
currentParallelism);
responseFuture.complete(wrap(SchemaChangeResponse.waitingForFlush()));
return;
}

if (!activeSinkWriters.equals(flushedSinkWriters.get(subTaskId))) {
LOG.info(
"Not all active sink writers have completed flush. Flushed writers: {}, expected: {}.",
flushedSinkWriters.get(subTaskId),
activeSinkWriters);
responseFuture.complete(wrap(SchemaChangeResponse.waitingForFlush()));
return;
}

LOG.info(
"All sink writers have flushed for subTaskId {}. Switching to APPLYING state and starting schema evolution...",
subTaskId);
flushedSinkWriters.remove(subTaskId);
schemaChangeStatus = RequestStatus.APPLYING;
pendingResponseFuture = responseFuture;
startSchemaChangesEvolve(request, responseFuture);
} else {
responseFuture.complete(wrap(SchemaChangeResponse.busy()));
}
pendingRequests.put(request.getSubTaskId(), Tuple2.of(request, responseFuture));
}

private void startSchemaChangesEvolve(
SchemaChangeRequest request, CompletableFuture<CoordinationResponse> responseFuture) {
SchemaChangeEvent originalEvent = request.getSchemaChangeEvent();
TableId originalTableId = originalEvent.tableId();
Schema currentUpstreamSchema =
schemaManager.getLatestOriginalSchema(originalTableId).orElse(null);

List<SchemaChangeEvent> deducedSchemaChangeEvents = new ArrayList<>();

// For redundant schema change events (possibly coming from duplicate emitted
// CreateTableEvents in snapshot stage), we just skip them.
if (!SchemaUtils.isSchemaChangeEventRedundant(currentUpstreamSchema, originalEvent)) {
schemaManager.applyOriginalSchemaChange(originalEvent);
deducedSchemaChangeEvents.addAll(deduceEvolvedSchemaChanges(originalEvent));
} else {
LOG.info(
"Schema change event {} is redundant for current schema {}, just skip it.",
originalEvent,
currentUpstreamSchema);
}

LOG.info(
"All sink subtask have flushed for table {}. Start to apply schema change request: \n\t{}\nthat extracts to:\n\t{}",
request.getTableId().toString(),
request,
deducedSchemaChangeEvents.stream()
.map(SchemaChangeEvent::toString)
.collect(Collectors.joining("\n\t")));
private void startSchemaChangesEvolve(int sourceSubTaskId) {
schemaChangeThreadPool.submit(
() -> {
try {
applySchemaChange(originalEvent, deducedSchemaChangeEvents);
applySchemaChange(sourceSubTaskId);
} catch (Throwable t) {
failJob(
"Schema change applying task",
Expand Down Expand Up @@ -379,8 +338,54 @@ private List<SchemaChangeEvent> deduceEvolvedSchemaChanges(SchemaChangeEvent eve
}

/** Applies the schema change to the external system. */
private void applySchemaChange(
SchemaChangeEvent originalEvent, List<SchemaChangeEvent> deducedSchemaChangeEvents) {
private void applySchemaChange(int sourceSubTaskId) {
try {
loopUntil(
() -> pendingRequests.containsKey(sourceSubTaskId),
() ->
LOG.info(
"SchemaOperator {} has not submitted schema change request yet. Waiting...",
sourceSubTaskId),
rpcTimeout,
Duration.ofMillis(100));
} catch (TimeoutException e) {
throw new RuntimeException(
"Timeout waiting for schema change request from SchemaOperator.", e);
}

Tuple2<SchemaChangeRequest, CompletableFuture<CoordinationResponse>> requestBody =
pendingRequests.get(sourceSubTaskId);
SchemaChangeRequest request = requestBody.f0;
CompletableFuture<CoordinationResponse> responseFuture = requestBody.f1;

SchemaChangeEvent originalEvent = request.getSchemaChangeEvent();

TableId originalTableId = originalEvent.tableId();
Schema currentUpstreamSchema =
schemaManager.getLatestOriginalSchema(originalTableId).orElse(null);

List<SchemaChangeEvent> deducedSchemaChangeEvents = new ArrayList<>();

// For redundant schema change events (possibly coming from duplicate emitted
// CreateTableEvents in snapshot stage), we just skip them.
if (!SchemaUtils.isSchemaChangeEventRedundant(currentUpstreamSchema, originalEvent)) {
schemaManager.applyOriginalSchemaChange(originalEvent);
deducedSchemaChangeEvents.addAll(deduceEvolvedSchemaChanges(originalEvent));
} else {
LOG.info(
"Schema change event {} is redundant for current schema {}, just skip it.",
originalEvent,
currentUpstreamSchema);
}

LOG.info(
"All sink subtask have flushed for table {}. Start to apply schema change request: \n\t{}\nthat extracts to:\n\t{}",
request.getTableId().toString(),
request,
deducedSchemaChangeEvents.stream()
.map(SchemaChangeEvent::toString)
.collect(Collectors.joining("\n\t")));

if (SchemaChangeBehavior.EXCEPTION.equals(behavior)) {
if (deducedSchemaChangeEvents.stream()
.anyMatch(evt -> !(evt instanceof CreateTableEvent))) {
Expand Down Expand Up @@ -415,18 +420,14 @@ private void applySchemaChange(
}

// And returns all successfully applied schema change events to SchemaOperator.
pendingResponseFuture.complete(
wrap(
SchemaChangeResponse.success(
appliedSchemaChangeEvents, refreshedEvolvedSchemas)));
pendingResponseFuture = null;

Preconditions.checkState(
schemaChangeStatus == RequestStatus.APPLYING,
"Illegal schemaChangeStatus state: should be APPLYING before applySchemaChange finishes, not "
+ schemaChangeStatus);
schemaChangeStatus = RequestStatus.IDLE;
LOG.info("SchemaChangeStatus switched from APPLYING to IDLE.");
responseFuture.complete(
wrap(new SchemaChangeResponse(appliedSchemaChangeEvents, refreshedEvolvedSchemas)));

pendingRequests.remove(sourceSubTaskId);
LOG.info(
"Finished handling schema change request from {}. Pending requests: {}",
sourceSubTaskId,
pendingRequests);
}

private boolean applyAndUpdateEvolvedSchemaChange(SchemaChangeEvent schemaChangeEvent) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,6 @@
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

import static org.apache.flink.cdc.common.pipeline.PipelineOptions.DEFAULT_SCHEMA_OPERATOR_RPC_TIMEOUT;

Expand Down Expand Up @@ -172,45 +171,26 @@ private void handleSchemaChangeEvent(SchemaChangeEvent originalEvent) throws Exc
// Then, queue to request schema change to SchemaCoordinator.
SchemaChangeResponse response = requestSchemaChange(tableId, originalEvent);

if (response.isSuccess()) {
LOG.info("{}> Successfully requested schema change.", subTaskId);
LOG.info(
"{}> Finished schema change events: {}",
subTaskId,
response.getAppliedSchemaChangeEvents());
LOG.info("{}> Refreshed evolved schemas: {}", subTaskId, response.getEvolvedSchemas());
LOG.info("{}> Successfully requested schema change.", subTaskId);
LOG.info(
"{}> Finished schema change events: {}",
subTaskId,
response.getAppliedSchemaChangeEvents());
LOG.info("{}> Refreshed evolved schemas: {}", subTaskId, response.getEvolvedSchemas());

// After this request got successfully applied to DBMS, we can...
List<SchemaChangeEvent> finishedSchemaChangeEvents =
response.getAppliedSchemaChangeEvents();
// After this request got successfully applied to DBMS, we can...
List<SchemaChangeEvent> finishedSchemaChangeEvents =
response.getAppliedSchemaChangeEvents();

// Update local evolved schema map's cache
evolvedSchemaMap.putAll(response.getEvolvedSchemas());
// Update local evolved schema map's cache
evolvedSchemaMap.putAll(response.getEvolvedSchemas());

// and emit the finished event to downstream
for (SchemaChangeEvent finishedEvent : finishedSchemaChangeEvents) {
output.collect(new StreamRecord<>(finishedEvent));
}

schemaOperatorMetrics.increaseFinishedSchemaChangeEvents(
finishedSchemaChangeEvents.size());
} else if (response.isDuplicate()) {
LOG.info(
"{}> Schema change event {} has been handled in another subTask already.",
subTaskId,
originalEvent);

schemaOperatorMetrics.increaseIgnoredSchemaChangeEvents(1);
} else if (response.isIgnored()) {
LOG.info(
"{}> Schema change event {} has been ignored. No schema evolution needed.",
subTaskId,
originalEvent);

schemaOperatorMetrics.increaseIgnoredSchemaChangeEvents(1);
} else {
throw new IllegalStateException("Unexpected response status: " + response);
// and emit the finished event to downstream
for (SchemaChangeEvent finishedEvent : finishedSchemaChangeEvents) {
output.collect(new StreamRecord<>(finishedEvent));
}

schemaOperatorMetrics.increaseFinishedSchemaChangeEvents(finishedSchemaChangeEvents.size());
}

private void handleDataChangeEvent(DataChangeEvent dataChangeEvent) {
Expand Down Expand Up @@ -243,31 +223,9 @@ private void handleDataChangeEvent(DataChangeEvent dataChangeEvent) {
}

private SchemaChangeResponse requestSchemaChange(
TableId tableId, SchemaChangeEvent schemaChangeEvent)
throws InterruptedException, TimeoutException {
long deadline = System.currentTimeMillis() + rpcTimeout.toMillis();
while (true) {
SchemaChangeResponse response =
sendRequestToCoordinator(
new SchemaChangeRequest(tableId, schemaChangeEvent, subTaskId));
if (System.currentTimeMillis() < deadline) {
if (response.isRegistryBusy()) {
LOG.info(
"{}> Schema Registry is busy now, waiting for next request...",
subTaskId);
Thread.sleep(1000);
} else if (response.isWaitingForFlush()) {
LOG.info(
"{}> Schema change event has not collected enough flush success events from writers, waiting...",
subTaskId);
Thread.sleep(1000);
} else {
return response;
}
} else {
throw new TimeoutException("Timeout when requesting schema change.");
}
}
TableId tableId, SchemaChangeEvent schemaChangeEvent) {
return sendRequestToCoordinator(
new SchemaChangeRequest(tableId, schemaChangeEvent, subTaskId));
}

private <REQUEST extends CoordinationRequest, RESPONSE extends CoordinationResponse>
Expand Down
Loading
Loading