-
Notifications
You must be signed in to change notification settings - Fork 2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-37278] Optimize regular schema evolution topology's performance #3912
Open
yuxiqian
wants to merge
2
commits into
apache:master
Choose a base branch
from
yuxiqian:FLINK-37278
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+93
−123
Open
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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; | ||
|
@@ -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; | ||
|
@@ -73,16 +75,13 @@ public class SchemaCoordinator extends SchemaRegistry { | |
/** Executor service to execute schema change. */ | ||
private final ExecutorService schemaChangeThreadPool; | ||
|
||
/** | ||
* Atomic flag indicating if current RequestHandler could accept more schema changes for now. | ||
*/ | ||
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; | ||
/** Currently handling requests' completable future. */ | ||
private transient Map< | ||
Integer, Tuple2<SchemaChangeRequest, CompletableFuture<CoordinationResponse>>> | ||
pendingRequests; | ||
|
||
// Static fields | ||
public SchemaCoordinator( | ||
|
@@ -108,7 +107,7 @@ public SchemaCoordinator( | |
public void start() throws Exception { | ||
super.start(); | ||
this.flushedSinkWriters = new ConcurrentHashMap<>(); | ||
this.schemaChangeStatus = RequestStatus.IDLE; | ||
this.pendingRequests = new ConcurrentHashMap<>(); | ||
} | ||
|
||
@Override | ||
|
@@ -185,7 +184,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( | ||
|
@@ -200,16 +199,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); | ||
}); | ||
} | ||
|
||
/** | ||
|
@@ -219,73 +227,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())); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Method |
||
} | ||
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", | ||
|
@@ -379,8 +328,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))) { | ||
|
@@ -415,18 +410,16 @@ private void applySchemaChange( | |
} | ||
|
||
// And returns all successfully applied schema change events to SchemaOperator. | ||
pendingResponseFuture.complete( | ||
responseFuture.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."); | ||
|
||
pendingRequests.remove(sourceSubTaskId); | ||
LOG.info( | ||
"Finished handling schema change request from {}. Pending requests: {}", | ||
sourceSubTaskId, | ||
pendingRequests); | ||
} | ||
|
||
private boolean applyAndUpdateEvolvedSchemaChange(SchemaChangeEvent schemaChangeEvent) { | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Method
SchemaChangeResponse.waitingForFlush()
can be removed. Also forResponseCode.WAITING_FOR_FLUSH
.