[hotfix] Run schema coordinator logic asynchronously to avoid blocking the main thread

pull/3582/head
yuxiqian 5 months ago committed by Leonard Xu
parent 9816848c4b
commit 1ebefd6d2e

@ -40,6 +40,7 @@ import java.util.Set;
import java.util.SortedMap; import java.util.SortedMap;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executors;
import java.util.stream.Collectors; import java.util.stream.Collectors;
/** Dummy classes for migration test. Called via reflection. */ /** Dummy classes for migration test. Called via reflection. */
@ -69,6 +70,7 @@ public class SchemaRegistryMigrationMock implements MigrationMockBase {
return new SchemaRegistry( return new SchemaRegistry(
"Dummy Name", "Dummy Name",
null, null,
Executors.newFixedThreadPool(1),
new MetadataApplier() { new MetadataApplier() {
@Override @Override
public boolean acceptsSchemaEvolutionType( public boolean acceptsSchemaEvolutionType(

@ -37,7 +37,9 @@ import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandle
import org.apache.flink.runtime.operators.coordination.CoordinationResponse; import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorEvent; import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkException;
import org.apache.flink.util.function.ThrowingRunnable;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -55,6 +57,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.wrap; import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.wrap;
@ -83,6 +86,9 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
/** The name of the operator this SchemaOperatorCoordinator is associated with. */ /** The name of the operator this SchemaOperatorCoordinator is associated with. */
private final String operatorName; private final String operatorName;
/** A single-thread executor to handle async execution of the coordinator. */
private final ExecutorService coordinatorExecutor;
/** /**
* Tracks the subtask failed reason to throw a more meaningful exception in {@link * Tracks the subtask failed reason to throw a more meaningful exception in {@link
* #subtaskReset}. * #subtaskReset}.
@ -113,18 +119,27 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
public SchemaRegistry( public SchemaRegistry(
String operatorName, String operatorName,
OperatorCoordinator.Context context, OperatorCoordinator.Context context,
ExecutorService executorService,
MetadataApplier metadataApplier, MetadataApplier metadataApplier,
List<RouteRule> routes) { List<RouteRule> routes) {
this(operatorName, context, metadataApplier, routes, SchemaChangeBehavior.EVOLVE); this(
operatorName,
context,
executorService,
metadataApplier,
routes,
SchemaChangeBehavior.LENIENT);
} }
public SchemaRegistry( public SchemaRegistry(
String operatorName, String operatorName,
OperatorCoordinator.Context context, OperatorCoordinator.Context context,
ExecutorService coordinatorExecutor,
MetadataApplier metadataApplier, MetadataApplier metadataApplier,
List<RouteRule> routes, List<RouteRule> routes,
SchemaChangeBehavior schemaChangeBehavior) { SchemaChangeBehavior schemaChangeBehavior) {
this.context = context; this.context = context;
this.coordinatorExecutor = coordinatorExecutor;
this.operatorName = operatorName; this.operatorName = operatorName;
this.failedReasons = new HashMap<>(); this.failedReasons = new HashMap<>();
this.metadataApplier = metadataApplier; this.metadataApplier = metadataApplier;
@ -133,7 +148,11 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
this.schemaDerivation = new SchemaDerivation(schemaManager, routes, new HashMap<>()); this.schemaDerivation = new SchemaDerivation(schemaManager, routes, new HashMap<>());
this.requestHandler = this.requestHandler =
new SchemaRegistryRequestHandler( new SchemaRegistryRequestHandler(
metadataApplier, schemaManager, schemaDerivation, schemaChangeBehavior); metadataApplier,
schemaManager,
schemaDerivation,
schemaChangeBehavior,
context);
this.schemaChangeBehavior = schemaChangeBehavior; this.schemaChangeBehavior = schemaChangeBehavior;
} }
@ -153,48 +172,87 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
} }
@Override @Override
public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) {
throws Exception { runInEventLoop(
try { () -> {
if (event instanceof FlushSuccessEvent) { try {
FlushSuccessEvent flushSuccessEvent = (FlushSuccessEvent) event; if (event instanceof FlushSuccessEvent) {
LOG.info( FlushSuccessEvent flushSuccessEvent = (FlushSuccessEvent) event;
"Sink subtask {} succeed flushing for table {}.", LOG.info(
flushSuccessEvent.getSubtask(), "Sink subtask {} succeed flushing for table {}.",
flushSuccessEvent.getTableId().toString()); flushSuccessEvent.getSubtask(),
requestHandler.flushSuccess( flushSuccessEvent.getTableId().toString());
flushSuccessEvent.getTableId(), requestHandler.flushSuccess(
flushSuccessEvent.getSubtask(), flushSuccessEvent.getTableId(),
currentParallelism); flushSuccessEvent.getSubtask(),
} else if (event instanceof SinkWriterRegisterEvent) { currentParallelism);
requestHandler.registerSinkWriter(((SinkWriterRegisterEvent) event).getSubtask()); } else if (event instanceof SinkWriterRegisterEvent) {
} else { requestHandler.registerSinkWriter(
throw new FlinkException("Unrecognized Operator Event: " + event); ((SinkWriterRegisterEvent) event).getSubtask());
} } else {
} catch (Throwable t) { throw new FlinkException("Unrecognized Operator Event: " + event);
context.failJob(t); }
throw t; } catch (Throwable t) {
} context.failJob(t);
throw t;
}
},
"handling event %s from subTask %d",
event,
subtask);
} }
@Override @Override
public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> resultFuture) public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> resultFuture) {
throws Exception { // we generate checkpoint in an async thread to not block the JobManager's main thread, the
try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); // coordinator state might be large if there are many schema changes and monitor many
DataOutputStream out = new DataOutputStream(baos)) { // tables.
// Serialize SchemaManager runInEventLoop(
int schemaManagerSerializerVersion = SchemaManager.SERIALIZER.getVersion(); () -> {
out.writeInt(schemaManagerSerializerVersion); try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
byte[] serializedSchemaManager = SchemaManager.SERIALIZER.serialize(schemaManager); DataOutputStream out = new DataOutputStream(baos)) {
out.writeInt(serializedSchemaManager.length); // Serialize SchemaManager
out.write(serializedSchemaManager); int schemaManagerSerializerVersion = SchemaManager.SERIALIZER.getVersion();
// Serialize SchemaDerivation mapping out.writeInt(schemaManagerSerializerVersion);
SchemaDerivation.serializeDerivationMapping(schemaDerivation, out); byte[] serializedSchemaManager =
resultFuture.complete(baos.toByteArray()); SchemaManager.SERIALIZER.serialize(schemaManager);
} catch (Throwable t) { out.writeInt(serializedSchemaManager.length);
context.failJob(t); out.write(serializedSchemaManager);
throw t; // Serialize SchemaDerivation mapping
} SchemaDerivation.serializeDerivationMapping(schemaDerivation, out);
resultFuture.complete(baos.toByteArray());
} catch (Throwable t) {
context.failJob(t);
throw t;
}
},
"taking checkpoint %d",
checkpointId);
}
private void runInEventLoop(
final ThrowingRunnable<Throwable> action,
final String actionName,
final Object... actionNameFormatParameters) {
coordinatorExecutor.execute(
() -> {
try {
action.run();
} 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 anymore
ExceptionUtils.rethrowIfFatalErrorOrOOM(t);
final String actionString =
String.format(actionName, actionNameFormatParameters);
LOG.error(
"Uncaught exception in the SchemaEvolutionCoordinator for {} while {}. Triggering job failover.",
operatorName,
actionString,
t);
context.failJob(t);
}
});
} }
@Override @Override
@ -205,26 +263,34 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
@Override @Override
public CompletableFuture<CoordinationResponse> handleCoordinationRequest( public CompletableFuture<CoordinationResponse> handleCoordinationRequest(
CoordinationRequest request) { CoordinationRequest request) {
try { CompletableFuture<CoordinationResponse> responseFuture = new CompletableFuture<>();
if (request instanceof SchemaChangeRequest) { runInEventLoop(
SchemaChangeRequest schemaChangeRequest = (SchemaChangeRequest) request; () -> {
return requestHandler.handleSchemaChangeRequest(schemaChangeRequest); try {
} else if (request instanceof SchemaChangeResultRequest) { if (request instanceof SchemaChangeRequest) {
return requestHandler.getSchemaChangeResult(); SchemaChangeRequest schemaChangeRequest = (SchemaChangeRequest) request;
} else if (request instanceof GetEvolvedSchemaRequest) { requestHandler.handleSchemaChangeRequest(
return CompletableFuture.completedFuture( schemaChangeRequest, responseFuture);
wrap(handleGetEvolvedSchemaRequest(((GetEvolvedSchemaRequest) request)))); } else if (request instanceof SchemaChangeResultRequest) {
} else if (request instanceof GetOriginalSchemaRequest) { requestHandler.getSchemaChangeResult(responseFuture);
return CompletableFuture.completedFuture( } else if (request instanceof GetEvolvedSchemaRequest) {
wrap(handleGetOriginalSchemaRequest((GetOriginalSchemaRequest) request))); handleGetEvolvedSchemaRequest(
} else { ((GetEvolvedSchemaRequest) request), responseFuture);
throw new IllegalArgumentException( } else if (request instanceof GetOriginalSchemaRequest) {
"Unrecognized CoordinationRequest type: " + request); handleGetOriginalSchemaRequest(
} (GetOriginalSchemaRequest) request, responseFuture);
} catch (Throwable t) { } else {
context.failJob(t); throw new IllegalArgumentException(
throw t; "Unrecognized CoordinationRequest type: " + request);
} }
} catch (Throwable t) {
context.failJob(t);
throw t;
}
},
"handling coordination request %s",
request);
return responseFuture;
} }
@Override @Override
@ -253,7 +319,8 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
metadataApplier, metadataApplier,
schemaManager, schemaManager,
schemaDerivation, schemaDerivation,
schemaManager.getBehavior()); schemaManager.getBehavior(),
context);
break; break;
} }
case 1: case 1:
@ -274,7 +341,8 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
metadataApplier, metadataApplier,
schemaManager, schemaManager,
schemaDerivation, schemaDerivation,
schemaChangeBehavior); schemaChangeBehavior,
context);
break; break;
} }
default: default:
@ -307,46 +375,56 @@ public class SchemaRegistry implements OperatorCoordinator, CoordinationRequestH
// do nothing // do nothing
} }
private GetEvolvedSchemaResponse handleGetEvolvedSchemaRequest( private void handleGetEvolvedSchemaRequest(
GetEvolvedSchemaRequest getEvolvedSchemaRequest) { GetEvolvedSchemaRequest getEvolvedSchemaRequest,
CompletableFuture<CoordinationResponse> response) {
LOG.info("Handling evolved schema request: {}", getEvolvedSchemaRequest); LOG.info("Handling evolved schema request: {}", getEvolvedSchemaRequest);
int schemaVersion = getEvolvedSchemaRequest.getSchemaVersion(); int schemaVersion = getEvolvedSchemaRequest.getSchemaVersion();
TableId tableId = getEvolvedSchemaRequest.getTableId(); TableId tableId = getEvolvedSchemaRequest.getTableId();
if (schemaVersion == GetEvolvedSchemaRequest.LATEST_SCHEMA_VERSION) { if (schemaVersion == GetEvolvedSchemaRequest.LATEST_SCHEMA_VERSION) {
return new GetEvolvedSchemaResponse( response.complete(
schemaManager.getLatestEvolvedSchema(tableId).orElse(null)); wrap(
new GetEvolvedSchemaResponse(
schemaManager.getLatestEvolvedSchema(tableId).orElse(null))));
} else { } else {
try { try {
return new GetEvolvedSchemaResponse( response.complete(
schemaManager.getEvolvedSchema(tableId, schemaVersion)); wrap(
new GetEvolvedSchemaResponse(
schemaManager.getEvolvedSchema(tableId, schemaVersion))));
} catch (IllegalArgumentException iae) { } catch (IllegalArgumentException iae) {
LOG.warn( LOG.warn(
"Some client is requesting an non-existed evolved schema for table {} with version {}", "Some client is requesting an non-existed evolved schema for table {} with version {}",
tableId, tableId,
schemaVersion); schemaVersion);
return new GetEvolvedSchemaResponse(null); response.complete(wrap(new GetEvolvedSchemaResponse(null)));
} }
} }
} }
private GetOriginalSchemaResponse handleGetOriginalSchemaRequest( private void handleGetOriginalSchemaRequest(
GetOriginalSchemaRequest getOriginalSchemaRequest) { GetOriginalSchemaRequest getOriginalSchemaRequest,
CompletableFuture<CoordinationResponse> response) {
LOG.info("Handling original schema request: {}", getOriginalSchemaRequest); LOG.info("Handling original schema request: {}", getOriginalSchemaRequest);
int schemaVersion = getOriginalSchemaRequest.getSchemaVersion(); int schemaVersion = getOriginalSchemaRequest.getSchemaVersion();
TableId tableId = getOriginalSchemaRequest.getTableId(); TableId tableId = getOriginalSchemaRequest.getTableId();
if (schemaVersion == GetOriginalSchemaRequest.LATEST_SCHEMA_VERSION) { if (schemaVersion == GetOriginalSchemaRequest.LATEST_SCHEMA_VERSION) {
return new GetOriginalSchemaResponse( response.complete(
schemaManager.getLatestOriginalSchema(tableId).orElse(null)); wrap(
new GetOriginalSchemaResponse(
schemaManager.getLatestOriginalSchema(tableId).orElse(null))));
} else { } else {
try { try {
return new GetOriginalSchemaResponse( response.complete(
schemaManager.getOriginalSchema(tableId, schemaVersion)); wrap(
new GetOriginalSchemaResponse(
schemaManager.getOriginalSchema(tableId, schemaVersion))));
} catch (IllegalArgumentException iae) { } catch (IllegalArgumentException iae) {
LOG.warn( LOG.warn(
"Some client is requesting an non-existed original schema for table {} with version {}", "Some client is requesting an non-existed original schema for table {} with version {}",
tableId, tableId,
schemaVersion); schemaVersion);
return new GetOriginalSchemaResponse(null); response.complete(wrap(new GetOriginalSchemaResponse(null)));
} }
} }
} }

@ -23,8 +23,12 @@ import org.apache.flink.cdc.common.route.RouteRule;
import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.common.sink.MetadataApplier;
import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.util.FatalExitExceptionHandler;
import java.util.List; import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ThreadFactory;
/** Provider of {@link SchemaRegistry}. */ /** Provider of {@link SchemaRegistry}. */
@Internal @Internal
@ -57,7 +61,55 @@ public class SchemaRegistryProvider implements OperatorCoordinator.Provider {
@Override @Override
public OperatorCoordinator create(OperatorCoordinator.Context context) throws Exception { public OperatorCoordinator create(OperatorCoordinator.Context context) throws Exception {
CoordinatorExecutorThreadFactory coordinatorThreadFactory =
new CoordinatorExecutorThreadFactory(
"schema-evolution-coordinator", context.getUserCodeClassloader());
ExecutorService coordinatorExecutor =
Executors.newSingleThreadExecutor(coordinatorThreadFactory);
return new SchemaRegistry( return new SchemaRegistry(
operatorName, context, metadataApplier, routingRules, schemaChangeBehavior); operatorName,
context,
coordinatorExecutor,
metadataApplier,
routingRules,
schemaChangeBehavior);
}
/** A thread factory class that provides some helper methods. */
public static class CoordinatorExecutorThreadFactory implements ThreadFactory {
private final String coordinatorThreadName;
private final ClassLoader cl;
private final Thread.UncaughtExceptionHandler errorHandler;
private Thread t;
CoordinatorExecutorThreadFactory(
final String coordinatorThreadName, final ClassLoader contextClassLoader) {
this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE);
}
CoordinatorExecutorThreadFactory(
final String coordinatorThreadName,
final ClassLoader contextClassLoader,
final Thread.UncaughtExceptionHandler errorHandler) {
this.coordinatorThreadName = coordinatorThreadName;
this.cl = contextClassLoader;
this.errorHandler = errorHandler;
}
@Override
public synchronized Thread newThread(Runnable r) {
if (t != null) {
throw new Error(
"This indicates that a fatal error has happened and caused the "
+ "coordinator executor thread to exit. Check the earlier logs"
+ "to see the root cause of the problem.");
}
t = new Thread(r, coordinatorThreadName);
t.setContextClassLoader(cl);
t.setUncaughtExceptionHandler(errorHandler);
return t;
}
} }
} }

@ -38,6 +38,7 @@ import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeRequest;
import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeResponse;
import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeResultResponse; import org.apache.flink.cdc.runtime.operators.schema.event.SchemaChangeResultResponse;
import org.apache.flink.runtime.operators.coordination.CoordinationResponse; import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -94,15 +95,19 @@ public class SchemaRegistryRequestHandler implements Closeable {
private final SchemaChangeBehavior schemaChangeBehavior; private final SchemaChangeBehavior schemaChangeBehavior;
private final OperatorCoordinator.Context context;
public SchemaRegistryRequestHandler( public SchemaRegistryRequestHandler(
MetadataApplier metadataApplier, MetadataApplier metadataApplier,
SchemaManager schemaManager, SchemaManager schemaManager,
SchemaDerivation schemaDerivation, SchemaDerivation schemaDerivation,
SchemaChangeBehavior schemaChangeBehavior) { SchemaChangeBehavior schemaChangeBehavior,
OperatorCoordinator.Context context) {
this.metadataApplier = metadataApplier; this.metadataApplier = metadataApplier;
this.schemaManager = schemaManager; this.schemaManager = schemaManager;
this.schemaDerivation = schemaDerivation; this.schemaDerivation = schemaDerivation;
this.schemaChangeBehavior = schemaChangeBehavior; this.schemaChangeBehavior = schemaChangeBehavior;
this.context = context;
this.activeSinkWriters = ConcurrentHashMap.newKeySet(); this.activeSinkWriters = ConcurrentHashMap.newKeySet();
this.flushedSinkWriters = ConcurrentHashMap.newKeySet(); this.flushedSinkWriters = ConcurrentHashMap.newKeySet();
@ -122,8 +127,8 @@ public class SchemaRegistryRequestHandler implements Closeable {
* *
* @param request the received SchemaChangeRequest * @param request the received SchemaChangeRequest
*/ */
public CompletableFuture<CoordinationResponse> handleSchemaChangeRequest( public void handleSchemaChangeRequest(
SchemaChangeRequest request) { SchemaChangeRequest request, CompletableFuture<CoordinationResponse> response) {
// We use requester subTask ID as the pending ticket, because there will be at most 1 schema // We use requester subTask ID as the pending ticket, because there will be at most 1 schema
// change requests simultaneously from each subTask // change requests simultaneously from each subTask
@ -156,7 +161,8 @@ public class SchemaRegistryRequestHandler implements Closeable {
if (!pendingSubTaskIds.contains(requestSubTaskId)) { if (!pendingSubTaskIds.contains(requestSubTaskId)) {
pendingSubTaskIds.add(requestSubTaskId); pendingSubTaskIds.add(requestSubTaskId);
} }
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.busy())); response.complete(wrap(SchemaChangeResponse.busy()));
return;
} }
SchemaChangeEvent event = request.getSchemaChangeEvent(); SchemaChangeEvent event = request.getSchemaChangeEvent();
@ -168,8 +174,8 @@ public class SchemaRegistryRequestHandler implements Closeable {
LOG.info( LOG.info(
"SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to duplicated request.", "SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to duplicated request.",
request); request);
return CompletableFuture.completedFuture( response.complete(wrap(SchemaChangeResponse.duplicate()));
wrap(SchemaChangeResponse.duplicate())); return;
} }
schemaManager.applyOriginalSchemaChange(event); schemaManager.applyOriginalSchemaChange(event);
List<SchemaChangeEvent> derivedSchemaChangeEvents = List<SchemaChangeEvent> derivedSchemaChangeEvents =
@ -183,7 +189,9 @@ public class SchemaRegistryRequestHandler implements Closeable {
LOG.info( LOG.info(
"SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to ignored request.", "SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to ignored request.",
request); request);
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.ignored()));
response.complete(wrap(SchemaChangeResponse.ignored()));
return;
} }
LOG.info( LOG.info(
@ -191,8 +199,8 @@ public class SchemaRegistryRequestHandler implements Closeable {
// This request has been accepted. // This request has been accepted.
schemaChangeStatus = RequestStatus.WAITING_FOR_FLUSH; schemaChangeStatus = RequestStatus.WAITING_FOR_FLUSH;
currentDerivedSchemaChangeEvents = new ArrayList<>(derivedSchemaChangeEvents); currentDerivedSchemaChangeEvents = new ArrayList<>(derivedSchemaChangeEvents);
return CompletableFuture.completedFuture(
wrap(SchemaChangeResponse.accepted(derivedSchemaChangeEvents))); response.complete(wrap(SchemaChangeResponse.accepted(derivedSchemaChangeEvents)));
} else { } else {
LOG.info( LOG.info(
"Schema Registry is busy processing a schema change request, could not handle request {} for now. Added {} to pending list ({}).", "Schema Registry is busy processing a schema change request, could not handle request {} for now. Added {} to pending list ({}).",
@ -202,7 +210,7 @@ public class SchemaRegistryRequestHandler implements Closeable {
if (!pendingSubTaskIds.contains(requestSubTaskId)) { if (!pendingSubTaskIds.contains(requestSubTaskId)) {
pendingSubTaskIds.add(requestSubTaskId); pendingSubTaskIds.add(requestSubTaskId);
} }
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.busy())); response.complete(wrap(SchemaChangeResponse.busy()));
} }
} }
} }
@ -299,7 +307,7 @@ public class SchemaRegistryRequestHandler implements Closeable {
} }
} }
public CompletableFuture<CoordinationResponse> getSchemaChangeResult() { public void getSchemaChangeResult(CompletableFuture<CoordinationResponse> response) {
Preconditions.checkState( Preconditions.checkState(
schemaChangeStatus != RequestStatus.IDLE, schemaChangeStatus != RequestStatus.IDLE,
"Illegal schemaChangeStatus: should not be IDLE before getting schema change request results."); "Illegal schemaChangeStatus: should not be IDLE before getting schema change request results.");
@ -311,11 +319,12 @@ public class SchemaRegistryRequestHandler implements Closeable {
// This request has been finished, return it and prepare for the next request // This request has been finished, return it and prepare for the next request
List<SchemaChangeEvent> finishedEvents = clearCurrentSchemaChangeRequest(); List<SchemaChangeEvent> finishedEvents = clearCurrentSchemaChangeRequest();
return CompletableFuture.supplyAsync( SchemaChangeResultResponse resultResponse =
() -> wrap(new SchemaChangeResultResponse(finishedEvents))); new SchemaChangeResultResponse(finishedEvents);
response.complete(wrap(resultResponse));
} else { } else {
// Still working on schema change request, waiting it // Still working on schema change request, waiting it
return CompletableFuture.supplyAsync(() -> wrap(new SchemaChangeProcessingResponse())); response.complete(wrap(new SchemaChangeProcessingResponse()));
} }
} }
@ -444,7 +453,8 @@ public class SchemaRegistryRequestHandler implements Closeable {
private List<SchemaChangeEvent> clearCurrentSchemaChangeRequest() { private List<SchemaChangeEvent> clearCurrentSchemaChangeRequest() {
if (currentChangeException != null) { if (currentChangeException != null) {
throw new RuntimeException("Failed to apply schema change.", currentChangeException); context.failJob(
new RuntimeException("Failed to apply schema change.", currentChangeException));
} }
List<SchemaChangeEvent> finishedSchemaChanges = List<SchemaChangeEvent> finishedSchemaChanges =
new ArrayList<>(currentFinishedSchemaChanges); new ArrayList<>(currentFinishedSchemaChanges);

@ -30,6 +30,7 @@ import org.apache.flink.cdc.common.event.RenameColumnEvent;
import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.SchemaChangeEventType;
import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily;
import org.apache.flink.cdc.common.event.TableId; import org.apache.flink.cdc.common.event.TableId;
import org.apache.flink.cdc.common.exceptions.UnsupportedSchemaChangeEventException;
import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior; import org.apache.flink.cdc.common.pipeline.SchemaChangeBehavior;
import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.schema.Column;
import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.schema.Schema;
@ -1035,11 +1036,16 @@ public class SchemaEvolveTest {
new AddColumnEvent.ColumnWithPosition( new AddColumnEvent.ColumnWithPosition(
Column.physicalColumn( Column.physicalColumn(
"height", DOUBLE, "Height data"))))); "height", DOUBLE, "Height data")))));
Assertions.assertThatThrownBy(() -> processEvent(schemaOperator, addColumnEvents)) processEvent(schemaOperator, addColumnEvents);
Assertions.assertThat(harness.isJobFailed()).isEqualTo(true);
Assertions.assertThat(harness.getJobFailureCause())
.cause() .cause()
.cause() .isExactlyInstanceOf(UnsupportedSchemaChangeEventException.class)
.isExactlyInstanceOf(RuntimeException.class) .matches(
.hasMessageContaining("Failed to apply schema change"); e ->
((UnsupportedSchemaChangeEventException) e)
.getExceptionMessage()
.equals("Sink doesn't support such schema change event."));
harness.close(); harness.close();
} }

