|
|
|
@ -54,7 +54,6 @@ import java.util.concurrent.CompletableFuture;
|
|
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
|
|
import java.util.concurrent.Executors;
|
|
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
import java.util.stream.Stream;
|
|
|
|
|
|
|
|
|
@ -77,7 +76,10 @@ public class SchemaRegistryRequestHandler implements Closeable {
|
|
|
|
|
/**
|
|
|
|
|
* Atomic flag indicating if current RequestHandler could accept more schema changes for now.
|
|
|
|
|
*/
|
|
|
|
|
private final AtomicReference<RequestStatus> schemaChangeStatus;
|
|
|
|
|
private volatile RequestStatus schemaChangeStatus;
|
|
|
|
|
|
|
|
|
|
private final List<Integer> pendingSubTaskIds;
|
|
|
|
|
private final Object schemaChangeRequestLock;
|
|
|
|
|
|
|
|
|
|
private volatile Throwable currentChangeException;
|
|
|
|
|
private volatile List<SchemaChangeEvent> currentDerivedSchemaChangeEvents;
|
|
|
|
@ -109,7 +111,10 @@ public class SchemaRegistryRequestHandler implements Closeable {
|
|
|
|
|
this.currentDerivedSchemaChangeEvents = new ArrayList<>();
|
|
|
|
|
this.currentFinishedSchemaChanges = new ArrayList<>();
|
|
|
|
|
this.currentIgnoredSchemaChanges = new ArrayList<>();
|
|
|
|
|
this.schemaChangeStatus = new AtomicReference<>(RequestStatus.IDLE);
|
|
|
|
|
|
|
|
|
|
this.schemaChangeStatus = RequestStatus.IDLE;
|
|
|
|
|
this.pendingSubTaskIds = new ArrayList<>();
|
|
|
|
|
this.schemaChangeRequestLock = new Object();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
@ -119,54 +124,86 @@ public class SchemaRegistryRequestHandler implements Closeable {
|
|
|
|
|
*/
|
|
|
|
|
public CompletableFuture<CoordinationResponse> handleSchemaChangeRequest(
|
|
|
|
|
SchemaChangeRequest request) {
|
|
|
|
|
if (schemaChangeStatus.compareAndSet(RequestStatus.IDLE, RequestStatus.WAITING_FOR_FLUSH)) {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"Received schema change event request {} from table {}. SchemaChangeStatus switched from IDLE to WAITING_FOR_FLUSH, other requests will be blocked.",
|
|
|
|
|
request.getSchemaChangeEvent(),
|
|
|
|
|
request.getTableId().toString());
|
|
|
|
|
SchemaChangeEvent event = request.getSchemaChangeEvent();
|
|
|
|
|
|
|
|
|
|
// If this schema change event has been requested by another subTask, ignore it.
|
|
|
|
|
if (schemaManager.isOriginalSchemaChangeEventRedundant(event)) {
|
|
|
|
|
LOG.info("Event {} has been addressed before, ignoring it.", event);
|
|
|
|
|
clearCurrentSchemaChangeRequest();
|
|
|
|
|
Preconditions.checkState(
|
|
|
|
|
schemaChangeStatus.compareAndSet(
|
|
|
|
|
RequestStatus.WAITING_FOR_FLUSH, RequestStatus.IDLE),
|
|
|
|
|
"Illegal schemaChangeStatus state: should still in WAITING_FOR_FLUSH state if event was duplicated, not "
|
|
|
|
|
+ schemaChangeStatus.get());
|
|
|
|
|
|
|
|
|
|
// We use requester subTask ID as the pending ticket, because there will be at most 1 schema
|
|
|
|
|
// change requests simultaneously from each subTask
|
|
|
|
|
int requestSubTaskId = request.getSubTaskId();
|
|
|
|
|
|
|
|
|
|
synchronized (schemaChangeRequestLock) {
|
|
|
|
|
// Make sure we handle the first request in the pending list to avoid out-of-order
|
|
|
|
|
// waiting and blocks checkpointing mechanism.
|
|
|
|
|
if (schemaChangeStatus == RequestStatus.IDLE) {
|
|
|
|
|
if (pendingSubTaskIds.isEmpty()) {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"Received schema change event request {} from table {} from subTask {}. Pending list is empty, handling this.",
|
|
|
|
|
request.getSchemaChangeEvent(),
|
|
|
|
|
request.getTableId().toString(),
|
|
|
|
|
requestSubTaskId);
|
|
|
|
|
} else if (pendingSubTaskIds.get(0) == requestSubTaskId) {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"Received schema change event request {} from table {} from subTask {}. It is on the first of the pending list, handling this.",
|
|
|
|
|
request.getSchemaChangeEvent(),
|
|
|
|
|
request.getTableId().toString(),
|
|
|
|
|
requestSubTaskId);
|
|
|
|
|
pendingSubTaskIds.remove(0);
|
|
|
|
|
} else {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"Received schema change event request {} from table {} from subTask {}. It is not the first of the pending list ({}).",
|
|
|
|
|
request.getSchemaChangeEvent(),
|
|
|
|
|
request.getTableId().toString(),
|
|
|
|
|
requestSubTaskId,
|
|
|
|
|
pendingSubTaskIds);
|
|
|
|
|
if (!pendingSubTaskIds.contains(requestSubTaskId)) {
|
|
|
|
|
pendingSubTaskIds.add(requestSubTaskId);
|
|
|
|
|
}
|
|
|
|
|
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.busy()));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
SchemaChangeEvent event = request.getSchemaChangeEvent();
|
|
|
|
|
|
|
|
|
|
// If this schema change event has been requested by another subTask, ignore it.
|
|
|
|
|
if (schemaManager.isOriginalSchemaChangeEventRedundant(event)) {
|
|
|
|
|
LOG.info("Event {} has been addressed before, ignoring it.", event);
|
|
|
|
|
clearCurrentSchemaChangeRequest();
|
|
|
|
|
LOG.info(
|
|
|
|
|
"SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to duplicated request.",
|
|
|
|
|
request);
|
|
|
|
|
return CompletableFuture.completedFuture(
|
|
|
|
|
wrap(SchemaChangeResponse.duplicate()));
|
|
|
|
|
}
|
|
|
|
|
schemaManager.applyOriginalSchemaChange(event);
|
|
|
|
|
List<SchemaChangeEvent> derivedSchemaChangeEvents =
|
|
|
|
|
calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent());
|
|
|
|
|
|
|
|
|
|
// If this schema change event is filtered out by LENIENT mode or merging table
|
|
|
|
|
// route strategies, ignore it.
|
|
|
|
|
if (derivedSchemaChangeEvents.isEmpty()) {
|
|
|
|
|
LOG.info("Event {} is omitted from sending to downstream, ignoring it.", event);
|
|
|
|
|
clearCurrentSchemaChangeRequest();
|
|
|
|
|
LOG.info(
|
|
|
|
|
"SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to ignored request.",
|
|
|
|
|
request);
|
|
|
|
|
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.ignored()));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG.info(
|
|
|
|
|
"SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to duplicated request.",
|
|
|
|
|
request);
|
|
|
|
|
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.duplicate()));
|
|
|
|
|
}
|
|
|
|
|
schemaManager.applyOriginalSchemaChange(event);
|
|
|
|
|
List<SchemaChangeEvent> derivedSchemaChangeEvents =
|
|
|
|
|
calculateDerivedSchemaChangeEvents(request.getSchemaChangeEvent());
|
|
|
|
|
|
|
|
|
|
// If this schema change event is filtered out by LENIENT mode or merging table route
|
|
|
|
|
// strategies, ignore it.
|
|
|
|
|
if (derivedSchemaChangeEvents.isEmpty()) {
|
|
|
|
|
LOG.info("Event {} is omitted from sending to downstream, ignoring it.", event);
|
|
|
|
|
clearCurrentSchemaChangeRequest();
|
|
|
|
|
Preconditions.checkState(
|
|
|
|
|
schemaChangeStatus.compareAndSet(
|
|
|
|
|
RequestStatus.WAITING_FOR_FLUSH, RequestStatus.IDLE),
|
|
|
|
|
"Illegal schemaChangeStatus state: should still in WAITING_FOR_FLUSH state if event was ignored, not "
|
|
|
|
|
+ schemaChangeStatus.get());
|
|
|
|
|
"SchemaChangeStatus switched from IDLE to WAITING_FOR_FLUSH, other requests will be blocked.");
|
|
|
|
|
// This request has been accepted.
|
|
|
|
|
schemaChangeStatus = RequestStatus.WAITING_FOR_FLUSH;
|
|
|
|
|
currentDerivedSchemaChangeEvents = new ArrayList<>(derivedSchemaChangeEvents);
|
|
|
|
|
return CompletableFuture.completedFuture(
|
|
|
|
|
wrap(SchemaChangeResponse.accepted(derivedSchemaChangeEvents)));
|
|
|
|
|
} else {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"SchemaChangeStatus switched from WAITING_FOR_FLUSH to IDLE for request {} due to ignored request.",
|
|
|
|
|
request);
|
|
|
|
|
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.ignored()));
|
|
|
|
|
"Schema Registry is busy processing a schema change request, could not handle request {} for now. Added {} to pending list ({}).",
|
|
|
|
|
request,
|
|
|
|
|
requestSubTaskId,
|
|
|
|
|
pendingSubTaskIds);
|
|
|
|
|
if (!pendingSubTaskIds.contains(requestSubTaskId)) {
|
|
|
|
|
pendingSubTaskIds.add(requestSubTaskId);
|
|
|
|
|
}
|
|
|
|
|
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.busy()));
|
|
|
|
|
}
|
|
|
|
|
currentDerivedSchemaChangeEvents = new ArrayList<>(derivedSchemaChangeEvents);
|
|
|
|
|
return CompletableFuture.completedFuture(
|
|
|
|
|
wrap(SchemaChangeResponse.accepted(derivedSchemaChangeEvents)));
|
|
|
|
|
} else {
|
|
|
|
|
LOG.info(
|
|
|
|
|
"Schema Registry is busy processing a schema change request, could not handle request {} for now.",
|
|
|
|
|
request);
|
|
|
|
|
return CompletableFuture.completedFuture(wrap(SchemaChangeResponse.busy()));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
@ -213,9 +250,10 @@ public class SchemaRegistryRequestHandler implements Closeable {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
Preconditions.checkState(
|
|
|
|
|
schemaChangeStatus.compareAndSet(RequestStatus.APPLYING, RequestStatus.FINISHED),
|
|
|
|
|
schemaChangeStatus == RequestStatus.APPLYING,
|
|
|
|
|
"Illegal schemaChangeStatus state: should be APPLYING before applySchemaChange finishes, not "
|
|
|
|
|
+ schemaChangeStatus.get());
|
|
|
|
|
+ schemaChangeStatus);
|
|
|
|
|
schemaChangeStatus = RequestStatus.FINISHED;
|
|
|
|
|
LOG.info(
|
|
|
|
|
"SchemaChangeStatus switched from APPLYING to FINISHED for request {}.",
|
|
|
|
|
currentDerivedSchemaChangeEvents);
|
|
|
|
@ -248,10 +286,11 @@ public class SchemaRegistryRequestHandler implements Closeable {
|
|
|
|
|
}
|
|
|
|
|
if (flushedSinkWriters.equals(activeSinkWriters)) {
|
|
|
|
|
Preconditions.checkState(
|
|
|
|
|
schemaChangeStatus.compareAndSet(
|
|
|
|
|
RequestStatus.WAITING_FOR_FLUSH, RequestStatus.APPLYING),
|
|
|
|
|
schemaChangeStatus == RequestStatus.WAITING_FOR_FLUSH,
|
|
|
|
|
"Illegal schemaChangeStatus state: should be WAITING_FOR_FLUSH before collecting enough FlushEvents, not "
|
|
|
|
|
+ schemaChangeStatus);
|
|
|
|
|
|
|
|
|
|
schemaChangeStatus = RequestStatus.APPLYING;
|
|
|
|
|
LOG.info(
|
|
|
|
|
"All sink subtask have flushed for table {}. Start to apply schema change.",
|
|
|
|
|
tableId.toString());
|
|
|
|
@ -262,9 +301,10 @@ public class SchemaRegistryRequestHandler implements Closeable {
|
|
|
|
|
|
|
|
|
|
public CompletableFuture<CoordinationResponse> getSchemaChangeResult() {
|
|
|
|
|
Preconditions.checkState(
|
|
|
|
|
!schemaChangeStatus.get().equals(RequestStatus.IDLE),
|
|
|
|
|
schemaChangeStatus != RequestStatus.IDLE,
|
|
|
|
|
"Illegal schemaChangeStatus: should not be IDLE before getting schema change request results.");
|
|
|
|
|
if (schemaChangeStatus.compareAndSet(RequestStatus.FINISHED, RequestStatus.IDLE)) {
|
|
|
|
|
if (schemaChangeStatus == RequestStatus.FINISHED) {
|
|
|
|
|
schemaChangeStatus = RequestStatus.IDLE;
|
|
|
|
|
LOG.info(
|
|
|
|
|
"SchemaChangeStatus switched from FINISHED to IDLE for request {}",
|
|
|
|
|
currentDerivedSchemaChangeEvents);
|
|
|
|
|