|
| 1 | +package org.apache.flink.cdc.runtime.operators.sink; |
| 2 | + |
| 3 | +import org.apache.flink.cdc.common.event.ChangeEvent; |
| 4 | +import org.apache.flink.cdc.common.event.CreateTableEvent; |
| 5 | +import org.apache.flink.cdc.common.event.Event; |
| 6 | +import org.apache.flink.cdc.common.event.FlushEvent; |
| 7 | +import org.apache.flink.cdc.common.event.TableId; |
| 8 | +import org.apache.flink.cdc.common.schema.Schema; |
| 9 | +import org.apache.flink.runtime.jobgraph.OperatorID; |
| 10 | +import org.apache.flink.runtime.state.StateInitializationContext; |
| 11 | +import org.apache.flink.runtime.state.StateSnapshotContext; |
| 12 | +import org.apache.flink.streaming.api.graph.StreamConfig; |
| 13 | +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; |
| 14 | +import org.apache.flink.streaming.api.operators.BoundedOneInput; |
| 15 | +import org.apache.flink.streaming.api.operators.ChainingStrategy; |
| 16 | +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; |
| 17 | +import org.apache.flink.streaming.api.operators.Output; |
| 18 | +import org.apache.flink.streaming.api.watermark.Watermark; |
| 19 | +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; |
| 20 | +import org.apache.flink.streaming.runtime.tasks.StreamTask; |
| 21 | +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; |
| 22 | + |
| 23 | +import java.util.HashSet; |
| 24 | +import java.util.Optional; |
| 25 | +import java.util.Set; |
| 26 | + |
| 27 | +/** |
| 28 | + * The DataSinkOperatorAdapter class acts as an adapter for testing the core schema evolution |
| 29 | + * process in both {@link DataSinkWriterOperator} and {@link DataSinkFunctionOperator}. |
| 30 | + */ |
| 31 | +public class DataSinkOperatorAdapter extends AbstractStreamOperator<Event> |
| 32 | + implements OneInputStreamOperator<Event, Event>, BoundedOneInput { |
| 33 | + |
| 34 | + private SchemaEvolutionClient schemaEvolutionClient; |
| 35 | + |
| 36 | + private final OperatorID schemaOperatorID; |
| 37 | + |
| 38 | + /** A set of {@link TableId} that already processed {@link CreateTableEvent}. */ |
| 39 | + private final Set<TableId> processedTableIds; |
| 40 | + |
| 41 | + public DataSinkOperatorAdapter() { |
| 42 | + this.schemaOperatorID = new OperatorID(); |
| 43 | + this.processedTableIds = new HashSet<>(); |
| 44 | + this.chainingStrategy = ChainingStrategy.ALWAYS; |
| 45 | + } |
| 46 | + |
| 47 | + @Override |
| 48 | + public void setup( |
| 49 | + StreamTask<?, ?> containingTask, |
| 50 | + StreamConfig config, |
| 51 | + Output<StreamRecord<Event>> output) { |
| 52 | + super.setup(containingTask, config, output); |
| 53 | + schemaEvolutionClient = |
| 54 | + new SchemaEvolutionClient( |
| 55 | + containingTask.getEnvironment().getOperatorCoordinatorEventGateway(), |
| 56 | + schemaOperatorID); |
| 57 | + } |
| 58 | + |
| 59 | + @Override |
| 60 | + public void open() throws Exception {} |
| 61 | + |
| 62 | + @Override |
| 63 | + public void initializeState(StateInitializationContext context) throws Exception { |
| 64 | + schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask()); |
| 65 | + } |
| 66 | + |
| 67 | + @Override |
| 68 | + public void snapshotState(StateSnapshotContext context) {} |
| 69 | + |
| 70 | + @Override |
| 71 | + public void processWatermark(Watermark mark) {} |
| 72 | + |
| 73 | + @Override |
| 74 | + public void processWatermarkStatus(WatermarkStatus watermarkStatus) {} |
| 75 | + |
| 76 | + @Override |
| 77 | + public void processElement(StreamRecord<Event> element) throws Exception { |
| 78 | + Event event = element.getValue(); |
| 79 | + |
| 80 | + // FlushEvent triggers flush |
| 81 | + if (event instanceof FlushEvent) { |
| 82 | + handleFlushEvent(((FlushEvent) event)); |
| 83 | + return; |
| 84 | + } |
| 85 | + |
| 86 | + // CreateTableEvent marks the table as processed directly |
| 87 | + if (event instanceof CreateTableEvent) { |
| 88 | + processedTableIds.add(((CreateTableEvent) event).tableId()); |
| 89 | + // replace FlinkWriterOperator/StreamSink and emit the event for testing |
| 90 | + output.collect(element); |
| 91 | + return; |
| 92 | + } |
| 93 | + |
| 94 | + // Check if the table is processed before emitting all other events, because we have to make |
| 95 | + // sure that sink have a view of the full schema before processing any change events, |
| 96 | + // including schema changes. |
| 97 | + ChangeEvent changeEvent = (ChangeEvent) event; |
| 98 | + if (!processedTableIds.contains(changeEvent.tableId())) { |
| 99 | + emitLatestSchema(changeEvent.tableId()); |
| 100 | + processedTableIds.add(changeEvent.tableId()); |
| 101 | + } |
| 102 | + processedTableIds.add(changeEvent.tableId()); |
| 103 | + output.collect(element); |
| 104 | + } |
| 105 | + |
| 106 | + @Override |
| 107 | + public void prepareSnapshotPreBarrier(long checkpointId) {} |
| 108 | + |
| 109 | + @Override |
| 110 | + public void close() throws Exception {} |
| 111 | + |
| 112 | + @Override |
| 113 | + public void endInput() {} |
| 114 | + |
| 115 | + // ----------------------------- Helper functions ------------------------------- |
| 116 | + |
| 117 | + private void handleFlushEvent(FlushEvent event) throws Exception { |
| 118 | + // omit copySinkWriter/userFunction flush from testing |
| 119 | + if (!processedTableIds.contains(event.getTableId()) && !event.getIsForCreateTableEvent()) { |
| 120 | + LOG.info("Table {} has not been processed", event.getTableId()); |
| 121 | + emitLatestSchema(event.getTableId()); |
| 122 | + processedTableIds.add(event.getTableId()); |
| 123 | + } |
| 124 | + schemaEvolutionClient.notifyFlushSuccess( |
| 125 | + getRuntimeContext().getIndexOfThisSubtask(), event.getTableId()); |
| 126 | + } |
| 127 | + |
| 128 | + private void emitLatestSchema(TableId tableId) throws Exception { |
| 129 | + Optional<Schema> schema = schemaEvolutionClient.getLatestEvolvedSchema(tableId); |
| 130 | + if (schema.isPresent()) { |
| 131 | + // request and process CreateTableEvent because SinkWriter need to retrieve |
| 132 | + // Schema to deserialize RecordData after resuming job. |
| 133 | + output.collect(new StreamRecord<>(new CreateTableEvent(tableId, schema.get()))); |
| 134 | + processedTableIds.add(tableId); |
| 135 | + } else { |
| 136 | + throw new RuntimeException( |
| 137 | + "Could not find schema message from SchemaRegistry for " + tableId); |
| 138 | + } |
| 139 | + } |
| 140 | +} |
0 commit comments