@ -38,7 +38,6 @@ import org.apache.flink.cdc.runtime.testutils.schema.TestingSchemaRegistryGatewa
import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway;
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.Output;
@ -56,6 +55,7 @@ import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Executors;
import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.unwrap; import static org.apache.flink.cdc.runtime.operators.schema.event.CoordinationResponseUtils.unwrap;
@ -81,6 +81,7 @@ public class EventOperatorTestHarness<OP extends AbstractStreamOperator<E>, E ex
private final SchemaRegistry schemaRegistry; private final SchemaRegistry schemaRegistry;
private final TestingSchemaRegistryGateway schemaRegistryGateway; private final TestingSchemaRegistryGateway schemaRegistryGateway;
private final LinkedList<StreamRecord<E>> outputRecords = new LinkedList<>(); private final LinkedList<StreamRecord<E>> outputRecords = new LinkedList<>();
private final MockedOperatorCoordinatorContext mockedContext;
public EventOperatorTestHarness(OP operator, int numOutputs) { public EventOperatorTestHarness(OP operator, int numOutputs) {
this(operator, numOutputs, null, SchemaChangeBehavior.EVOLVE); this(operator, numOutputs, null, SchemaChangeBehavior.EVOLVE);
@ -94,11 +95,14 @@ public class EventOperatorTestHarness<OP extends AbstractStreamOperator<E>, E ex
OP operator, int numOutputs, Duration duration, SchemaChangeBehavior behavior) { OP operator, int numOutputs, Duration duration, SchemaChangeBehavior behavior) {
this.operator = operator; this.operator = operator;
this.numOutputs = numOutputs; this.numOutputs = numOutputs;
this.mockedContext =
new MockedOperatorCoordinatorContext(
SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader());
schemaRegistry = schemaRegistry =
new SchemaRegistry( new SchemaRegistry(
"SchemaOperator", "SchemaOperator",
new MockOperatorCoordinatorContext( mockedContext,
SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader()), Executors.newFixedThreadPool(1),
new CollectingMetadataApplier(duration), new CollectingMetadataApplier(duration),
new ArrayList<>(), new ArrayList<>(),
behavior); behavior);
@ -113,11 +117,14 @@ public class EventOperatorTestHarness<OP extends AbstractStreamOperator<E>, E ex
Set<SchemaChangeEventType> enabledEventTypes) { Set<SchemaChangeEventType> enabledEventTypes) {
this.operator = operator; this.operator = operator;
this.numOutputs = numOutputs; this.numOutputs = numOutputs;
this.mockedContext =
new MockedOperatorCoordinatorContext(
SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader());
schemaRegistry = schemaRegistry =
new SchemaRegistry( new SchemaRegistry(
"SchemaOperator", "SchemaOperator",
new MockOperatorCoordinatorContext( mockedContext,
SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader()), Executors.newFixedThreadPool(1),
new CollectingMetadataApplier(duration, enabledEventTypes), new CollectingMetadataApplier(duration, enabledEventTypes),
new ArrayList<>(), new ArrayList<>(),
behavior); behavior);
@ -133,11 +140,14 @@ public class EventOperatorTestHarness<OP extends AbstractStreamOperator<E>, E ex
Set<SchemaChangeEventType> errorsOnEventTypes) { Set<SchemaChangeEventType> errorsOnEventTypes) {
this.operator = operator; this.operator = operator;
this.numOutputs = numOutputs; this.numOutputs = numOutputs;
this.mockedContext =
new MockedOperatorCoordinatorContext(
SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader());
schemaRegistry = schemaRegistry =
new SchemaRegistry( new SchemaRegistry(
"SchemaOperator", "SchemaOperator",
new MockOperatorCoordinatorContext( mockedContext,
SCHEMA_OPERATOR_ID, Thread.currentThread().getContextClassLoader()), Executors.newFixedThreadPool(1),
new CollectingMetadataApplier( new CollectingMetadataApplier(
duration, enabledEventTypes, errorsOnEventTypes), duration, enabledEventTypes, errorsOnEventTypes),
new ArrayList<>(), new ArrayList<>(),
@ -196,6 +206,14 @@ public class EventOperatorTestHarness<OP extends AbstractStreamOperator<E>, E ex
.orElse(null); .orElse(null);
} }
public boolean isJobFailed() {
return mockedContext.isJobFailed();
}
public Throwable getJobFailureCause() {
return mockedContext.getFailureCause();
}
@Override @Override
public void close() throws Exception { public void close() throws Exception {
operator.close(); operator.close();

@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.cdc.runtime.testutils.operators;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
/**
* This is a mocked version of Operator coordinator context that stores failure cause for testing
* purposes only.
*/
public class MockedOperatorCoordinatorContext extends MockOperatorCoordinatorContext {
public MockedOperatorCoordinatorContext(
OperatorID operatorID, ClassLoader userCodeClassLoader) {
super(operatorID, userCodeClassLoader);
}
private Throwable failureCause;
@Override
public void failJob(Throwable cause) {
super.failJob(cause);
failureCause = cause;
}
public Throwable getFailureCause() {
return failureCause;
}
}
Loading…
Cancel
Save