|
24 | 24 | import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher;
|
25 | 25 | import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset;
|
26 | 26 | import org.apache.seatunnel.connectors.cdc.base.source.reader.external.JdbcSourceFetchTaskContext;
|
27 |
| -import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; |
28 |
| -import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; |
29 | 27 | import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase;
|
30 |
| -import org.apache.seatunnel.connectors.cdc.debezium.EmbeddedDatabaseHistory; |
31 | 28 | import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.config.MySqlSourceConfig;
|
32 | 29 | import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source.offset.BinlogOffset;
|
33 | 30 | import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlConnectionUtils;
|
34 | 31 | import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlUtils;
|
35 | 32 |
|
36 |
| -import org.apache.kafka.connect.data.SchemaAndValue; |
37 | 33 | import org.apache.kafka.connect.data.Struct;
|
38 | 34 | import org.apache.kafka.connect.source.SourceRecord;
|
39 | 35 |
|
|
66 | 62 | import io.debezium.relational.Table;
|
67 | 63 | import io.debezium.relational.TableId;
|
68 | 64 | import io.debezium.relational.Tables;
|
69 |
| -import io.debezium.relational.history.TableChanges; |
70 | 65 | import io.debezium.schema.DataCollectionId;
|
71 | 66 | import io.debezium.schema.TopicSelector;
|
72 | 67 | import io.debezium.util.Collect;
|
|
75 | 70 | import java.io.IOException;
|
76 | 71 | import java.sql.SQLException;
|
77 | 72 | import java.time.Instant;
|
78 |
| -import java.util.ArrayList; |
79 |
| -import java.util.Collections; |
80 |
| -import java.util.Iterator; |
81 | 73 | import java.util.List;
|
82 | 74 | import java.util.Map;
|
83 | 75 | import java.util.Optional;
|
@@ -118,7 +110,7 @@ public MySqlSourceFetchTaskContext(
|
118 | 110 |
|
119 | 111 | @Override
|
120 | 112 | public void configure(SourceSplitBase sourceSplitBase) {
|
121 |
| - registerDatabaseHistory(sourceSplitBase); |
| 113 | + super.registerDatabaseHistory(sourceSplitBase, connection); |
122 | 114 |
|
123 | 115 | // initial stateful objects
|
124 | 116 | final MySqlConnectorConfig connectorConfig = getDbzConnectorConfig();
|
@@ -385,49 +377,6 @@ private void validateAndLoadDatabaseHistory(
|
385 | 377 | schema.recover(Offsets.of(mySqlPartition, offset));
|
386 | 378 | }
|
387 | 379 |
|
388 |
| - private void registerDatabaseHistory(SourceSplitBase sourceSplitBase) { |
389 |
| - List<TableChanges.TableChange> engineHistory = new ArrayList<>(); |
390 |
| - // TODO: support save table schema |
391 |
| - if (sourceSplitBase instanceof SnapshotSplit) { |
392 |
| - SnapshotSplit snapshotSplit = (SnapshotSplit) sourceSplitBase; |
393 |
| - engineHistory.add( |
394 |
| - dataSourceDialect.queryTableSchema(connection, snapshotSplit.getTableId())); |
395 |
| - } else { |
396 |
| - IncrementalSplit incrementalSplit = (IncrementalSplit) sourceSplitBase; |
397 |
| - Map<TableId, byte[]> historyTableChanges = incrementalSplit.getHistoryTableChanges(); |
398 |
| - for (TableId tableId : incrementalSplit.getTableIds()) { |
399 |
| - if (historyTableChanges != null && historyTableChanges.containsKey(tableId)) { |
400 |
| - SchemaAndValue schemaAndValue = |
401 |
| - jsonConverter.toConnectData("topic", historyTableChanges.get(tableId)); |
402 |
| - Struct deserializedStruct = (Struct) schemaAndValue.value(); |
403 |
| - |
404 |
| - TableChanges tableChanges = |
405 |
| - tableChangeSerializer.deserialize( |
406 |
| - Collections.singletonList(deserializedStruct), false); |
407 |
| - |
408 |
| - Iterator<TableChanges.TableChange> iterator = tableChanges.iterator(); |
409 |
| - TableChanges.TableChange tableChange = null; |
410 |
| - while (iterator.hasNext()) { |
411 |
| - if (tableChange != null) { |
412 |
| - throw new IllegalStateException( |
413 |
| - "The table changes should only have one element"); |
414 |
| - } |
415 |
| - tableChange = iterator.next(); |
416 |
| - } |
417 |
| - engineHistory.add(tableChange); |
418 |
| - continue; |
419 |
| - } |
420 |
| - engineHistory.add(dataSourceDialect.queryTableSchema(connection, tableId)); |
421 |
| - } |
422 |
| - } |
423 |
| - |
424 |
| - EmbeddedDatabaseHistory.registerHistory( |
425 |
| - sourceConfig |
426 |
| - .getDbzConfiguration() |
427 |
| - .getString(EmbeddedDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), |
428 |
| - engineHistory); |
429 |
| - } |
430 |
| - |
431 | 380 | /** A subclass implementation of {@link MySqlTaskContext} which reuses one BinaryLogClient. */
|
432 | 381 | public class MySqlTaskContextImpl extends MySqlTaskContext {
|
433 | 382 |
|
|
0 commit comments