Flink CDC版本:3.2.1
说明:本文从SchemaOperator接收到,表结构变更事件开始,表结构变更事件应由source端产生,本文不讨论。
可以先看流程图,研究源码。
参考文章:
Flink cdc3.0动态变更表结构——源码解析
一、源码解析
以Sink to doris举例:
SchemaOperator
org.apache.flink.cdc.runtime.operators.schema.SchemaOperator
判断是否是SchemaChangeEvent
事件,调用processSchemaChangeEvents
方法
/**
* This method is guaranteed to not be called concurrently with other methods of the operator.
*/
@Override
public void processElement(StreamRecord<Event> streamRecord)
throws InterruptedException, TimeoutException, ExecutionException {
Event event = streamRecord.getValue();
if (event instanceof SchemaChangeEvent) {
// (0)
processSchemaChangeEvents((SchemaChangeEvent) event);
} else if (event instanceof DataChangeEvent) {
// (13)
processDataChangeEvents(streamRecord, (DataChangeEvent) event);
} else {
throw new RuntimeException("Unknown event type in Stream record: " + event);
}
}
调用handleSchemaChangeEvent
方法:
private void processSchemaChangeEvents(SchemaChangeEvent event)
throws InterruptedException, TimeoutException, ExecutionException {
TableId tableId = event.tableId();
LOG.info(
"{}> Table {} received SchemaChangeEvent {} and start to be blocked.",
subTaskId,
tableId,
event);
handleSchemaChangeEvent(tableId, event);
// Update caches
originalSchema.put(tableId, getLatestOriginalSchema(tableId));
schemaDivergesMap.put(tableId, checkSchemaDiverges(tableId));
List<TableId> optionalRoutedTable = getRoutedTables(tableId);
if (!optionalRoutedTable.isEmpty()) {
tableIdMappingCache
.get(tableId)
.forEach(routed -> evolvedSchema.put(routed, getLatestEvolvedSchema(routed)));
} else {
evolvedSchema.put(tableId, getLatestEvolvedSchema(tableId));
}
}
handleSchemaChangeEvent
调用requestSchemaChange
方法,请求修改Schema:
response.isAccepted()
就是注册中心接收了此修改需求。进入if后,重点来了:output.collect(new StreamRecord<>(new FlushEvent(tableId)));
。注意这里发送了一个new FlushEvent(tableId)
事件,这个事件会在SinkWriter
用到,就是通知SinkWriter
要执行flush
,即把数据刷入到sink端数据库,和jdbc的commit
相似。
FlushEvent内容非常简单只有tableId
但是其类型是FlushEvent
,此类的doc内容是:
- An {@link Event} from {@code SchemaOperator} to notify {@code DataSinkWriterOperator} that it
- start flushing.
也就是FlushEvent作为特殊数据传递事件,接收到此数据的DataSinkWriterOperator
会触发其执行flushing
操作,也就是将目前收到的所有数据都写入目标数据库。可以理解为:
schema修改后的数据 --> FlushEvent(新插入) --> schema修改前的数据
发送FlushEvent
事件后执行requestSchemaChangeResult
方法,此方法是while阻塞的方法,简而言之是等所有writer都完成了FlushEvent
前数据的(旧表结构的数据)写入前,一直阻塞不发送新表结构的数据至下游。
最后finishedSchemaChangeEvents.forEach(e -> output.collect(new StreamRecord<>(e)));
,简略的说其内部是handler方法中生成的SchemaRegistryRequestHandler#applySchemaChange
事件,将原始的SchemaChangeEvent转换成新的数据,还是根据Flink CDC的schema.change.behavior
转换,其类型如下:
![[image-20250106113512324.png]]
具体将这些时间发送至下游怎么用暂时没有研究了。
private void handleSchemaChangeEvent(TableId tableId, SchemaChangeEvent schemaChangeEvent)
throws InterruptedException, TimeoutException {
if (schemaChangeBehavior == SchemaChangeBehavior.EXCEPTION
&& schemaChangeEvent.getType() != SchemaChangeEventType.CREATE_TABLE) {
// CreateTableEvent should be applied even in EXCEPTION mode
throw new RuntimeException(
String.format(
"Refused to apply schema change event %s in EXCEPTION mode.",
schemaChangeEvent));
}
// The request will block if another schema change event is being handled
SchemaChangeResponse response = requestSchemaChange(tableId, schemaChangeEvent); // (1)
if (response.isAccepted()) {
// (3)
LOG.info("{}> Sending the FlushEvent for table {}.", subTaskId, tableId);
output.collect(new StreamRecord<>(new FlushEvent(tableId))); // (4)
List<SchemaChangeEvent> expectedSchemaChangeEvents = response.getSchemaChangeEvents();
schemaOperatorMetrics.increaseSchemaChangeEvents(expectedSchemaChangeEvents.size());
// The request will block until flushing finished in each sink writer
SchemaChangeResultResponse schemaEvolveResponse = requestSchemaChangeResult(); // (5)
List<SchemaChangeEvent> finishedSchemaChangeEvents =
schemaEvolveResponse.getFinishedSchemaChangeEvents();
// Update evolved schema changes based on apply results
finishedSchemaChangeEvents.forEach(e -> output.collect(new StreamRecord<>(e)));
} else if (response.isDuplicate()) {
LOG.info(
"{}> Schema change event {} has been handled in another subTask already.",
subTaskId,
schemaChangeEvent);
} else if (response.isIgnored()) {
LOG.info(
"{}> Schema change event {} has been ignored. No schema evolution needed.",
subTaskId,
schemaChangeEvent);
} else {
throw new IllegalStateException("Unexpected response status " + response);
}
}
requestSchemaChange
是一个阻塞的方法(while (true)
),发送SchemaChangeRequest
直到返回的response
不是Busy
。可以看到发送的的SchemaChangeRequest
。
private SchemaChangeResponse requestSchemaChange(
TableId tableId, SchemaChangeEvent schemaChangeEvent)
throws InterruptedException, TimeoutException {
long schemaEvolveTimeOutMillis = System.currentTimeMillis() + rpcTimeOutInMillis;
while (true) {
SchemaChangeResponse response =
sendRequestToCoordinator(
new SchemaChangeRequest(tableId, schemaChangeEvent, subTaskId));
if (response.isRegistryBusy()) {
// (2)
if (System.currentTimeMillis() < schemaEvolveTimeOutMillis) {
LOG.info(
"{}> Schema Registry is busy now, waiting for next request...",
subTaskId);
Thread.sleep(1000);
} else {
throw new TimeoutException("TimeOut when requesting schema change");
}
} else {
return response;
}
}
}
sendRequestToCoordinator
方法是org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway
类的,也就Flink的内部类。
实习类有:
(1)org.apache.flink.runtime.taskmanager.NoOpTaskOperatorEventGateway
(2)org.apache.flink.runtime.taskexecutor.rpc.RpcTaskOperatorEventGateway
内部具体逻辑暂不深入了解。
其实际发送至 SchemaRegistry#handleEventFromOperator
private <REQUEST extends CoordinationRequest, RESPONSE extends CoordinationResponse>
RESPONSE sendRequestToCoordinator(REQUEST request) {
try {
CompletableFuture<CoordinationRespons